From 738345df3b2f4283fe189dd898b7e852762b6e0e Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 28 Mar 2019 14:05:34 +0300 Subject: [PATCH 01/24] IGNITE-11584 Batch write to data pages. Stub for IgniteTree.invokeAll. --- .../jmh/pagemem/JmhBatchUpdatesBenchmark.java | 421 ++++++++++++++++ .../apache/ignite/IgniteSystemProperties.java | 3 + .../processors/cache/CacheMapEntries.java | 469 ++++++++++++++++++ .../processors/cache/CacheMetricsImpl.java | 9 + .../processors/cache/GridCacheMapEntry.java | 166 ++++--- .../cache/IgniteCacheOffheapManager.java | 31 ++ .../cache/IgniteCacheOffheapManagerImpl.java | 77 +++ .../preloader/GridDhtPartitionDemander.java | 176 +++++-- .../cache/persistence/DataStructure.java | 22 + .../persistence/GridCacheOffheapManager.java | 13 +- .../IgniteCacheDatabaseSharedManager.java | 2 +- .../cache/persistence/RowStore.java | 20 + .../freelist/AbstractFreeList.java | 250 +++++++++- .../cache/persistence/freelist/FreeList.java | 7 + .../cache/persistence/tree/BPlusTree.java | 5 + .../tree/io/AbstractDataPageIO.java | 83 +++- .../persistence/tree/util/PageHandler.java | 96 ++++ .../processors/cache/tree/CacheDataTree.java | 87 ++++ .../ignite/internal/util/IgniteTree.java | 36 ++ .../FreeListPreloadWithBatchUpdatesTest.java | 359 ++++++++++++++ 20 files changed, 2211 insertions(+), 121 deletions(-) create mode 100644 modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListPreloadWithBatchUpdatesTest.java diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java new file mode 100644 index 0000000000000..225e702bfea74 --- /dev/null +++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java @@ -0,0 +1,421 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.benchmarks.jmh.pagemem; + +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader; +import org.apache.ignite.logger.NullLogger; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +/** + * Batch updates in pagemem through preloader. + * + * todo benchmark for internal testing purposes. + */ +@BenchmarkMode(Mode.AverageTime) +@Fork(value = 1, jvmArgsAppend = {"-Xms3g", "-Xmx3g", "-server", "-XX:+AggressiveOpts", "-XX:MaxMetaspaceSize=256m"}) +@Measurement(iterations = 11) +@OutputTimeUnit(TimeUnit.MICROSECONDS) +@State(Scope.Benchmark) +@Threads(1) +@Warmup(iterations = 15) +public class JmhBatchUpdatesBenchmark { + /** */ + private static final long DEF_REG_SIZE = 3 * 1024 * 1024 * 1024L; + + /** */ + private static final int BATCH_SIZE = 500; + + /** */ + private static final String REG_BATCH = "batch-region"; + + /** */ + private static final String REG_SINGLE = "single-region"; + + /** */ + private static final String CACHE_BATCH = "batch"; + + /** */ + private static final String CACHE_SINGLE = "single"; + + /** */ + private static final String NODE_NAME = "srv0"; + + /** */ + private static int iteration = 0; + + /** */ + public enum OBJECT_SIZE_RANGE { + /** */ + r0_4(0, 4), + + /** */ + r4_16(4, 16), + + /** */ + r16_64(16, 64), + + /** */ + r100_200(100, 200), + + /** */ + r200_500(200, 500), + + /** */ + r500_800(500, 800), + + /** */ + r800_1200(800, 1200), + + /** */ + r2000_3000(2_000, 3_000), + + /** */ + r1000_8000(1_000, 8_000), + + /** Large objects only. */ + r4000_16000(4_000, 16_000), + + /** Mixed objects, mostly large objects. */ + r100_32000(100, 32_000); + + /** */ + private final int min; + + /** */ + private final int max; + + /** */ + OBJECT_SIZE_RANGE(int min, int max) { + this.min = min; + this.max = max; + } + } + + /** + * Create Ignite configuration. + * + * @return Ignite configuration. + */ + private IgniteConfiguration getConfiguration(String cfgName) { + IgniteConfiguration cfg = new IgniteConfiguration(); + + cfg.setGridLogger(new NullLogger()); + + cfg.setIgniteInstanceName(cfgName); + + DataRegionConfiguration reg1 = new DataRegionConfiguration(); + reg1.setInitialSize(DEF_REG_SIZE); + reg1.setMaxSize(DEF_REG_SIZE); + reg1.setName(REG_BATCH); + + DataRegionConfiguration reg2 = new DataRegionConfiguration(); + reg2.setInitialSize(DEF_REG_SIZE); + reg2.setMaxSize(DEF_REG_SIZE); + reg2.setName(REG_SINGLE); + + DataStorageConfiguration storeCfg = new DataStorageConfiguration(); + + storeCfg.setDataRegionConfigurations(reg1, reg2); + + cfg.setDataStorageConfiguration(storeCfg); + + cfg.setCacheConfiguration(ccfg(false), ccfg(true)); + + return cfg; + } + + /** + * @return Cache configuration. + */ + private CacheConfiguration ccfg(boolean batch) { + return new CacheConfiguration(batch ? CACHE_BATCH : CACHE_SINGLE) + .setAffinity(new RendezvousAffinityFunction(false, 1)) + .setCacheMode(CacheMode.REPLICATED) + .setAtomicityMode(CacheAtomicityMode.ATOMIC) + .setDataRegionName(batch ? REG_BATCH : REG_SINGLE); + } + + /** + * Test single updates. + * + * @param data Data that will be preloaded. + * @param preloader Data preloader. + */ + @Benchmark + public void checkSingle(Data data, Preloader preloader) throws IgniteCheckedException { + preloader.demanderSingle.preloadEntriesSingle(null, 0, data.singleData, data.cctxSingle.topology().readyTopologyVersion()); + } + + /** + * Test batch updates. + * + * @param data Data that will be preloaded. + * @param preloader Data preloader. + */ + @Benchmark + public void checkBatch(Data data, Preloader preloader) throws IgniteCheckedException { + preloader.demanderBatch.preloadEntriesBatch(null, 0, data.batchData, data.cctxBatch.topology().readyTopologyVersion()); + } + + /** + * Start 2 servers and 1 client. + */ + @Setup(Level.Trial) + public void setup() { + Ignition.start(getConfiguration(NODE_NAME)); + } + + /** + * Stop all grids after tests. + */ + @TearDown(Level.Trial) + public void tearDown() { + Ignition.stopAll(true); + } + + /** + * Create streamer on client cache. + */ + @State(Scope.Benchmark) + public static class Preloader { + /** */ + final GridDhtPartitionDemander demanderBatch = demander(CACHE_BATCH); + + /** */ + final GridDhtPartitionDemander demanderSingle = demander(CACHE_SINGLE); + + /** */ + GridDhtPartitionDemander demander(String name) { + GridCacheContext cctx = ((IgniteEx)Ignition.ignite(NODE_NAME)).cachex(name).context(); + + GridDhtPreloader preloader = (GridDhtPreloader)cctx.group().preloader(); + + return getFieldValue(preloader, "demander"); + } + + /** + * Get object field value via reflection. + * + * @param obj Object or class to get field value from. + * @param fieldNames Field names to get value for: obj->field1->field2->...->fieldN. + * @param Expected field class. + * @return Field value. + * @throws IgniteException In case of error. + */ + public static T getFieldValue(Object obj, String... fieldNames) throws IgniteException { + assert obj != null; + assert fieldNames != null; + assert fieldNames.length >= 1; + + try { + for (String fieldName : fieldNames) { + Class cls = obj instanceof Class ? (Class)obj : obj.getClass(); + + try { + obj = findField(cls, obj, fieldName); + } + catch (NoSuchFieldException e) { + throw new RuntimeException(e); + } + } + + return (T)obj; + } + catch (IllegalAccessException e) { + throw new IgniteException("Failed to get object field [obj=" + obj + + ", fieldNames=" + Arrays.toString(fieldNames) + ']', e); + } + } + + /** + * @param cls Class for searching. + * @param obj Target object. + * @param fieldName Field name for search. + * @return Field from object if it was found. + */ + private static Object findField(Class cls, Object obj, + String fieldName) throws NoSuchFieldException, IllegalAccessException { + // Resolve inner field. + Field field = cls.getDeclaredField(fieldName); + + boolean accessible = field.isAccessible(); + + if (!accessible) + field.setAccessible(true); + + return field.get(obj); + } + } + + /** + * Prepare and clean collection with streaming data. + */ + @State(Scope.Thread) + public static class Data { + /** */ + @Param + private OBJECT_SIZE_RANGE range; + + /** */ + private int[] sizes; + + /** */ + Collection batchData = new ArrayList<>(BATCH_SIZE); + + /** */ + Collection singleData = new ArrayList<>(BATCH_SIZE); + + /** */ + GridCacheContext cctxBatch = ((IgniteEx)Ignition.ignite(NODE_NAME)).cachex(CACHE_BATCH).context(); + + /** */ + GridCacheContext cctxSingle = ((IgniteEx)Ignition.ignite(NODE_NAME)).cachex(CACHE_SINGLE).context(); + + /** */ + @Setup(Level.Trial) + public void setup() { + sizes = sizes(range.min, range.max, BATCH_SIZE); + } + + /** + * Prepare collection. + */ + @Setup(Level.Iteration) + public void prepare() { + int iter = iteration++; + + int off = iter * BATCH_SIZE; + + batchData = prepareBatch(cctxBatch, off, BATCH_SIZE, sizes); + singleData = prepareBatch(cctxSingle, off, BATCH_SIZE, sizes); + } + + /** + * Clean collection after each test. + */ + @TearDown(Level.Iteration) + public void cleanCollection() { + batchData = null; + singleData = null; + } + + /** */ + int[] sizes(int minObjSize, int maxObjSize, int batchSize) { + int sizes[] = new int[batchSize]; + int minSize = maxObjSize; + int maxSize = minObjSize; + + int delta = maxObjSize - minObjSize; + + for (int i = 0; i < batchSize; i++) { + int size = sizes[i] = minObjSize + (delta > 0 ? ThreadLocalRandom.current().nextInt(delta) : 0); + + if (size < minSize) + minSize = size; + + if (size > maxSize) + maxSize = size; + } + + return sizes; + } + + /** + * Generates rebalance info objects. + * + * @param cctx Cache context. + * @param off Offset. + * @param cnt Count. + * @param sizes Object sizes. + * @return List of generated objects. + */ + private List prepareBatch(GridCacheContext cctx, int off, int cnt, int[] sizes) { + List infos = new ArrayList<>(); + + for (int i = off; i < off + cnt; i++) { + int size = sizes[i - off]; + + KeyCacheObject key = cctx.toCacheKeyObject(i); + CacheObject val = cctx.toCacheObject(new byte[size]); + + GridCacheEntryInfo info = new GridCacheEntryInfo(); + info.key(key); + info.value(val); + info.cacheId(cctx.cacheId()); + info.version(cctx.shared().versions().startVersion()); + + infos.add(info); + } + + return infos; + } + } + + /** + * Run benchmark. + * + * @param args Args. + */ + public static void main(String[] args) throws RunnerException { + final Options options = new OptionsBuilder() + .include(JmhBatchUpdatesBenchmark.class.getSimpleName()) + .build(); + + new Runner(options).run(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index f6eb650a092ad..8535e3763390b 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -1097,6 +1097,9 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_DISCOVERY_DISABLE_CACHE_METRICS_UPDATE = "IGNITE_DISCOVERY_DISABLE_CACHE_METRICS_UPDATE"; + /** */ + public static final String IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE = "IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE"; + /** * Maximum number of different partitions to be extracted from between expression within sql query. * In case of limit exceeding all partitions will be used. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java new file mode 100644 index 0000000000000..a7e583c398daf --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -0,0 +1,469 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.OffheapInvokeAllClosure; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.tree.DataRow; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.dr.GridDrType; +import org.apache.ignite.internal.util.IgniteTree; +import org.apache.ignite.internal.util.typedef.T3; +import org.apache.ignite.internal.util.typedef.internal.CU; + +import static org.apache.ignite.internal.util.IgniteTree.OperationType.NOOP; +import static org.apache.ignite.internal.util.IgniteTree.OperationType.PUT; +import static org.apache.ignite.internal.util.IgniteTree.OperationType.REMOVE; + +/** + * Batch of cache map entries. + */ +public class CacheMapEntries { + /** */ + private final GridDhtLocalPartition part; + + /** */ + private final GridCacheContext cctx; + + /** */ + private final AffinityTopologyVersion topVer; + + /** */ + private final boolean preload; + + /** */ + private final LinkedHashMap infos = new LinkedHashMap<>(); + + /** */ + private Set skipped = new HashSet<>(); + + /** */ + private boolean ordered = true; + + /** */ + private KeyCacheObject lastKey; + + /** */ + public CacheMapEntries(AffinityTopologyVersion topVer, int partId, GridCacheContext cctx, boolean preload) { + this.topVer = topVer; + this.cctx = cctx; + this.preload = preload; + this.part = cctx.topology().localPartition(partId, topVer, true, true); + } + + /** */ + public void add(KeyCacheObject key, CacheObject val, long expTime, long ttl, GridCacheVersion ver, GridDrType drType) { + if (lastKey != null && ordered && lastKey.hashCode() >= key.hashCode()) + ordered = false; + + CacheMapEntryInfo old = + infos.put(lastKey = key, new CacheMapEntryInfo(this, val, expTime, ttl, ver, drType)); + + assert old == null || GridCacheMapEntry.ATOMIC_VER_COMPARATOR.compare(old.version(), ver) < 0 : + "Version order mismatch: prev=" + old.version() + ", current=" + ver; + } + + /** */ + public GridDhtLocalPartition part() { + return part; + } + + /** */ + public GridCacheContext context() { + return cctx; + } + + /** */ + public Set keys() { + return infos.keySet(); + } + + /** + * @return Count of batch entries. + */ + public int size() { + return infos.size() - skipped.size(); + } + + /** + * @return Off heap update closure. + */ + public OffheapInvokeAllClosure offheapUpdateClosure() { + return new UpdateAllClosure(this, cctx); + } + + /** */ + public void lock() { + List locked = new ArrayList<>(infos.size()); + + while (true) { + for (Map.Entry e : infos.entrySet()) { + GridDhtCacheEntry entry = (GridDhtCacheEntry)cctx.cache().entryEx(e.getKey(), topVer); + + locked.add(entry); + + e.getValue().cacheEntry(entry); + } + + boolean retry = false; + + for (int i = 0; i < locked.size(); i++) { + GridCacheMapEntry entry = locked.get(i); + + if (entry == null) + continue; + + entry.lockEntry(); + + if (entry.obsolete()) { + // Unlock all locked. + for (int j = 0; j <= i; j++) { + if (locked.get(j) != null) + locked.get(j).unlockEntry(); + } + + // Clear entries. + locked.clear(); + + // Retry. + retry = true; + + break; + } + } + + if (!retry) + return; + } + } + + /** + * Releases java-level locks on cache entries. + */ + public void unlock() { + // Process deleted entries before locks release. + // todo + assert cctx.deferredDelete() : this; + + // Entries to skip eviction manager notification for. + // Enqueue entries while holding locks. + int size = infos.size(); + + try { + for (Map.Entry e : infos.entrySet()) { + KeyCacheObject key = e.getKey(); + GridCacheMapEntry entry = e.getValue().cacheEntry(); + + if (skipped.contains(key)) + continue; + + if (entry != null && entry.deleted()) + skipped.add(entry.key()); + + try { + e.getValue().updateCacheEntry(); + } catch (IgniteCheckedException ex) { + skipped.add(entry.key()); + } + } + } + finally { + // At least RuntimeException can be thrown by the code above when GridCacheContext is cleaned and there is + // an attempt to use cleaned resources. + // That's why releasing locks in the finally block.. + for (CacheMapEntryInfo info : infos.values()) { + GridCacheMapEntry entry = info.cacheEntry(); + + if (entry != null) + entry.unlockEntry(); + } + } + + // Try evict partitions. + for (CacheMapEntryInfo info : infos.values()) { + GridDhtCacheEntry entry = info.cacheEntry(); + + if (entry != null) + entry.onUnlock(); + } + + if (skipped.size() == size) + // Optimization. + return; + + // Must touch all entries since update may have deleted entries. + // Eviction manager will remove empty entries. + for (CacheMapEntryInfo info : infos.values()) { + GridCacheMapEntry entry = info.cacheEntry(); + + if (entry != null && !skipped.contains(entry.key())) + entry.touch(); + } + } + + /** */ + public void onRemove(KeyCacheObject key) { + skipped.add(key); + } + + /** */ + boolean preload() { + return preload; + } + + /** */ + boolean skip(KeyCacheObject key) { + return skipped.contains(key); + } + + /** */ + private static class UpdateAllClosure implements OffheapInvokeAllClosure { + /** */ + private final List> finalOps; + + /** */ + private final int cacheId; + + /** */ + private final CacheMapEntries entries; + + /** */ + public UpdateAllClosure(CacheMapEntries entries, GridCacheContext cctx) { + this.entries = entries; + finalOps = new ArrayList<>(entries.size()); + cacheId = cctx.group().storeCacheIdInDataPage() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + } + + /** {@inheritDoc} */ + @Override public void call(Collection rows) throws IgniteCheckedException { + assert rows.size() == entries.size() : "size mismatch, expect=" + entries.size() + ", input=" + rows.size(); + + List newRows = new ArrayList<>(8); + + int partId = entries.part().id(); + GridCacheContext cctx = entries.context(); + + Iterator rowsIter = rows.iterator(); + + for (Map.Entry e : entries.infos.entrySet()) { + KeyCacheObject key = e.getKey(); + CacheMapEntryInfo newRowInfo = e.getValue(); + CacheDataRow oldRow = rowsIter.next(); + + try { + if (newRowInfo.needUpdate(oldRow)) { + CacheObject val = newRowInfo.value(); + + if (val != null) { + if (oldRow != null) { + CacheDataRow newRow = cctx.offheap().dataStore(entries.part()).createRow( + cctx, + key, + newRowInfo.value(), + newRowInfo.version(), + newRowInfo.expireTime(), + oldRow); + + finalOps.add(new T3<>(oldRow.link() == newRow.link() ? NOOP : PUT, oldRow, newRow)); + } + else { + CacheObjectContext coCtx = cctx.cacheObjectContext(); + + val.valueBytes(coCtx); + key.valueBytes(coCtx); + + if (key.partition() == -1) + key.partition(partId); + + DataRow newRow = new DataRow(key, val, newRowInfo.version(), partId, + newRowInfo.expireTime(), cacheId); + + newRows.add(newRow); + + finalOps.add(new T3<>(PUT, oldRow, newRow)); + } + } + else { + // todo we should pass key somehow to remove old row + // todo (in particular case oldRow should not contain key) + DataRow newRow = new DataRow(key, null, null, 0, 0, cacheId); + + finalOps.add(new T3<>(oldRow != null ? REMOVE : NOOP, oldRow, newRow)); + } + } + } + catch (GridCacheEntryRemovedException ex) { + entries.onRemove(key); + } + } + + if (!newRows.isEmpty()) { + // cctx.shared().database().ensureFreeSpace(cctx.dataRegion()); + + cctx.offheap().dataStore(entries.part()).rowStore(). + addRows(newRows, cctx.group().statisticsHolderData()); + + if (cacheId == CU.UNDEFINED_CACHE_ID) { + // Set cacheId before write keys into tree. + for (DataRow row : newRows) + row.cacheId(cctx.cacheId()); + + } + } + } + + /** {@inheritDoc} */ + @Override public Collection> result() { + return finalOps; + } + + /** {@inheritDoc} */ + @Override public boolean fastpath() { + return entries.ordered; + } + } + + /** */ + private static class CacheMapEntryInfo { + /** */ + private final CacheMapEntries batch; + + /** */ + private final CacheObject val; + + /** */ + private final long expireTime; + + /** */ + private final long ttl; + + /** */ + private final GridCacheVersion ver; + + /** */ + private final GridDrType drType; + + /** */ + private GridDhtCacheEntry entry; + + /** */ + private boolean update; + + /** */ + public CacheMapEntryInfo( + CacheMapEntries batch, + CacheObject val, + long expireTime, + long ttl, + GridCacheVersion ver, + GridDrType drType + ) { + this.batch = batch; + this.val = val; + this.expireTime = expireTime; + this.ver = ver; + this.drType = drType; + this.ttl = ttl; + } + + /** + * @return Version. + */ + public GridCacheVersion version() { + return ver; + } + + /** + * @return Value. + */ + public CacheObject value() { + return val; + } + + /** + * @return Expire time. + */ + public long expireTime() { + return expireTime; + } + + /** + * @param entry Cache entry. + */ + public void cacheEntry(GridDhtCacheEntry entry) { + this.entry = entry; + } + + /** + * @return Cache entry. + */ + public GridDhtCacheEntry cacheEntry() { + return entry; + } + + /** */ + public void updateCacheEntry() throws IgniteCheckedException { + if (!update) + return; + + entry.finishInitialUpdate(val, expireTime, ttl, ver, batch.topVer, drType, null, batch.preload); + } + + /** */ + public boolean needUpdate(CacheDataRow row) throws GridCacheEntryRemovedException { + GridCacheVersion currVer = row != null ? row.version() : entry.version(); + + GridCacheContext cctx = batch.context(); + + boolean isStartVer = cctx.versions().isStartVersion(currVer); + + boolean update0; + + if (cctx.group().persistenceEnabled()) { + if (!isStartVer) { + if (cctx.atomic()) + update0 = GridCacheMapEntry.ATOMIC_VER_COMPARATOR.compare(currVer, version()) < 0; + else + update0 = currVer.compareTo(version()) < 0; + } + else + update0 = true; + } + else + update0 = (isStartVer && row == null); + + update0 |= (!batch.preload() && entry.deletedUnlocked()); + + update = update0; + + return update0; + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java index 8ce21c59de3d1..aa88b2e1bb04e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java @@ -1163,6 +1163,15 @@ public void onRebalanceKeyReceived() { rebalancingKeysRate.onHit(); } + /** + * Rebalance entry store callback. + */ + public void onRebalanceKeysReceived(long batchSize) { + rebalancedKeys.addAndGet(batchSize); + + rebalancingKeysRate.onHits(batchSize); + } + /** * Rebalance supply message callback. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 352ca0c6e662b..39c9a6cfb1624 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -3431,75 +3431,7 @@ else if (val == null) } if (update) { - update(val, expTime, ttl, ver, true); - - boolean skipQryNtf = false; - - if (val == null) { - skipQryNtf = true; - - if (cctx.deferredDelete() && !deletedUnlocked() && !isInternal()) - deletedUnlocked(true); - } - else if (deletedUnlocked()) - deletedUnlocked(false); - - long updateCntr = 0; - - if (!preload) - updateCntr = nextPartitionCounter(topVer, true, null); - - if (walEnabled) { - if (cctx.mvccEnabled()) { - cctx.shared().wal().log(new MvccDataRecord(new MvccDataEntry( - cctx.cacheId(), - key, - val, - val == null ? DELETE : GridCacheOperation.CREATE, - null, - ver, - expireTime, - partition(), - updateCntr, - mvccVer == null ? MvccUtils.INITIAL_VERSION : mvccVer - ))); - } else { - cctx.shared().wal().log(new DataRecord(new DataEntry( - cctx.cacheId(), - key, - val, - val == null ? DELETE : GridCacheOperation.CREATE, - null, - ver, - expireTime, - partition(), - updateCntr - ))); - } - } - - drReplicate(drType, val, ver, topVer); - - if (!skipQryNtf) { - cctx.continuousQueries().onEntryUpdated( - key, - val, - null, - this.isInternal() || !this.context().userCache(), - this.partition(), - true, - preload, - updateCntr, - null, - topVer); - } - - onUpdateFinished(updateCntr); - - if (!fromStore && cctx.store().isLocal()) { - if (val != null) - cctx.store().put(null, key, val, ver); - } + finishInitialUpdate(val, expireTime, ttl, ver, topVer, drType, mvccVer, preload); return true; } @@ -3526,6 +3458,102 @@ else if (deletedUnlocked()) } } + /** + * todo explain this and remove code duplication + * @param val New value. + * @param expireTime Expiration time. + * @param ttl Time to live. + * @param ver Version to use. + * @param topVer Topology version. + * @param drType DR type. + * @param mvccVer Mvcc version. + * @param preload Flag indicating whether entry is being preloaded. + * @throws IgniteCheckedException In case of error. + */ + protected void finishInitialUpdate( + @Nullable CacheObject val, + long expireTime, + long ttl, + GridCacheVersion ver, + AffinityTopologyVersion topVer, + GridDrType drType, + MvccVersion mvccVer, + boolean preload + ) throws IgniteCheckedException { + boolean fromStore = false; + boolean walEnabled = !cctx.isNear() && cctx.group().persistenceEnabled() && cctx.group().walEnabled(); + + update(val, expireTime, ttl, ver, true); + + boolean skipQryNtf = false; + + if (val == null) { + skipQryNtf = true; + + if (cctx.deferredDelete() && !deletedUnlocked() && !isInternal()) + deletedUnlocked(true); + } + else if (deletedUnlocked()) + deletedUnlocked(false); + + long updateCntr = 0; + + if (!preload) + updateCntr = nextPartitionCounter(topVer, true, null); + + if (walEnabled) { + if (cctx.mvccEnabled()) { + cctx.shared().wal().log(new MvccDataRecord(new MvccDataEntry( + cctx.cacheId(), + key, + val, + val == null ? DELETE : GridCacheOperation.CREATE, + null, + ver, + expireTime, + partition(), + updateCntr, + mvccVer == null ? MvccUtils.INITIAL_VERSION : mvccVer + ))); + } else { + cctx.shared().wal().log(new DataRecord(new DataEntry( + cctx.cacheId(), + key, + val, + val == null ? DELETE : GridCacheOperation.CREATE, + null, + ver, + expireTime, + partition(), + updateCntr + ))); + } + } + + drReplicate(drType, val, ver, topVer); + + if (!skipQryNtf) { + cctx.continuousQueries().onEntryUpdated( + key, + val, + null, + this.isInternal() || !this.context().userCache(), + this.partition(), + true, + preload, + updateCntr, + null, + topVer); + } + + onUpdateFinished(updateCntr); + + if (!fromStore && cctx.store().isLocal()) { + if (val != null) + cctx.store().put(null, key, val, ver); + } + } + /** * @param cntr Updated partition counter. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java index b7e8ec717fc38..19c6eaa671ef2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache; +import java.util.Collection; import java.util.List; import java.util.Map; import javax.cache.Cache; @@ -47,6 +48,7 @@ import org.apache.ignite.internal.util.lang.GridIterator; import org.apache.ignite.internal.util.lang.IgniteInClosure2X; import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; /** @@ -188,6 +190,20 @@ public boolean expire(GridCacheContext cctx, IgniteInClosure2X keys, + GridDhtLocalPartition part, + OffheapInvokeAllClosure c + ) throws IgniteCheckedException; + /** * @param cctx Cache context. * @param key Key. @@ -579,6 +595,13 @@ interface OffheapInvokeClosure extends IgniteTree.InvokeClosure { @Nullable public CacheDataRow oldRow(); } + /** + * + */ + interface OffheapInvokeAllClosure extends IgniteTree.InvokeAllClosure { +// boolean preload(); + } + /** * */ @@ -861,6 +884,14 @@ MvccUpdateResult mvccLock( */ public void invoke(GridCacheContext cctx, KeyCacheObject key, OffheapInvokeClosure c) throws IgniteCheckedException; + /** + * @param cctx Cache context. + * @param keys Keys. + * @param c Closure. + * @throws IgniteCheckedException If failed. + */ + public void invokeAll(GridCacheContext cctx, Collection keys, OffheapInvokeAllClosure c) throws IgniteCheckedException; + /** * * @param cctx Cache context. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index 04443be2cf7d7..3f9d2ccbbfb51 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.cache; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -103,6 +104,7 @@ import org.apache.ignite.internal.util.lang.GridIterator; import org.apache.ignite.internal.util.lang.IgniteInClosure2X; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.T3; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; @@ -434,6 +436,16 @@ private Iterator cacheData(boolean primary, boolean backup, Affi dataStore(part).invoke(cctx, key, c); } + /** {@inheritDoc} */ + @Override public void invokeAll( + GridCacheContext cctx, + Collection keys, + GridDhtLocalPartition part, + OffheapInvokeAllClosure c) + throws IgniteCheckedException { + dataStore(part).invokeAll(cctx, keys, c); + } + /** {@inheritDoc} */ @Override public void update( GridCacheContext cctx, @@ -1607,6 +1619,20 @@ private boolean canUpdateOldRow(GridCacheContext cctx, @Nullable CacheDataRow ol } } + + /** {@inheritDoc} */ + @Override public void invokeAll(GridCacheContext cctx, Collection keys, OffheapInvokeAllClosure c) + throws IgniteCheckedException { + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + List searchRows = new ArrayList<>(keys.size()); + + for (KeyCacheObject key : keys) + searchRows.add(new SearchRow(cacheId, key)); + + invokeAll0(cctx, searchRows, c); + } + /** * @param cctx Cache context. * @param row Search row. @@ -1646,6 +1672,57 @@ private void invoke0(GridCacheContext cctx, CacheSearchRow row, OffheapInvokeClo } } + /** + * @param cctx Cache context. + * @param rows Search rows. + * @param c Closure. + * @throws IgniteCheckedException If failed. + */ + private void invokeAll0(GridCacheContext cctx, List rows, OffheapInvokeAllClosure c) + throws IgniteCheckedException { + if (!busyLock.enterBusy()) + throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); + + try { + assert cctx.shared().database().checkpointLockIsHeldByThread(); + + dataTree.invokeAll(rows, CacheDataRowAdapter.RowData.NO_KEY, c); + + for (T3 tuple : c.result()) { + IgniteTree.OperationType opType = tuple.get1(); + CacheDataRow oldRow = tuple.get2(); + CacheDataRow newRow = tuple.get3(); + + switch (opType) { + case PUT: { + assert newRow != null : tuple; + + finishUpdate(cctx, newRow, oldRow); + + break; + } + + case REMOVE: { + finishRemove(cctx, newRow.key(), oldRow); + + break; + } + + case NOOP: + break; + + default: + assert false : opType; + } + } + + + } + finally { + busyLock.leaveBusy(); + } + } + /** {@inheritDoc} */ @Override public CacheDataRow createRow( GridCacheContext cctx, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index 93602db7fb84b..62e76745ca8bf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -31,9 +31,11 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheRebalanceMode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataPageEvictionMode; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.internal.IgniteInternalFuture; @@ -44,6 +46,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection; import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.CacheMapEntries; import org.apache.ignite.internal.processors.cache.CacheMetricsImpl; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; @@ -88,6 +91,16 @@ * Thread pool for requesting partitions from other nodes and populating local cache. */ public class GridDhtPartitionDemander { + /** todo explain the origin */ + private static final int BATCH_PRELOAD_THRESHOLD = 5; + + /** */ + private static final int CHECKPOINT_THRESHOLD = 200; + + /** */ + private static final boolean batchPageWriteEnabled = + IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, true); + /** */ private final GridCacheSharedContext ctx; @@ -774,10 +787,10 @@ public void handleSupplyMessage( part.lock(); try { - Iterator infos = e.getValue().infos().iterator(); + Collection infos = e.getValue().infos(); if (grp.mvccEnabled()) - mvccPreloadEntries(topVer, node, p, infos); + mvccPreloadEntries(topVer, node, p, infos.iterator()); else preloadEntries(topVer, node, p, infos); @@ -859,6 +872,111 @@ public void handleSupplyMessage( } } + /** + * todo should be removed (kept for benchmarking) + */ + public void preloadEntriesSingle(ClusterNode from, + int p, + Collection entries, + AffinityTopologyVersion topVer + ) throws IgniteCheckedException { + GridCacheContext cctx = null; + + // Loop through all received entries and try to preload them. + for (GridCacheEntryInfo entry : entries) { + if (cctx == null || (grp.sharedGroup() && entry.cacheId() != cctx.cacheId())) { + cctx = grp.sharedGroup() ? grp.shared().cacheContext(entry.cacheId()) : grp.singleCacheContext(); + + if (cctx == null) + continue; + else if (cctx.isNear()) + cctx = cctx.dhtCache().context(); + } + + if (!preloadEntry(from, p, entry, topVer, cctx)) { + if (log.isTraceEnabled()) + log.trace("Got entries for invalid partition during " + + "preloading (will skip) [p=" + p + ", entry=" + entry + ']'); + + break; + } + + for (GridCacheContext cctx0 : grp.caches()) { + if (cctx0.statisticsEnabled()) + cctx0.cache().metrics0().onRebalanceKeyReceived(); + } + } + } + + /** + * @param from Node which sent entry. + * @param p Partition id. + * @param entries Preloaded entries. + * @param topVer Topology version. + * @throws IgniteCheckedException If failed. + */ + public void preloadEntriesBatch(ClusterNode from, + int p, + Collection entries, + AffinityTopologyVersion topVer + ) throws IgniteCheckedException { + if (entries.isEmpty()) + return; + + grp.listenerLock().readLock().lock(); + + try { + Map cctxs = new HashMap<>(); + + // Map by context. + for (GridCacheEntryInfo e : entries) { + try { + GridCacheContext cctx0 = grp.sharedGroup() ? ctx.cacheContext(e.cacheId()) : grp.singleCacheContext(); + + if (cctx0 == null) + return; + + if (cctx0.isNear()) + cctx0 = cctx0.dhtCache().context(); + + final GridCacheContext cctx = cctx0; + + if (log.isTraceEnabled()) + log.trace("Rebalancing key [key=" + e.key() + ", part=" + p + ", node=" + from.id() + ']'); + + CacheMapEntries batch = + cctxs.computeIfAbsent(cctx.cacheId(), v -> new CacheMapEntries(topVer, p, cctx, true)); + + batch.add(e.key(), e.value(), e.expireTime(), e.ttl(), e.version(), DR_PRELOAD); + } + catch (GridDhtInvalidPartitionException ignored) { + if (log.isDebugEnabled()) + log.debug("Partition became invalid during rebalancing (will ignore): " + p); + } + } + + for (CacheMapEntries batch : cctxs.values()) { + GridCacheContext cctx = batch.context(); + + batch.lock(); + + try { + cctx.offheap().invokeAll(cctx, batch.keys(), batch.part(), batch.offheapUpdateClosure()); + } + finally { + batch.unlock(); + + for (GridCacheContext cctx0 : grp.caches()) { + if (cctx0.statisticsEnabled()) + cctx0.cache().metrics0().onRebalanceKeysReceived(batch.size()); + } + } + } + } finally { + grp.listenerLock().readLock().unlock(); + } + } + /** * Adds mvcc entries with theirs history to partition p. * @@ -950,48 +1068,43 @@ private void mvccPreloadEntries(AffinityTopologyVersion topVer, ClusterNode node * * @param node Node which sent entry. * @param p Partition id. - * @param infos Entries info for preload. + * @param infosCol Entries info for preload. * @param topVer Topology version. * @throws IgniteInterruptedCheckedException If interrupted. */ private void preloadEntries(AffinityTopologyVersion topVer, ClusterNode node, int p, - Iterator infos) throws IgniteCheckedException { - GridCacheContext cctx = null; + Collection infosCol) throws IgniteCheckedException { + + int size = infosCol.size(); + int n = 0; + int cpTail = size % CHECKPOINT_THRESHOLD; + int cpTotal = size <= CHECKPOINT_THRESHOLD ? 1 : size / CHECKPOINT_THRESHOLD; + + Iterator infos = infosCol.iterator(); + + // todo + boolean batchPageWriteEnabled0 = batchPageWriteEnabled && (grp.dataRegion().config().isPersistenceEnabled() || + grp.dataRegion().config().getPageEvictionMode() == DataPageEvictionMode.DISABLED); // Loop through all received entries and try to preload them. while (infos.hasNext()) { ctx.database().checkpointReadLock(); try { - for (int i = 0; i < 100; i++) { - if (!infos.hasNext()) - break; - - GridCacheEntryInfo entry = infos.next(); + int cnt = cpTotal == 1 ? size : CHECKPOINT_THRESHOLD + (++n == cpTotal ? cpTail : 0); - if (cctx == null || (grp.sharedGroup() && entry.cacheId() != cctx.cacheId())) { - cctx = grp.sharedGroup() ? grp.shared().cacheContext(entry.cacheId()) : grp.singleCacheContext(); + List infosBatch = new ArrayList<>(cnt); - if (cctx == null) - continue; - else if (cctx.isNear()) - cctx = cctx.dhtCache().context(); - } - - if (!preloadEntry(node, p, entry, topVer, cctx)) { - if (log.isTraceEnabled()) - log.trace("Got entries for invalid partition during " + - "preloading (will skip) [p=" + p + ", entry=" + entry + ']'); - - return; - } + for (int i = 0; i < cnt; i++) { + GridCacheEntryInfo entry = infos.next(); - //TODO: IGNITE-11330: Update metrics for touched cache only. - for (GridCacheContext ctx : grp.caches()) { - if (ctx.statisticsEnabled()) - ctx.cache().metrics0().onRebalanceKeyReceived(); - } + infosBatch.add(entry); } + + if (batchPageWriteEnabled0 && infosBatch.size() > BATCH_PRELOAD_THRESHOLD) + preloadEntriesBatch(node, p, infosBatch, topVer); + else + preloadEntriesSingle(node, p, infosBatch, topVer); } finally { ctx.database().checkpointReadUnlock(); @@ -1398,7 +1511,8 @@ private void partitionDone(UUID nodeId, int p, boolean updateState) { "rebalancing [grp=" + grp.cacheOrGroupName() + ", supplier=" + nodeId + ", topVer=" + topologyVersion() + - ", progress=" + (routines - remainingRoutines) + "/" + routines + "]")); + ", progress=" + (routines - remainingRoutines) + "/" + routines + "," + + ", batch=" + batchPageWriteEnabled + "]")); remaining.remove(nodeId); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java index 35dd3c46ee431..6bbe1316b8ca5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence; +import java.util.List; import java.util.Random; import java.util.concurrent.ThreadLocalRandom; import org.apache.ignite.IgniteCheckedException; @@ -307,6 +308,27 @@ protected final R write( return PageHandler.writePage(pageMem, grpId, pageId, this, h, init, wal, null, arg, intArg, lockFailed, statHolder); } + /** + * @param pageId Page ID. + * @param h Handler. + * @param init IO for new page initialization or {@code null} if it is an existing page. + * @param arg Argument. + * @param lockFailed Result in case of lock failure due to page recycling. + * @param statHolder Statistics holder to track IO operations. + * @return Handler result. + * @throws IgniteCheckedException If failed. + */ + protected final R write( + long pageId, + PageHandler h, + PageIO init, + List arg, + int intArg, + R lockFailed, + IoStatisticsHolder statHolder) throws IgniteCheckedException { + return PageHandler.writePageBatch(pageMem, grpId, pageId, this, h, init, wal, null, arg, intArg, lockFailed, statHolder); + } + /** * @param pageId Page ID. * @param h Handler. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java index ab48540f6dfc5..1c2cbedf11452 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -1990,7 +1991,7 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { return delegate.mvccInitialValue(cctx, key, val, ver, expireTime, mvccVer, newMvccVer); } - + /** {@inheritDoc} */ @Override public boolean mvccApplyHistoryIfAbsent( GridCacheContext cctx, @@ -2130,6 +2131,16 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { delegate.invoke(cctx, key, c); } + /** {@inheritDoc} */ + @Override public void invokeAll(GridCacheContext cctx, Collection keys, OffheapInvokeAllClosure c) + throws IgniteCheckedException { + assert ctx.database().checkpointLockIsHeldByThread(); + + CacheDataStore delegate = init0(false); + + delegate.invokeAll(cctx, keys, c); + } + /** {@inheritDoc} */ @Override public void remove(GridCacheContext cctx, KeyCacheObject key, int partId) throws IgniteCheckedException { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java index 4cc59f17e40b7..11eb0a92c2d28 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java @@ -250,7 +250,7 @@ protected void initPageMemoryDataStructures(DataStorageConfiguration dbCfg) thro boolean persistenceEnabled = memPlcCfg.isPersistenceEnabled(); CacheFreeListImpl freeList = new CacheFreeListImpl(0, - cctx.igniteInstanceName(), + memPlc.config().getName(), memMetrics, memPlc, null, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java index 91fd2070cc048..5b721710bde89 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence; +import java.util.Collection; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.processors.cache.CacheGroupContext; @@ -111,6 +112,25 @@ public void addRow(CacheDataRow row, IoStatisticsHolder statHolder) throws Ignit } } + /** + * @param rows Rows. + * @throws IgniteCheckedException If failed. + */ + public void addRows(Collection rows, IoStatisticsHolder statHolder) throws IgniteCheckedException { + if (!persistenceEnabled) + freeList.insertDataRows(rows, statHolder); + else { + ctx.database().checkpointReadLock(); + + try { + freeList.insertDataRows(rows, statHolder); + } + finally { + ctx.database().checkpointReadUnlock(); + } + } + } + /** * @param link Row link. * @param row New row data. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java index 60aefb927ce6f..bd81d757f03ac 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java @@ -17,6 +17,9 @@ package org.apache.ignite.internal.processors.cache.persistence.freelist; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; import java.util.concurrent.atomic.AtomicReferenceArray; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; @@ -65,6 +68,9 @@ public abstract class AbstractFreeList extends PagesList imp /** */ private static final int MIN_PAGE_FREE_SPACE = 8; + /** */ + private static final int MAX_DATA_ROWS_PER_PAGE = 255; // Item index on data page has 1-byte length. + /** * Step between buckets in free list, measured in powers of two. * For example, for page size 4096 and 256 buckets, shift is 4 and step is 16 bytes. @@ -133,12 +139,15 @@ private final class UpdateRowHandler extends PageHandler { /** */ private final PageHandler writeRow = new WriteRowHandler(); + /** */ + private final PageHandler writeRows = new WriteRowsHandler(); + /** * */ - private final class WriteRowHandler extends PageHandler { - @Override public Integer run( - int cacheId, + private class WriteRowHandler extends PageHandler { + /** {@inheritDoc} */ + @Override public Integer run(int cacheId, long pageId, long page, long pageAddr, @@ -146,6 +155,33 @@ private final class WriteRowHandler extends PageHandler { Boolean walPlc, T row, int written, + IoStatisticsHolder statHolder + ) throws IgniteCheckedException { + written = run0(pageId, page, pageAddr, iox, row, written, statHolder); + + putPage((AbstractDataPageIO)iox, pageId, page, pageAddr, statHolder); + + return written; + } + + /** + * @param pageId Page ID. + * @param page Page absolute pointer. + * @param pageAddr Page address. + * @param iox IO. + * @param row Data row. + * @param written Count of bytes written. + * @param statHolder Statistics holder to track IO operations. + * @return Result. + * @throws IgniteCheckedException If failed. + */ + protected Integer run0( + long pageId, + long page, + long pageAddr, + PageIO iox, + T row, + int written, IoStatisticsHolder statHolder) throws IgniteCheckedException { AbstractDataPageIO io = (AbstractDataPageIO)iox; @@ -159,15 +195,6 @@ private final class WriteRowHandler extends PageHandler { written = (written == 0 && oldFreeSpace >= rowSize) ? addRow(pageId, page, pageAddr, io, row, rowSize) : addRowFragment(pageId, page, pageAddr, io, row, written, rowSize); - // Reread free space after update. - int newFreeSpace = io.getFreeSpace(pageAddr); - - if (newFreeSpace > MIN_PAGE_FREE_SPACE) { - int bucket = bucket(newFreeSpace, false); - - put(null, pageId, page, pageAddr, bucket, statHolder); - } - if (written == rowSize) evictionTracker.touchPage(pageId); @@ -185,7 +212,7 @@ private final class WriteRowHandler extends PageHandler { * @return Written size which is always equal to row size here. * @throws IgniteCheckedException If failed. */ - private int addRow( + protected int addRow( long pageId, long page, long pageAddr, @@ -225,7 +252,7 @@ private int addRow( * @return Updated written size. * @throws IgniteCheckedException If failed. */ - private int addRowFragment( + protected int addRowFragment( long pageId, long page, long pageAddr, @@ -254,6 +281,111 @@ private int addRowFragment( return written + payloadSize; } + + /** + * Put page into freelist if needed. + * + * @param iox IO. + * @param pageId Page ID. + * @param page Paege pointer. + * @param pageAddr Page address. + * @param statHolder Statistics holder to track IO operations. + */ + protected void putPage( + AbstractDataPageIO iox, + long pageId, + long page, + long pageAddr, + IoStatisticsHolder statHolder + ) throws IgniteCheckedException { + // Reread free space after update. + int newFreeSpace = ((AbstractDataPageIO)iox).getFreeSpace(pageAddr); + + if (newFreeSpace > MIN_PAGE_FREE_SPACE) { + int bucket = bucket(newFreeSpace, false); + + put(null, pageId, page, pageAddr, bucket, statHolder); + } + } + } + + /** + * + */ + private class WriteRowsHandler extends WriteRowHandler { + + /** {@inheritDoc} */ + @Override public Integer runAll( + int cacheId, + long pageId, + long page, + long pageAddr, + PageIO io, + Boolean walPlc, + List rows, + int writtenCnt, + IoStatisticsHolder statHolder + ) throws IgniteCheckedException { + AbstractDataPageIO iox = (AbstractDataPageIO)io; + + int idx = writtenCnt; + int remainSpace = iox.getFreeSpace(pageAddr); + int remainItems = MAX_DATA_ROWS_PER_PAGE - iox.getRowsCount(pageAddr); + + boolean pageIsEmpty = remainItems == MAX_DATA_ROWS_PER_PAGE; + + List rows0 = pageIsEmpty ? new ArrayList<>(8) : null; + + while (idx < rows.size() && remainItems > 0) { + T row = rows.get(idx); + + int size = row.size(); + int payloadSize = size % MIN_SIZE_FOR_DATA_PAGE; + + // If there is not enough space on page. + if (remainSpace < payloadSize) + break; + + if (pageIsEmpty) + rows0.add(row); + else { + int written = size > MIN_SIZE_FOR_DATA_PAGE ? + addRowFragment(pageId, page, pageAddr, iox, row, size - payloadSize, size) : + addRow(pageId, page, pageAddr, iox, row, size); + + assert written == size : "The object is not fully written into page: pageId=" + pageId + + ", written=" + written + ", size=" + row.size(); + } + + remainSpace -= getPageEntrySize(row, iox); + remainItems -= 1; + + idx += 1; + } + + // Update page counters only once. + if (pageIsEmpty) + iox.addRows(pageMem, pageId, pageAddr, rows0, pageSize()); + + assert idx != writtenCnt; + + evictionTracker.touchPage(pageId); + + putPage((AbstractDataPageIO)io, pageId, page, pageAddr, statHolder); + + return idx; + } + + /** */ + private int getPageEntrySize(T row, AbstractDataPageIO io) throws IgniteCheckedException { + int size = row.size(); + + int sizeSetup = size > MIN_SIZE_FOR_DATA_PAGE ? + AbstractDataPageIO.SHOW_PAYLOAD_LEN | AbstractDataPageIO.SHOW_LINK | AbstractDataPageIO.SHOW_ITEM : + AbstractDataPageIO.SHOW_PAYLOAD_LEN | AbstractDataPageIO.SHOW_ITEM; + + return io.getPageEntrySize(size % MIN_SIZE_FOR_DATA_PAGE, sizeSetup); + } } /** */ @@ -509,6 +641,96 @@ else if (PageIdUtils.tag(pageId) != PageIdAllocator.FLAG_DATA) while (written != COMPLETE); } + /** {@inheritDoc} */ + @Override public void insertDataRows(Collection rows, IoStatisticsHolder statHolder) throws IgniteCheckedException { + // Objects that don't fit into a single data page. + List largeRows = new ArrayList<>(); + + // Ordinary objects and the remaining parts of large objects. + List regularRows = new ArrayList<>(8); + + for (T dataRow : rows) { + int size = dataRow.size(); + + if (size < MIN_SIZE_FOR_DATA_PAGE) + regularRows.add(dataRow); + else { + largeRows.add(dataRow); + + if (size % MIN_SIZE_FOR_DATA_PAGE > 0) + regularRows.add(dataRow); + } + } + + for (T row : largeRows) { + int size = row.size(); + + int written = 0; + + do { + int remaining = size - written; + + if (remaining < MIN_SIZE_FOR_DATA_PAGE) + break; + + long pageId = takeEmptyPage(REUSE_BUCKET, ioVersions(), statHolder); + + AbstractDataPageIO initIo = null; + + if (pageId == 0L) { + pageId = allocateDataPage(row.partition()); + + initIo = ioVersions().latest(); + } + else if (PageIdUtils.tag(pageId) != PageIdAllocator.FLAG_DATA) + pageId = initReusedPage(pageId, row.partition(), statHolder); + else + pageId = PageIdUtils.changePartitionId(pageId, (row.partition())); + + written = write(pageId, writeRow, initIo, row, written, FAIL_I, statHolder); + + assert written != FAIL_I; // We can't fail here. + + memMetrics.incrementLargeEntriesPages(); + } + while (written != COMPLETE); + } + + for (int writtenCnt = 0; writtenCnt < regularRows.size(); ) { + T row = regularRows.get(writtenCnt); + + int size = row.size() % MIN_SIZE_FOR_DATA_PAGE; + + int minBucket = bucket(size, false); + + AbstractDataPageIO initIo = null; + + long pageId = 0; + + for (int b = REUSE_BUCKET - 1; b > minBucket; b--) { + pageId = takeEmptyPage(b, ioVersions(), statHolder); + + if (pageId != 0L) + break; + } + + if (pageId == 0) + pageId = takeEmptyPage(REUSE_BUCKET, ioVersions(), statHolder); + + if (pageId == 0) { + pageId = allocateDataPage(row.partition()); + + initIo = ioVersions().latest(); + } + else if (PageIdUtils.tag(pageId) != PageIdAllocator.FLAG_DATA) + pageId = initReusedPage(pageId, row.partition(), statHolder); + else + pageId = PageIdUtils.changePartitionId(pageId, row.partition()); + + writtenCnt = write(pageId, writeRows, initIo, regularRows, writtenCnt, FAIL_I, statHolder);; + } + } + /** * @param reusedPageId Reused page id. * @param partId Partition id. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java index e28d421bdf063..894c1aa64faca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence.freelist; +import java.util.Collection; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.processors.cache.persistence.Storable; @@ -32,6 +33,12 @@ public interface FreeList { */ public void insertDataRow(T row, IoStatisticsHolder statHolder) throws IgniteCheckedException; + /** + * @param rows Rows. + * @throws IgniteCheckedException If failed. + */ + public void insertDataRows(Collection rows, IoStatisticsHolder statHolder) throws IgniteCheckedException; + /** * @param link Row link. * @param row New row data. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java index 54d9816bcb7e2..10b082a72af28 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java @@ -1822,6 +1822,11 @@ public final boolean removex(L row) throws IgniteCheckedException { } } + /** {@inheritDoc} */ + @Override public void invokeAll(List keys, Object z, InvokeAllClosure c) throws IgniteCheckedException { + throw new UnsupportedOperationException("Not implemented yet"); + } + /** * @param x Invoke operation. * @param pageId Page ID. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java index 78752bbfefc84..d047d4714b150 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java @@ -22,6 +22,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.PageIdUtils; @@ -31,6 +32,7 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler; import org.apache.ignite.internal.util.GridStringBuilder; import org.apache.ignite.internal.util.typedef.internal.SB; +import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.util.GridUnsafe.bufferAddress; @@ -41,13 +43,13 @@ public abstract class AbstractDataPageIO extends PageIO implements CompactablePageIO { /** */ - private static final int SHOW_ITEM = 0b0001; + public static final int SHOW_ITEM = 0b0001; /** */ - private static final int SHOW_PAYLOAD_LEN = 0b0010; + public static final int SHOW_PAYLOAD_LEN = 0b0010; /** */ - private static final int SHOW_LINK = 0b0100; + public static final int SHOW_LINK = 0b0100; /** */ private static final int FREE_LIST_PAGE_ID_OFF = COMMON_HEADER_END; @@ -147,7 +149,7 @@ private int getPageEntrySize(long pageAddr, int dataOff, int show) { * @param show What elements of data page entry to show in the result. * @return Data page entry size. */ - private int getPageEntrySize(int payloadLen, int show) { + public int getPageEntrySize(int payloadLen, int show) { assert payloadLen > 0 : payloadLen; int res = payloadLen; @@ -810,7 +812,7 @@ public void addRow( final int rowSize, final int pageSize ) throws IgniteCheckedException { - assert rowSize <= getFreeSpace(pageAddr) : "can't call addRow if not enough space for the whole row"; + assert rowSize <= getFreeSpace(pageAddr) : "can't call addRow if not enough space for the whole row (free=" + getFreeSpace(pageAddr) + ", required=" + rowSize + ")"; int fullEntrySize = getPageEntrySize(rowSize, SHOW_PAYLOAD_LEN | SHOW_ITEM); @@ -977,6 +979,76 @@ public void addRowFragment( addRowFragment(null, pageId, pageAddr, 0, 0, lastLink, null, payload, pageSize); } + /** + * @param pageMem Page memory. + * @param pageId Page ID to use to construct a link. + * @param pageAddr Page address. + * @param rows Data rows. + * @param pageSize Page size. + * @throws IgniteCheckedException If failed. + */ + public void addRows( + final PageMemory pageMem, + final long pageId, + final long pageAddr, + final Collection rows, + final int pageSize + ) throws IgniteCheckedException { + int maxPayloadSIze = pageSize - MIN_DATA_PAGE_OVERHEAD; + + int regularSizeFlags = SHOW_PAYLOAD_LEN | SHOW_ITEM; + int fragmentSizeFlags = regularSizeFlags | SHOW_LINK; + + int dataOff = pageSize; + int cnt = 0; + + int written = 0; + + for (T row : rows) { + int size = row.size(); + + boolean fragment = size > maxPayloadSIze; + + int payloadSize = size % maxPayloadSIze; + + int fullEntrySize = getPageEntrySize(payloadSize, fragment ? fragmentSizeFlags : regularSizeFlags); + + written += fullEntrySize; + + dataOff -= (fullEntrySize - ITEM_SIZE); + + if (fragment) { + ByteBuffer buf = pageMem.pageBuffer(pageAddr); + + buf.position(dataOff); + + buf.putShort((short)(payloadSize | FRAGMENTED_FLAG)); + buf.putLong(row.link()); + + writeFragmentData(row, buf, 0, payloadSize); + } + else + writeRowData(pageAddr, dataOff, payloadSize, row, true); + + setItem(pageAddr, cnt, directItemFromOffset(dataOff)); + + assert checkIndex(cnt) : cnt; + assert getIndirectCount(pageAddr) <= getDirectCount(pageAddr); + + setLinkByPageId(row, pageId, cnt); + + cnt += 1; + } + + setDirectCount(pageAddr, cnt); + + setFirstEntryOffset(pageAddr, dataOff, pageSize); + + // Update free space. If number of indirect items changed, then we were able to reuse an item slot. + // + (getIndirectCount(pageAddr) != indirectCnt ? ITEM_SIZE : 0) + setRealFreeSpace(pageAddr, getRealFreeSpace(pageAddr) - written, pageSize); + } + /** * Adds maximum possible fragment of the given row to this data page and sets respective link to the row. * @@ -1112,6 +1184,7 @@ private int insertItem(long pageAddr, int dataOff, int directCnt, int indirectCn setItem(pageAddr, directCnt, directItemFromOffset(dataOff)); setDirectCount(pageAddr, directCnt + 1); + assert getDirectCount(pageAddr) == directCnt + 1; return directCnt; // Previous directCnt will be our itemId. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java index 5ab1bf38dbc18..07467f42c166e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.cache.persistence.tree.util; import java.nio.ByteBuffer; +import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.PageSupport; @@ -70,6 +71,32 @@ public abstract R run( ) throws IgniteCheckedException; + /** + * @param cacheId Cache ID. + * @param pageId Page ID. + * @param page Page absolute pointer. + * @param pageAddr Page address. + * @param io IO. + * @param walPlc Full page WAL record policy. + * @param args Arguments. + * @param statHolder Statistics holder to track IO operations. + * @return Result. + * @throws IgniteCheckedException If failed. + */ + public R runAll( + int cacheId, + long pageId, + long page, + long pageAddr, + PageIO io, + Boolean walPlc, + List args, + int intArg, + IoStatisticsHolder statHolder + ) throws IgniteCheckedException { + throw new UnsupportedOperationException(); + } + /** * @param cacheId Cache ID. * @param pageId Page ID. @@ -308,6 +335,75 @@ public static R writePage( } } + /** + * @param pageMem Page memory. + * @param grpId Group ID. + * @param pageId Page ID. + * @param lsnr Lock listener. + * @param h Handler. + * @param init IO for new page initialization or {@code null} if it is an existing page. + * @param wal Write ahead log. + * @param walPlc Full page WAL record policy. + * @param args Argument. + * @param lockFailed Result in case of lock failure due to page recycling. + * @param statHolder Statistics holder to track IO operations. + * @return Handler result. + * @throws IgniteCheckedException If failed. + */ + public static R writePageBatch( + PageMemory pageMem, + int grpId, + final long pageId, + PageLockListener lsnr, + PageHandler h, + PageIO init, + IgniteWriteAheadLogManager wal, + Boolean walPlc, + List args, + int intArg, + R lockFailed, + IoStatisticsHolder statHolder + ) throws IgniteCheckedException { + boolean releaseAfterWrite = true; + + long page = pageMem.acquirePage(grpId, pageId, statHolder); + + try { + long pageAddr = writeLock(pageMem, grpId, pageId, page, lsnr, false); + + if (pageAddr == 0L) + return lockFailed; + + boolean ok = false; + + try { + if (init != null) { + // It is a new page and we have to initialize it. + doInitPage(pageMem, grpId, pageId, page, pageAddr, init, wal); + walPlc = FALSE; + } + else + init = PageIO.getPageIO(pageAddr); + + R res = h.runAll(grpId, pageId, page, pageAddr, init, walPlc, args, intArg, statHolder); + + ok = true; + + return res; + } + finally { + assert PageIO.getCrc(pageAddr) == 0; //TODO GG-11480 + + if (releaseAfterWrite = h.releaseAfterWrite(grpId, pageId, page, pageAddr, null, 0)) + writeUnlock(pageMem, grpId, pageId, page, pageAddr, lsnr, walPlc, ok); + } + } + finally { + if (releaseAfterWrite) + pageMem.releasePage(grpId, pageId, page); + } + } + /** * @param pageMem Page memory. * @param grpId Group ID. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java index b3c1c69e66319..1ce8e560886e4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java @@ -17,6 +17,9 @@ package org.apache.ignite.internal.processors.cache.tree; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.PageUtils; import org.apache.ignite.internal.pagemem.store.PageStore; @@ -45,6 +48,7 @@ import org.apache.ignite.internal.stat.IoStatisticsHolder; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.lang.GridCursor; +import org.apache.ignite.internal.util.typedef.T3; import org.apache.ignite.internal.util.typedef.internal.CU; import static java.lang.Boolean.FALSE; @@ -327,6 +331,89 @@ public CacheDataRowStore rowStore() { return rowStore; } + /** + * todo workaround (fake) implementation only for checking that closure is working properly with preloader. + * todo rework + */ + @Override public void invokeAll(List keys, Object x, InvokeAllClosure c) + throws IgniteCheckedException { + checkDestroyed(); + + List rows = new ArrayList<>(keys.size()); + + if (c.fastpath()) { + GridCursor cur = find(keys.get(0), keys.get(keys.size() - 1), CacheDataRowAdapter.RowData.FULL); + Iterator keyItr = keys.iterator(); + + CacheSearchRow lastRow = null; + CacheSearchRow row = null; + KeyCacheObject key = null; + + CacheDataRow oldRow = null; + KeyCacheObject oldKey = null; + + while (cur.next()) { + oldRow = cur.get(); + oldKey = oldRow.key(); + + while (key == null || key.hashCode() <= oldKey.hashCode()) { + if (key != null && key.hashCode() == oldKey.hashCode()) { + while (key.hashCode() == oldKey.hashCode()) { + // todo test collision resolution + rows.add(key.equals(oldKey) ? oldRow : null); + + lastRow = null; + + if (!keyItr.hasNext()) + break; + + lastRow = row = keyItr.next(); + key = row.key(); + } + } + else { + if (row != null) + rows.add(null); + + lastRow = null; + + if (keyItr.hasNext()) { + lastRow = row = keyItr.next(); + key = lastRow.key(); + } + } + + if (!keyItr.hasNext()) + break; + } + } + + if (lastRow != null) + rows.add(key.equals(oldKey) ? oldRow : null); + + for (; keyItr.hasNext(); keyItr.next()) + rows.add(null); + } else { + for (CacheSearchRow row : keys) { + // todo NO_KEY + CacheDataRow oldRow = findOne(row, null, CacheDataRowAdapter.RowData.FULL); + + rows.add(oldRow); + } + } + + c.call(rows); + + for (T3 t3 : c.result()) { + OperationType oper = t3.get1(); + + if (oper == OperationType.PUT) + put(t3.get3()); + else if (oper == OperationType.REMOVE) + remove(t3.get2()); // old row + } + } + /** {@inheritDoc} */ @Override protected int compare(BPlusIO iox, long pageAddr, int idx, CacheSearchRow row) throws IgniteCheckedException { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java index 9e854d28f6cb0..728b3d98ec576 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java @@ -17,8 +17,11 @@ package org.apache.ignite.internal.util; +import java.util.Collection; +import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.util.lang.GridCursor; +import org.apache.ignite.internal.util.typedef.T3; import org.jetbrains.annotations.Nullable; /** @@ -42,6 +45,14 @@ public interface IgniteTree { */ public void invoke(L key, Object x, InvokeClosure c) throws IgniteCheckedException; + /** + * @param keys Keys. + * @param x Implementation specific argument, {@code null} always means that we need a full detached data row. + * @param c Closure. + * @throws IgniteCheckedException If failed. + */ + public void invokeAll(List keys, Object x, InvokeAllClosure c) throws IgniteCheckedException; + /** * Returns the value to which the specified key is mapped, or {@code null} if this tree contains no mapping for the * key. @@ -130,6 +141,31 @@ interface InvokeClosure { OperationType operationType(); } + /** + * + */ + interface InvokeAllClosure { + /** + * + * @param rows Found/not found data rows. + * @throws IgniteCheckedException If failed. + */ + void call(Collection rows) throws IgniteCheckedException; + + /** + * + * @return List of final actions: operation, old row, new row. + */ + Collection> result(); + + /** + * @return Fast path flag. + * @deprecated Workaround to select B+ tree search strategy, should be removed in final implementation. + */ + @Deprecated + boolean fastpath(); + } + /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListPreloadWithBatchUpdatesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListPreloadWithBatchUpdatesTest.java new file mode 100644 index 0000000000000..539c0c4fc7e7b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListPreloadWithBatchUpdatesTest.java @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ignite.internal.processors.database; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cluster.BaselineNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState; +import org.apache.ignite.internal.util.typedef.PA; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.WithSystemProperty; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; +import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_PAGE_SIZE; + +/** + * + */ +@RunWith(Parameterized.class) +public class FreeListPreloadWithBatchUpdatesTest extends GridCommonAbstractTest { + /** */ + private static final long DEF_REG_SIZE_INIT = 3400 * 1024 * 1024L; + + /** */ + private static final long DEF_REG_SIZE = 6144 * 1024 * 1024L; + + /** */ + private static final int LARGE_PAGE = 16 * 1024; + + /** */ + private static final String DEF_CACHE_NAME = "some-cache"; + + /** */ + @Parameterized.Parameters(name = " atomicity={0}, persistence={1}, pageSize={2}") + public static Iterable setup() { + return Arrays.asList(new Object[][]{ + {CacheAtomicityMode.ATOMIC, false, DFLT_PAGE_SIZE}, + {CacheAtomicityMode.ATOMIC, true, DFLT_PAGE_SIZE}, + {CacheAtomicityMode.ATOMIC, false, LARGE_PAGE}, + {CacheAtomicityMode.ATOMIC, true, LARGE_PAGE}, + {CacheAtomicityMode.TRANSACTIONAL, false, DFLT_PAGE_SIZE}, + {CacheAtomicityMode.TRANSACTIONAL, true, DFLT_PAGE_SIZE}, + {CacheAtomicityMode.TRANSACTIONAL, false, LARGE_PAGE}, + {CacheAtomicityMode.TRANSACTIONAL, true, LARGE_PAGE}, + {CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT, false, DFLT_PAGE_SIZE}, + {CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT, true, DFLT_PAGE_SIZE} + }); + } + + /** */ + @Parameterized.Parameter + public CacheAtomicityMode cacheAtomicityMode; + + /** */ + @Parameterized.Parameter(1) + public boolean persistence; + + /** */ + @Parameterized.Parameter(2) + public Integer pageSize; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + DataRegionConfiguration def = new DataRegionConfiguration(); + def.setInitialSize(DEF_REG_SIZE_INIT); + def.setMaxSize(DEF_REG_SIZE); + def.setPersistenceEnabled(persistence); + + DataStorageConfiguration storeCfg = new DataStorageConfiguration(); + + storeCfg.setDefaultDataRegionConfiguration(def); + storeCfg.setPageSize(pageSize); + + if (persistence) { + storeCfg.setWalMode(WALMode.LOG_ONLY); + storeCfg.setMaxWalArchiveSize(Integer.MAX_VALUE); + } + + cfg.setDataStorageConfiguration(storeCfg); + + return cfg; + } + + /** + * + */ + @Before + public void before() throws Exception { + cleanPersistenceDir(); + } + + /** + * + */ + @After + public void after() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** + * + */ + @Test + @WithSystemProperty(key = IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, value = "true") + public void testBatchRebalance() throws Exception { + Ignite node = startGrid(0); + + node.cluster().active(true); + + node.cluster().baselineAutoAdjustEnabled(false); + + node.createCache(ccfg()); + + int cnt = 100_000; + + int minSize = 0; + int maxSize = 16384; + + Map srcMap = new HashMap<>(); + + for (int i = 0; i < cnt; i++) { + int size = maxSize == minSize ? maxSize : minSize + ThreadLocalRandom.current().nextInt(maxSize - minSize); + + byte[] obj = new byte[size]; + + srcMap.put(i, obj); + } + + try (IgniteDataStreamer streamer = node.dataStreamer(DEF_CACHE_NAME)) { + streamer.addData(srcMap); + } + + log.info("Data loaded."); + + if (persistence) + node.cluster().active(false); + + final IgniteEx node2 = startGrid(1); + + if (persistence) { + List list = new ArrayList<>(node.cluster().currentBaselineTopology()); + + list.add(node2.localNode()); + + node.cluster().active(true); + + node.cluster().setBaselineTopology(list); + } + + log.info("Await rebalance."); + + awaitRebalance(node2, DEF_CACHE_NAME); + + node.close(); + + validateCacheEntries(node2.cache(DEF_CACHE_NAME), srcMap); + + if (persistence) { + node2.close(); + + Ignite ignite = startGrid(1); + + ignite.cluster().active(true); + + log.info("Validate entries after restart"); + + validateCacheEntries(ignite.cache(DEF_CACHE_NAME), srcMap); + } + } + + /** + * + */ + @Test + @WithSystemProperty(key = IGNITE_PDS_WAL_REBALANCE_THRESHOLD, value = "100") + @WithSystemProperty(key = IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, value = "true") + public void testBatchHistoricalRebalance() throws Exception { + if (!persistence) + return; + + // TODO https://issues.apache.org/jira/browse/IGNITE-7384 + // TODO http://apache-ignite-developers.2346864.n4.nabble.com/Historical-rebalance-td38380.html + if (cacheAtomicityMode == CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT) + return; + + Ignite node = startGrids(2); + + node.cluster().active(true); + + IgniteCache cache = node.createCache(ccfg()); + + int cnt = 10_000; + + log.info("Loading " + cnt + " random entries."); + + Map srcMap = new HashMap<>(); + + for (int i = 0; i < cnt; i++) { + byte[] obj = new byte[ThreadLocalRandom.current().nextInt(16384)]; + + if (i > 0 && i < 1000 && i % 37 == 0) + continue; + + srcMap.put(i, obj); + } + + try (IgniteDataStreamer streamer = node.dataStreamer(DEF_CACHE_NAME)) { + streamer.addData(srcMap); + } + + forceCheckpoint(); + + log.info("Stopping node #2."); + + grid(1).close(); + + log.info("Updating values on node #1."); + + for (int i = 100; i < 2000; i++) { + if (i % 3 == 0) { + cache.remove(i); + + srcMap.remove(i); + } + else { + byte[] bytes = new byte[ThreadLocalRandom.current().nextInt(16384)]; + + Arrays.fill(bytes, (byte)1); + + srcMap.put(i, bytes); + cache.put(i, bytes); + } + } + + for (int i = 10_000; i < 11_000; i++) { + byte[] bytes = new byte[ThreadLocalRandom.current().nextInt(16384)]; + + srcMap.put(i, bytes); + cache.put(i, bytes); + } + + forceCheckpoint(); + + log.info("Starting node #2."); + + IgniteEx node2 = startGrid(1); + + log.info("Await rebalance on node #2."); + + awaitRebalance(node2, DEF_CACHE_NAME); + + log.info("Stop node #1."); + + node.close(); + + validateCacheEntries(node2.cache(DEF_CACHE_NAME), srcMap); + } + + /** + * @param node Ignite node. + * @param name Cache name. + */ + private void awaitRebalance(IgniteEx node, String name) throws IgniteInterruptedCheckedException { + boolean ok = GridTestUtils.waitForCondition(new PA() { + @Override public boolean apply() { + for ( GridDhtLocalPartition part : node.context().cache().cache(name).context().group().topology().localPartitions()) { + if (part.state() != GridDhtPartitionState.OWNING) + return false; + } + + return true; + } + }, 60_000); + + U.sleep(3000); + + assertTrue(ok); + } + + /** + * @param cache Cache. + * @param map Map. + */ + @SuppressWarnings("unchecked") + private void validateCacheEntries(IgniteCache cache, Map map) { + int size = cache.size(); + + assertEquals("Cache size mismatch.", map.size(), size); + + log.info("Validation " + cache.getName() + ", size=" + size); + + for (Map.Entry e : map.entrySet()) { + String idx = "key=" + e.getKey(); + + assertEquals(idx, e.getValue().length, ((byte[])cache.get(e.getKey())).length); + } + } + + /** + * @return Cache configuration. + */ + private CacheConfiguration ccfg() { + return ccfg(1, CacheMode.REPLICATED); + } + + /** + * @return Cache configuration. + */ + private CacheConfiguration ccfg(int parts, CacheMode mode) { + return new CacheConfiguration(DEF_CACHE_NAME) + .setAffinity(new RendezvousAffinityFunction(false, parts)) + .setCacheMode(mode) + .setAtomicityMode(cacheAtomicityMode); + } +} From ec35054ae74acbfda38890fe393970fa9e544888 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 28 Mar 2019 14:56:02 +0300 Subject: [PATCH 02/24] IGNITE-11584 Code cleanup. --- .../cache/IgniteCacheOffheapManagerImpl.java | 12 +++++------- .../processors/cache/tree/CacheDataTree.java | 13 +++++++------ 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index 3f9d2ccbbfb51..d181bd726f005 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -1688,14 +1688,14 @@ private void invokeAll0(GridCacheContext cctx, List rows, Offhea dataTree.invokeAll(rows, CacheDataRowAdapter.RowData.NO_KEY, c); - for (T3 tuple : c.result()) { - IgniteTree.OperationType opType = tuple.get1(); - CacheDataRow oldRow = tuple.get2(); - CacheDataRow newRow = tuple.get3(); + for (T3 finalOp : c.result()) { + IgniteTree.OperationType opType = finalOp.get1(); + CacheDataRow oldRow = finalOp.get2(); + CacheDataRow newRow = finalOp.get3(); switch (opType) { case PUT: { - assert newRow != null : tuple; + assert newRow != null : finalOp; finishUpdate(cctx, newRow, oldRow); @@ -1715,8 +1715,6 @@ private void invokeAll0(GridCacheContext cctx, List rows, Offhea assert false : opType; } } - - } finally { busyLock.leaveBusy(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java index 1ce8e560886e4..12f59358441d8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java @@ -332,17 +332,18 @@ public CacheDataRowStore rowStore() { } /** - * todo workaround (fake) implementation only for checking that closure is working properly with preloader. - * todo rework + * todo Stub implementation. After implementing BPlusTree.invokeAll(...), this stub will be removed. */ @Override public void invokeAll(List keys, Object x, InvokeAllClosure c) throws IgniteCheckedException { checkDestroyed(); - List rows = new ArrayList<>(keys.size()); + int keysCnt = keys.size(); + + List rows = new ArrayList<>(keysCnt); if (c.fastpath()) { - GridCursor cur = find(keys.get(0), keys.get(keys.size() - 1), CacheDataRowAdapter.RowData.FULL); + GridCursor cur = find(keys.get(0), keys.get(keysCnt - 1), CacheDataRowAdapter.RowData.FULL); Iterator keyItr = keys.iterator(); CacheSearchRow lastRow = null; @@ -408,9 +409,9 @@ public CacheDataRowStore rowStore() { OperationType oper = t3.get1(); if (oper == OperationType.PUT) - put(t3.get3()); + putx(t3.get3()); else if (oper == OperationType.REMOVE) - remove(t3.get2()); // old row + removex(t3.get2()); // old row } } From f1fcda776a55b0b1ce1631d8abfb86fcc08cf743 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 28 Mar 2019 19:53:21 +0300 Subject: [PATCH 03/24] IGNITE-11584 Rework bench to use reflections. --- .../jmh/pagemem/JmhBatchUpdatesBenchmark.java | 293 +++++++++++++----- 1 file changed, 220 insertions(+), 73 deletions(-) diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java index 225e702bfea74..9402a79b70ba7 100644 --- a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java +++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java @@ -17,15 +17,20 @@ package org.apache.ignite.internal.benchmarks.jmh.pagemem; +import java.lang.reflect.Constructor; import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Modifier; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.Ignition; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; @@ -35,12 +40,17 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; +import org.apache.ignite.internal.processors.cache.GridCacheGroupIdMessage; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState; import org.apache.ignite.logger.NullLogger; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -197,8 +207,9 @@ private CacheConfiguration ccfg(boolean batch) { * @param preloader Data preloader. */ @Benchmark + @Fork(jvmArgsAppend = "-D"+IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE+"=false") public void checkSingle(Data data, Preloader preloader) throws IgniteCheckedException { - preloader.demanderSingle.preloadEntriesSingle(null, 0, data.singleData, data.cctxSingle.topology().readyTopologyVersion()); + preloader.demanderSingle.handleSupplyMessage(0, data.node.localNode().id(), data.singleData); } /** @@ -208,8 +219,9 @@ public void checkSingle(Data data, Preloader preloader) throws IgniteCheckedExce * @param preloader Data preloader. */ @Benchmark + @Fork(jvmArgsAppend = "-D"+IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE+"=true") public void checkBatch(Data data, Preloader preloader) throws IgniteCheckedException { - preloader.demanderBatch.preloadEntriesBatch(null, 0, data.batchData, data.cctxBatch.topology().readyTopologyVersion()); + preloader.demanderBatch.handleSupplyMessage(0, data.node.localNode().id(), data.batchData); } /** @@ -217,7 +229,24 @@ public void checkBatch(Data data, Preloader preloader) throws IgniteCheckedExcep */ @Setup(Level.Trial) public void setup() { - Ignition.start(getConfiguration(NODE_NAME)); + IgniteEx node = (IgniteEx)Ignition.start(getConfiguration(NODE_NAME)); + + moving(node, CACHE_BATCH); + moving(node, CACHE_SINGLE); + } + + private void moving(IgniteEx node, String name) { + List parts = node.cachex(name).context().group().topology().localPartitions(); + + assert parts.size() == 1 : parts.size(); + + GridDhtLocalPartition part = parts.iterator().next(); + + assert part.state() == GridDhtPartitionState.OWNING : part.state(); + + part.setState(GridDhtPartitionState.MOVING); + + assert part.state() == GridDhtPartitionState.MOVING : part.state(); } /** @@ -241,64 +270,29 @@ public static class Preloader { /** */ GridDhtPartitionDemander demander(String name) { - GridCacheContext cctx = ((IgniteEx)Ignition.ignite(NODE_NAME)).cachex(name).context(); - - GridDhtPreloader preloader = (GridDhtPreloader)cctx.group().preloader(); + try { + GridCacheContext cctx = ((IgniteEx)Ignition.ignite(NODE_NAME)).cachex(name).context(); - return getFieldValue(preloader, "demander"); - } + GridDhtPreloader preloader = (GridDhtPreloader)cctx.group().preloader(); - /** - * Get object field value via reflection. - * - * @param obj Object or class to get field value from. - * @param fieldNames Field names to get value for: obj->field1->field2->...->fieldN. - * @param Expected field class. - * @return Field value. - * @throws IgniteException In case of error. - */ - public static T getFieldValue(Object obj, String... fieldNames) throws IgniteException { - assert obj != null; - assert fieldNames != null; - assert fieldNames.length >= 1; + Constructor constructor = + getDefaultConstructor(GridDhtPartitionDemander.RebalanceFuture.class); - try { - for (String fieldName : fieldNames) { - Class cls = obj instanceof Class ? (Class)obj : obj.getClass(); - - try { - obj = findField(cls, obj, fieldName); - } - catch (NoSuchFieldException e) { - throw new RuntimeException(e); - } - } + GridDhtPartitionDemander.RebalanceFuture fut = constructor.newInstance(); - return (T)obj; - } - catch (IllegalAccessException e) { - throw new IgniteException("Failed to get object field [obj=" + obj + - ", fieldNames=" + Arrays.toString(fieldNames) + ']', e); - } - } + setFieldValue(fut, "rebalanceId", 0); + setFieldValue(fut, "topVer", cctx.group().affinity().lastVersion()); - /** - * @param cls Class for searching. - * @param obj Target object. - * @param fieldName Field name for search. - * @return Field from object if it was found. - */ - private static Object findField(Class cls, Object obj, - String fieldName) throws NoSuchFieldException, IllegalAccessException { - // Resolve inner field. - Field field = cls.getDeclaredField(fieldName); + GridDhtPartitionDemander demander = getFieldValue(preloader, "demander"); - boolean accessible = field.isAccessible(); + setFieldValue(demander, "rebalanceFut", fut); - if (!accessible) - field.setAccessible(true); + setFieldValue(cctx.shared().exchange(), "rebTopVer", cctx.group().affinity().lastVersion()); - return field.get(obj); + return demander; + } catch (Exception e) { + throw new RuntimeException(e); + } } } @@ -315,16 +309,22 @@ public static class Data { private int[] sizes; /** */ - Collection batchData = new ArrayList<>(BATCH_SIZE); + private GridDhtPartitionSupplyMessage batchData; /** */ - Collection singleData = new ArrayList<>(BATCH_SIZE); + private GridDhtPartitionSupplyMessage singleData; /** */ - GridCacheContext cctxBatch = ((IgniteEx)Ignition.ignite(NODE_NAME)).cachex(CACHE_BATCH).context(); + private GridCacheContext cctxBatch = ((IgniteEx)Ignition.ignite(NODE_NAME)).cachex(CACHE_BATCH).context(); /** */ - GridCacheContext cctxSingle = ((IgniteEx)Ignition.ignite(NODE_NAME)).cachex(CACHE_SINGLE).context(); + private GridCacheContext cctxSingle = ((IgniteEx)Ignition.ignite(NODE_NAME)).cachex(CACHE_SINGLE).context(); + + /** */ + private IgniteEx node = ((IgniteEx)Ignition.ignite(NODE_NAME)); + + /** */ + int part = 0; /** */ @Setup(Level.Trial) @@ -336,13 +336,13 @@ public void setup() { * Prepare collection. */ @Setup(Level.Iteration) - public void prepare() { + public void prepare() throws IllegalAccessException, InstantiationException, InvocationTargetException { int iter = iteration++; - int off = iter * BATCH_SIZE; + int rebalanceId = 0; - batchData = prepareBatch(cctxBatch, off, BATCH_SIZE, sizes); - singleData = prepareBatch(cctxSingle, off, BATCH_SIZE, sizes); + batchData = prepareSupplyMessage(cctxBatch, rebalanceId, part, off, BATCH_SIZE, sizes); + singleData = prepareSupplyMessage(cctxSingle, rebalanceId, part, off, BATCH_SIZE, sizes); } /** @@ -375,16 +375,23 @@ int[] sizes(int minObjSize, int maxObjSize, int batchSize) { return sizes; } - /** - * Generates rebalance info objects. - * - * @param cctx Cache context. - * @param off Offset. - * @param cnt Count. - * @param sizes Object sizes. - * @return List of generated objects. - */ - private List prepareBatch(GridCacheContext cctx, int off, int cnt, int[] sizes) { + /** */ + private GridDhtPartitionSupplyMessage prepareSupplyMessage( + GridCacheContext cctx, + int rebalanceId, + int p, + int off, + int cnt, + int[] sizes + ) throws IllegalAccessException, InvocationTargetException, InstantiationException { + Constructor constructor = getDefaultConstructor(GridDhtPartitionSupplyMessage.class); + + GridDhtPartitionSupplyMessage msg = constructor.newInstance(); + + setFieldValue(msg, GridCacheGroupIdMessage.class, "grpId", cctx.group().groupId()); + setFieldValue(msg, "rebalanceId", rebalanceId); + setFieldValue(msg, "topVer", cctx.group().affinity().lastVersion()); + List infos = new ArrayList<>(); for (int i = off; i < off + cnt; i++) { @@ -402,7 +409,13 @@ private List prepareBatch(GridCacheContext cctx, int off, in infos.add(info); } - return infos; + Map map = new HashMap<>(); + + map.put(p, new CacheEntryInfoCollection(infos)); + + setFieldValue(msg, "infos", map); + + return msg; } } @@ -418,4 +431,138 @@ public static void main(String[] args) throws RunnerException { new Runner(options).run(); } + + /** */ + private static Constructor getDefaultConstructor(Class clazz) { + Constructor constructors[] = clazz.getDeclaredConstructors(); + + for (Constructor constructor : constructors) { + if (constructor.getParameterTypes().length == 0) { + constructor.setAccessible(true); + + return (Constructor) constructor; + } + } + + throw new RuntimeException("No default constructor"); + } + + /** + * Get object field value via reflection. + * + * @param obj Object or class to get field value from. + * @param fieldNames Field names to get value for: obj->field1->field2->...->fieldN. + * @param Expected field class. + * @return Field value. + * @throws IgniteException In case of error. + */ + private static T getFieldValue(Object obj, String... fieldNames) throws IgniteException { + assert obj != null; + assert fieldNames != null; + assert fieldNames.length >= 1; + + try { + for (String fieldName : fieldNames) { + Class cls = obj instanceof Class ? (Class)obj : obj.getClass(); + + try { + obj = findField(cls, obj, fieldName); + } + catch (NoSuchFieldException e) { + throw new RuntimeException(e); + } + } + + return (T)obj; + } + catch (IllegalAccessException e) { + throw new IgniteException("Failed to get object field [obj=" + obj + + ", fieldNames=" + Arrays.toString(fieldNames) + ']', e); + } + } + + /** + * Set object field value via reflection. + * + * @param obj Object to set field value to. + * @param fieldName Field name to set value for. + * @param val New field value. + * @throws IgniteException In case of error. + */ + private static void setFieldValue(Object obj, String fieldName, Object val) throws IgniteException { + assert obj != null; + assert fieldName != null; + + try { + Class cls = obj instanceof Class ? (Class)obj : obj.getClass(); + + Field field = cls.getDeclaredField(fieldName); + + boolean accessible = field.isAccessible(); + + if (!accessible) + field.setAccessible(true); + + field.set(obj, val); + } + catch (NoSuchFieldException | IllegalAccessException e) { + throw new IgniteException("Failed to set object field [obj=" + obj + ", field=" + fieldName + ']', e); + } + } + + /** + * @param cls Class for searching. + * @param obj Target object. + * @param fieldName Field name for search. + * @return Field from object if it was found. + */ + private static Object findField(Class cls, Object obj, + String fieldName) throws NoSuchFieldException, IllegalAccessException { + // Resolve inner field. + Field field = cls.getDeclaredField(fieldName); + + boolean accessible = field.isAccessible(); + + if (!accessible) + field.setAccessible(true); + + return field.get(obj); + } + + /** + * Set object field value via reflection. + * + * @param obj Object to set field value to. + * @param cls Class to get field from. + * @param fieldName Field name to set value for. + * @param val New field value. + * @throws IgniteException In case of error. + */ + private static void setFieldValue(Object obj, Class cls, String fieldName, Object val) throws IgniteException { + assert fieldName != null; + + try { + Field field = cls.getDeclaredField(fieldName); + + boolean accessible = field.isAccessible(); + + if (!accessible) + field.setAccessible(true); + + boolean isFinal = (field.getModifiers() & Modifier.FINAL) != 0; + + if (isFinal) { + Field modifiersField = Field.class.getDeclaredField("modifiers"); + + modifiersField.setAccessible(true); + + modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL); + } + + field.set(obj, val); + } + catch (NoSuchFieldException | IllegalAccessException e) { + throw new IgniteException("Failed to set object field [obj=" + obj + ", field=" + fieldName + ']', e); + } + } } From 96daeeb2a46ce21346955fcc1068d839b464cf31 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 28 Mar 2019 20:29:51 +0300 Subject: [PATCH 04/24] IGNITE-11584 Code cleanup (wip). --- .../preloader/GridDhtPartitionDemander.java | 113 ++++++++---------- 1 file changed, 47 insertions(+), 66 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index 62e76745ca8bf..bcde7cdc3818d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -91,14 +91,11 @@ * Thread pool for requesting partitions from other nodes and populating local cache. */ public class GridDhtPartitionDemander { - /** todo explain the origin */ - private static final int BATCH_PRELOAD_THRESHOLD = 5; - /** */ - private static final int CHECKPOINT_THRESHOLD = 200; + private static final int CHECKPOINT_THRESHOLD = 100; /** */ - private static final boolean batchPageWriteEnabled = + private static final boolean BATCH_PAGE_WRITE_ENABLED = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, true); /** */ @@ -787,10 +784,10 @@ public void handleSupplyMessage( part.lock(); try { - Collection infos = e.getValue().infos(); + Iterator infos = e.getValue().infos().iterator(); if (grp.mvccEnabled()) - mvccPreloadEntries(topVer, node, p, infos.iterator()); + mvccPreloadEntries(topVer, node, p, infos); else preloadEntries(topVer, node, p, infos); @@ -872,42 +869,6 @@ public void handleSupplyMessage( } } - /** - * todo should be removed (kept for benchmarking) - */ - public void preloadEntriesSingle(ClusterNode from, - int p, - Collection entries, - AffinityTopologyVersion topVer - ) throws IgniteCheckedException { - GridCacheContext cctx = null; - - // Loop through all received entries and try to preload them. - for (GridCacheEntryInfo entry : entries) { - if (cctx == null || (grp.sharedGroup() && entry.cacheId() != cctx.cacheId())) { - cctx = grp.sharedGroup() ? grp.shared().cacheContext(entry.cacheId()) : grp.singleCacheContext(); - - if (cctx == null) - continue; - else if (cctx.isNear()) - cctx = cctx.dhtCache().context(); - } - - if (!preloadEntry(from, p, entry, topVer, cctx)) { - if (log.isTraceEnabled()) - log.trace("Got entries for invalid partition during " + - "preloading (will skip) [p=" + p + ", entry=" + entry + ']'); - - break; - } - - for (GridCacheContext cctx0 : grp.caches()) { - if (cctx0.statisticsEnabled()) - cctx0.cache().metrics0().onRebalanceKeyReceived(); - } - } - } - /** * @param from Node which sent entry. * @param p Partition id. @@ -915,7 +876,7 @@ else if (cctx.isNear()) * @param topVer Topology version. * @throws IgniteCheckedException If failed. */ - public void preloadEntriesBatch(ClusterNode from, + private void preloadEntriesBatch(ClusterNode from, int p, Collection entries, AffinityTopologyVersion topVer @@ -1068,22 +1029,16 @@ private void mvccPreloadEntries(AffinityTopologyVersion topVer, ClusterNode node * * @param node Node which sent entry. * @param p Partition id. - * @param infosCol Entries info for preload. + * @param infos Entries info for preload. * @param topVer Topology version. * @throws IgniteInterruptedCheckedException If interrupted. */ private void preloadEntries(AffinityTopologyVersion topVer, ClusterNode node, int p, - Collection infosCol) throws IgniteCheckedException { - - int size = infosCol.size(); - int n = 0; - int cpTail = size % CHECKPOINT_THRESHOLD; - int cpTotal = size <= CHECKPOINT_THRESHOLD ? 1 : size / CHECKPOINT_THRESHOLD; + Iterator infos) throws IgniteCheckedException { - Iterator infos = infosCol.iterator(); + GridCacheContext cctx = null; - // todo - boolean batchPageWriteEnabled0 = batchPageWriteEnabled && (grp.dataRegion().config().isPersistenceEnabled() || + boolean batchWriteEnabled = BATCH_PAGE_WRITE_ENABLED && (grp.dataRegion().config().isPersistenceEnabled() || grp.dataRegion().config().getPageEvictionMode() == DataPageEvictionMode.DISABLED); // Loop through all received entries and try to preload them. @@ -1091,20 +1046,45 @@ private void preloadEntries(AffinityTopologyVersion topVer, ClusterNode node, in ctx.database().checkpointReadLock(); try { - int cnt = cpTotal == 1 ? size : CHECKPOINT_THRESHOLD + (++n == cpTotal ? cpTail : 0); + List infosBatch = batchWriteEnabled ? new ArrayList<>(CHECKPOINT_THRESHOLD) : null; - List infosBatch = new ArrayList<>(cnt); + for (int i = 0; i < CHECKPOINT_THRESHOLD; i++) { + if (!infos.hasNext()) + break; - for (int i = 0; i < cnt; i++) { GridCacheEntryInfo entry = infos.next(); - infosBatch.add(entry); + if (batchWriteEnabled) + infosBatch.add(entry); + else { + if (cctx == null || (grp.sharedGroup() && entry.cacheId() != cctx.cacheId())) { + cctx = grp.sharedGroup() ? + grp.shared().cacheContext(entry.cacheId()) : grp.singleCacheContext(); + + if (cctx == null) + continue; + else if (cctx.isNear()) + cctx = cctx.dhtCache().context(); + } + + if (!preloadEntry(node, p, entry, topVer, cctx)) { + if (log.isTraceEnabled()) + log.trace("Got entries for invalid partition during " + + "preloading (will skip) [p=" + p + ", entry=" + entry + ']'); + + return; + } + + //TODO: IGNITE-11330: Update metrics for touched cache only. + for (GridCacheContext ctx : grp.caches()) { + if (ctx.statisticsEnabled()) + ctx.cache().metrics0().onRebalanceKeyReceived(); + } + } } - if (batchPageWriteEnabled0 && infosBatch.size() > BATCH_PRELOAD_THRESHOLD) + if (batchWriteEnabled) preloadEntriesBatch(node, p, infosBatch, topVer); - else - preloadEntriesSingle(node, p, infosBatch, topVer); } finally { ctx.database().checkpointReadUnlock(); @@ -1508,11 +1488,11 @@ private void partitionDone(UUID nodeId, int p, boolean updateState) { int remainingRoutines = remaining.size() - 1; U.log(log, "Completed " + ((remainingRoutines == 0 ? "(final) " : "") + - "rebalancing [grp=" + grp.cacheOrGroupName() + - ", supplier=" + nodeId + - ", topVer=" + topologyVersion() + - ", progress=" + (routines - remainingRoutines) + "/" + routines + "," + - ", batch=" + batchPageWriteEnabled + "]")); + "rebalancing [grp=" + grp.cacheOrGroupName() + + ", supplier=" + nodeId + + ", topVer=" + topologyVersion() + + ", progress=" + (routines - remainingRoutines) + "/" + routines + "," + + ", batch=" + BATCH_PAGE_WRITE_ENABLED + "]")); remaining.remove(nodeId); } @@ -1616,3 +1596,4 @@ private void sendRebalanceFinishedEvent() { } } } + From 5862e84eaed7b49f0fba60101b7218249d8ef7e6 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Tue, 2 Apr 2019 14:55:47 +0300 Subject: [PATCH 05/24] IGNITE-11584 Simplify test. --- .../jmh/pagemem/JmhBatchUpdatesBenchmark.java | 132 +++++------- .../FreeListPreloadWithBatchUpdatesTest.java | 194 +++++------------- 2 files changed, 105 insertions(+), 221 deletions(-) diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java index 9402a79b70ba7..17a9080f1276a 100644 --- a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java +++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java @@ -28,7 +28,6 @@ import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.Ignition; @@ -40,6 +39,7 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; @@ -207,8 +207,8 @@ private CacheConfiguration ccfg(boolean batch) { * @param preloader Data preloader. */ @Benchmark - @Fork(jvmArgsAppend = "-D"+IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE+"=false") - public void checkSingle(Data data, Preloader preloader) throws IgniteCheckedException { + @Fork(jvmArgsAppend = "-D" + IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE + "=false") + public void checkSingle(Data data, Preloader preloader) { preloader.demanderSingle.handleSupplyMessage(0, data.node.localNode().id(), data.singleData); } @@ -219,8 +219,8 @@ public void checkSingle(Data data, Preloader preloader) throws IgniteCheckedExce * @param preloader Data preloader. */ @Benchmark - @Fork(jvmArgsAppend = "-D"+IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE+"=true") - public void checkBatch(Data data, Preloader preloader) throws IgniteCheckedException { + @Fork(jvmArgsAppend = "-D" + IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE + "=true") + public void checkBatch(Data data, Preloader preloader) { preloader.demanderBatch.handleSupplyMessage(0, data.node.localNode().id(), data.batchData); } @@ -231,22 +231,14 @@ public void checkBatch(Data data, Preloader preloader) throws IgniteCheckedExcep public void setup() { IgniteEx node = (IgniteEx)Ignition.start(getConfiguration(NODE_NAME)); - moving(node, CACHE_BATCH); - moving(node, CACHE_SINGLE); + partitionState(node, CACHE_BATCH, GridDhtPartitionState.MOVING); + partitionState(node, CACHE_SINGLE, GridDhtPartitionState.MOVING); } - private void moving(IgniteEx node, String name) { - List parts = node.cachex(name).context().group().topology().localPartitions(); - - assert parts.size() == 1 : parts.size(); - - GridDhtLocalPartition part = parts.iterator().next(); - - assert part.state() == GridDhtPartitionState.OWNING : part.state(); - - part.setState(GridDhtPartitionState.MOVING); - - assert part.state() == GridDhtPartitionState.MOVING : part.state(); + /** */ + private void partitionState(IgniteEx node, String name, GridDhtPartitionState state) { + for (GridDhtLocalPartition part : node.cachex(name).context().group().topology().localPartitions()) + part.setState(state); } /** @@ -272,22 +264,20 @@ public static class Preloader { GridDhtPartitionDemander demander(String name) { try { GridCacheContext cctx = ((IgniteEx)Ignition.ignite(NODE_NAME)).cachex(name).context(); - GridDhtPreloader preloader = (GridDhtPreloader)cctx.group().preloader(); + AffinityTopologyVersion topVer = cctx.group().affinity().lastVersion(); - Constructor constructor = - getDefaultConstructor(GridDhtPartitionDemander.RebalanceFuture.class); - - GridDhtPartitionDemander.RebalanceFuture fut = constructor.newInstance(); + GridDhtPartitionDemander.RebalanceFuture fut = + newInstance(GridDhtPartitionDemander.RebalanceFuture.class); setFieldValue(fut, "rebalanceId", 0); - setFieldValue(fut, "topVer", cctx.group().affinity().lastVersion()); + setFieldValue(fut, "topVer", topVer); GridDhtPartitionDemander demander = getFieldValue(preloader, "demander"); setFieldValue(demander, "rebalanceFut", fut); - setFieldValue(cctx.shared().exchange(), "rebTopVer", cctx.group().affinity().lastVersion()); + setFieldValue(cctx.shared().exchange(), "rebTopVer", topVer); return demander; } catch (Exception e) { @@ -336,7 +326,7 @@ public void setup() { * Prepare collection. */ @Setup(Level.Iteration) - public void prepare() throws IllegalAccessException, InstantiationException, InvocationTargetException { + public void prepare() { int iter = iteration++; int off = iter * BATCH_SIZE; int rebalanceId = 0; @@ -383,10 +373,8 @@ private GridDhtPartitionSupplyMessage prepareSupplyMessage( int off, int cnt, int[] sizes - ) throws IllegalAccessException, InvocationTargetException, InstantiationException { - Constructor constructor = getDefaultConstructor(GridDhtPartitionSupplyMessage.class); - - GridDhtPartitionSupplyMessage msg = constructor.newInstance(); + ) { + GridDhtPartitionSupplyMessage msg = newInstance(GridDhtPartitionSupplyMessage.class); setFieldValue(msg, GridCacheGroupIdMessage.class, "grpId", cctx.group().groupId()); setFieldValue(msg, "rebalanceId", rebalanceId); @@ -419,30 +407,22 @@ private GridDhtPartitionSupplyMessage prepareSupplyMessage( } } - /** - * Run benchmark. - * - * @param args Args. - */ - public static void main(String[] args) throws RunnerException { - final Options options = new OptionsBuilder() - .include(JmhBatchUpdatesBenchmark.class.getSimpleName()) - .build(); - - new Runner(options).run(); - } - /** */ - private static Constructor getDefaultConstructor(Class clazz) { + private static T newInstance(Class clazz) { Constructor constructors[] = clazz.getDeclaredConstructors(); - for (Constructor constructor : constructors) { - if (constructor.getParameterTypes().length == 0) { - constructor.setAccessible(true); + try { + for (Constructor constructor : constructors) { + if (constructor.getParameterTypes().length == 0) { + constructor.setAccessible(true); - return (Constructor) constructor; + return (T)constructor.newInstance(); + } } } + catch (IllegalAccessException | InstantiationException | InvocationTargetException e) { + throw new RuntimeException(e); + } throw new RuntimeException("No default constructor"); } @@ -481,35 +461,6 @@ private static T getFieldValue(Object obj, String... fieldNames) throws Igni } } - /** - * Set object field value via reflection. - * - * @param obj Object to set field value to. - * @param fieldName Field name to set value for. - * @param val New field value. - * @throws IgniteException In case of error. - */ - private static void setFieldValue(Object obj, String fieldName, Object val) throws IgniteException { - assert obj != null; - assert fieldName != null; - - try { - Class cls = obj instanceof Class ? (Class)obj : obj.getClass(); - - Field field = cls.getDeclaredField(fieldName); - - boolean accessible = field.isAccessible(); - - if (!accessible) - field.setAccessible(true); - - field.set(obj, val); - } - catch (NoSuchFieldException | IllegalAccessException e) { - throw new IgniteException("Failed to set object field [obj=" + obj + ", field=" + fieldName + ']', e); - } - } - /** * @param cls Class for searching. * @param obj Target object. @@ -529,6 +480,18 @@ private static Object findField(Class cls, Object obj, return field.get(obj); } + /** + * Set object field value via reflection. + * + * @param obj Object to set field value to. + * @param fieldName Field name to set value for. + * @param val New field value. + * @throws IgniteException In case of error. + */ + private static void setFieldValue(Object obj, String fieldName, Object val) throws IgniteException { + setFieldValue(obj, obj.getClass(), fieldName, val); + } + /** * Set object field value via reflection. * @@ -565,4 +528,17 @@ private static void setFieldValue(Object obj, Class cls, String fieldName, Objec throw new IgniteException("Failed to set object field [obj=" + obj + ", field=" + fieldName + ']', e); } } + + /** + * Run benchmark. + * + * @param args Args. + */ + public static void main(String[] args) throws RunnerException { + final Options options = new OptionsBuilder() + .include(JmhBatchUpdatesBenchmark.class.getSimpleName()) + .build(); + + new Runner(options).run(); + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListPreloadWithBatchUpdatesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListPreloadWithBatchUpdatesTest.java index 539c0c4fc7e7b..bbe6f59a7258d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListPreloadWithBatchUpdatesTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListPreloadWithBatchUpdatesTest.java @@ -16,10 +16,8 @@ */ package org.apache.ignite.internal.processors.database; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import org.apache.ignite.Ignite; @@ -28,19 +26,12 @@ import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; -import org.apache.ignite.cluster.BaselineNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; -import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState; -import org.apache.ignite.internal.util.typedef.PA; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.WithSystemProperty; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.After; @@ -49,6 +40,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED; import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_PAGE_SIZE; @@ -120,6 +112,11 @@ public static Iterable setup() { cfg.setDataStorageConfiguration(storeCfg); + cfg.setCacheConfiguration(new CacheConfiguration(DEF_CACHE_NAME) + .setAffinity(new RendezvousAffinityFunction(false, 1)) + .setCacheMode(CacheMode.REPLICATED) + .setAtomicityMode(cacheAtomicityMode)); + return cfg; } @@ -142,20 +139,18 @@ public void after() throws Exception { } /** - * + * @throws Exception If failed. */ @Test @WithSystemProperty(key = IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, value = "true") + @WithSystemProperty(key = IGNITE_PDS_WAL_REBALANCE_THRESHOLD, value = "100") + @WithSystemProperty(key = IGNITE_BASELINE_AUTO_ADJUST_ENABLED, value = "false") public void testBatchRebalance() throws Exception { - Ignite node = startGrid(0); - - node.cluster().active(true); + Ignite node1 = startGrid(0); - node.cluster().baselineAutoAdjustEnabled(false); + node1.cluster().active(true); - node.createCache(ccfg()); - - int cnt = 100_000; + int cnt = 50_000; int minSize = 0; int maxSize = 16384; @@ -170,155 +165,85 @@ public void testBatchRebalance() throws Exception { srcMap.put(i, obj); } - try (IgniteDataStreamer streamer = node.dataStreamer(DEF_CACHE_NAME)) { + try (IgniteDataStreamer streamer = node1.dataStreamer(DEF_CACHE_NAME)) { streamer.addData(srcMap); } log.info("Data loaded."); if (persistence) - node.cluster().active(false); + node1.cluster().active(false); final IgniteEx node2 = startGrid(1); if (persistence) { - List list = new ArrayList<>(node.cluster().currentBaselineTopology()); - - list.add(node2.localNode()); + node1.cluster().active(true); - node.cluster().active(true); - - node.cluster().setBaselineTopology(list); + node1.cluster().setBaselineTopology(node1.cluster().forServers().nodes()); } log.info("Await rebalance."); - awaitRebalance(node2, DEF_CACHE_NAME); - - node.close(); - - validateCacheEntries(node2.cache(DEF_CACHE_NAME), srcMap); - - if (persistence) { - node2.close(); - - Ignite ignite = startGrid(1); - - ignite.cluster().active(true); - - log.info("Validate entries after restart"); - - validateCacheEntries(ignite.cache(DEF_CACHE_NAME), srcMap); - } - } - - /** - * - */ - @Test - @WithSystemProperty(key = IGNITE_PDS_WAL_REBALANCE_THRESHOLD, value = "100") - @WithSystemProperty(key = IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, value = "true") - public void testBatchHistoricalRebalance() throws Exception { - if (!persistence) - return; - - // TODO https://issues.apache.org/jira/browse/IGNITE-7384 - // TODO http://apache-ignite-developers.2346864.n4.nabble.com/Historical-rebalance-td38380.html - if (cacheAtomicityMode == CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT) - return; - - Ignite node = startGrids(2); - - node.cluster().active(true); - - IgniteCache cache = node.createCache(ccfg()); - - int cnt = 10_000; - - log.info("Loading " + cnt + " random entries."); - - Map srcMap = new HashMap<>(); - - for (int i = 0; i < cnt; i++) { - byte[] obj = new byte[ThreadLocalRandom.current().nextInt(16384)]; + awaitPartitionMapExchange(); - if (i > 0 && i < 1000 && i % 37 == 0) - continue; + if (persistence) + forceCheckpoint(node1); - srcMap.put(i, obj); - } + node1.close(); - try (IgniteDataStreamer streamer = node.dataStreamer(DEF_CACHE_NAME)) { - streamer.addData(srcMap); - } + IgniteCache cache = node2.cache(DEF_CACHE_NAME); - forceCheckpoint(); + validateCacheEntries(cache, srcMap); - log.info("Stopping node #2."); + // Check WAL rebalance. + if (persistence) { + log.info("Updating values on node #1."); - grid(1).close(); + // Update existing extries. + for (int i = 100; i < 5_000; i++) { + if (i % 3 == 0) { + cache.remove(i); - log.info("Updating values on node #1."); + srcMap.remove(i); + } + else { + byte[] bytes = new byte[ThreadLocalRandom.current().nextInt(maxSize)]; - for (int i = 100; i < 2000; i++) { - if (i % 3 == 0) { - cache.remove(i); + Arrays.fill(bytes, (byte)1); - srcMap.remove(i); + srcMap.put(i, bytes); + cache.put(i, bytes); + } } - else { - byte[] bytes = new byte[ThreadLocalRandom.current().nextInt(16384)]; - Arrays.fill(bytes, (byte)1); + // New entries. + for (int i = cnt; i < cnt + 1_000; i++) { + byte[] bytes = new byte[ThreadLocalRandom.current().nextInt(maxSize)]; srcMap.put(i, bytes); cache.put(i, bytes); } - } - - for (int i = 10_000; i < 11_000; i++) { - byte[] bytes = new byte[ThreadLocalRandom.current().nextInt(16384)]; - - srcMap.put(i, bytes); - cache.put(i, bytes); - } - - forceCheckpoint(); - log.info("Starting node #2."); + forceCheckpoint(node2); - IgniteEx node2 = startGrid(1); + log.info("Starting node #1"); - log.info("Await rebalance on node #2."); + node1 = startGrid(0); - awaitRebalance(node2, DEF_CACHE_NAME); + node1.cluster().active(true); - log.info("Stop node #1."); + node1.cluster().setBaselineTopology(node1.cluster().forServers().nodes()); - node.close(); + log.info("Await rebalance on node #1."); - validateCacheEntries(node2.cache(DEF_CACHE_NAME), srcMap); - } - - /** - * @param node Ignite node. - * @param name Cache name. - */ - private void awaitRebalance(IgniteEx node, String name) throws IgniteInterruptedCheckedException { - boolean ok = GridTestUtils.waitForCondition(new PA() { - @Override public boolean apply() { - for ( GridDhtLocalPartition part : node.context().cache().cache(name).context().group().topology().localPartitions()) { - if (part.state() != GridDhtPartitionState.OWNING) - return false; - } + awaitPartitionMapExchange(); - return true; - } - }, 60_000); + log.info("Stop node #2."); - U.sleep(3000); + node2.close(); - assertTrue(ok); + validateCacheEntries(node1.cache(DEF_CACHE_NAME), srcMap); + } } /** @@ -339,21 +264,4 @@ private void validateCacheEntries(IgniteCache cache, Map map) { assertEquals(idx, e.getValue().length, ((byte[])cache.get(e.getKey())).length); } } - - /** - * @return Cache configuration. - */ - private CacheConfiguration ccfg() { - return ccfg(1, CacheMode.REPLICATED); - } - - /** - * @return Cache configuration. - */ - private CacheConfiguration ccfg(int parts, CacheMode mode) { - return new CacheConfiguration(DEF_CACHE_NAME) - .setAffinity(new RendezvousAffinityFunction(false, parts)) - .setCacheMode(mode) - .setAtomicityMode(cacheAtomicityMode); - } } From 37e8989225480494d0e054f38155c8af788df64e Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Wed, 3 Apr 2019 19:52:10 +0300 Subject: [PATCH 06/24] IGNTIE-11584 Remove invokeAll. --- .../processors/cache/CacheMapEntries.java | 438 +++++------------- .../processors/cache/CacheMapEntryInfo.java | 173 +++++++ .../cache/IgniteCacheOffheapManager.java | 39 +- .../cache/IgniteCacheOffheapManagerImpl.java | 188 +++++++- .../preloader/GridDhtPartitionDemander.java | 36 +- .../persistence/GridCacheOffheapManager.java | 10 +- .../cache/persistence/tree/BPlusTree.java | 5 - .../processors/cache/tree/CacheDataTree.java | 88 ---- .../ignite/internal/util/IgniteTree.java | 36 -- 9 files changed, 497 insertions(+), 516 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntryInfo.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java index a7e583c398daf..873e9f4a0b9cd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -19,117 +19,141 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.OffheapInvokeAllClosure; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; -import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; -import org.apache.ignite.internal.processors.cache.tree.DataRow; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; -import org.apache.ignite.internal.processors.dr.GridDrType; -import org.apache.ignite.internal.util.IgniteTree; -import org.apache.ignite.internal.util.typedef.T3; -import org.apache.ignite.internal.util.typedef.internal.CU; - -import static org.apache.ignite.internal.util.IgniteTree.OperationType.NOOP; -import static org.apache.ignite.internal.util.IgniteTree.OperationType.PUT; -import static org.apache.ignite.internal.util.IgniteTree.OperationType.REMOVE; + +import static org.apache.ignite.internal.processors.cache.GridCacheMapEntry.ATOMIC_VER_COMPARATOR; /** * Batch of cache map entries. */ public class CacheMapEntries { /** */ - private final GridDhtLocalPartition part; + static class BatchContext { + /** */ + private final GridCacheContext cctx; - /** */ - private final GridCacheContext cctx; + /** */ + private final GridDhtLocalPartition part; - /** */ - private final AffinityTopologyVersion topVer; + /** */ + private final boolean preload; - /** */ - private final boolean preload; + /** */ + private final AffinityTopologyVersion topVer; - /** */ - private final LinkedHashMap infos = new LinkedHashMap<>(); + /** */ + private boolean sorted; - /** */ - private Set skipped = new HashSet<>(); + /** */ + private final Set skipped = new HashSet<>(); - /** */ - private boolean ordered = true; + /** */ + BatchContext(GridCacheContext cctx, int partId, boolean preload, AffinityTopologyVersion topVer) { + this.cctx = cctx; + this.preload = preload; + this.topVer = topVer; - /** */ - private KeyCacheObject lastKey; + part = cctx.topology().localPartition(partId, topVer, true, true); + } - /** */ - public CacheMapEntries(AffinityTopologyVersion topVer, int partId, GridCacheContext cctx, boolean preload) { - this.topVer = topVer; - this.cctx = cctx; - this.preload = preload; - this.part = cctx.topology().localPartition(partId, topVer, true, true); - } + /** */ + void markRemoved(KeyCacheObject key) { + skipped.add(key); + } - /** */ - public void add(KeyCacheObject key, CacheObject val, long expTime, long ttl, GridCacheVersion ver, GridDrType drType) { - if (lastKey != null && ordered && lastKey.hashCode() >= key.hashCode()) - ordered = false; + /** */ + boolean preload() { + return preload; + } - CacheMapEntryInfo old = - infos.put(lastKey = key, new CacheMapEntryInfo(this, val, expTime, ttl, ver, drType)); + /** */ + boolean sorted() { + return sorted; + } - assert old == null || GridCacheMapEntry.ATOMIC_VER_COMPARATOR.compare(old.version(), ver) < 0 : - "Version order mismatch: prev=" + old.version() + ", current=" + ver; - } + /** */ + AffinityTopologyVersion topVer() { + return topVer; + } - /** */ - public GridDhtLocalPartition part() { - return part; - } + /** */ + GridDhtLocalPartition part() { + return part; + } - /** */ - public GridCacheContext context() { - return cctx; + /** */ + GridCacheContext context() { + return cctx; + } + + /** */ + boolean skip(KeyCacheObject key) { + return skipped.contains(key); + } + + /** */ + void sorted(boolean sorted) { + this.sorted = sorted; + } } /** */ - public Set keys() { - return infos.keySet(); - } + public int initialValues( + List infos, + AffinityTopologyVersion topVer, + GridCacheContext cctx, + int partId, + boolean preload + ) throws IgniteCheckedException { + BatchContext ctx = new BatchContext(cctx, partId, preload, topVer); - /** - * @return Count of batch entries. - */ - public int size() { - return infos.size() - skipped.size(); - } + Collection locked = lock(ctx, infos); - /** - * @return Off heap update closure. - */ - public OffheapInvokeAllClosure offheapUpdateClosure() { - return new UpdateAllClosure(this, cctx); + try { + cctx.offheap().updateAll(ctx.context(), ctx.part(), infos, ctx.sorted()); + } finally { + unlock(ctx, locked); + } + + return infos.size() - ctx.skipped.size(); } /** */ - public void lock() { + private Collection lock(BatchContext ctx, Collection infos) { List locked = new ArrayList<>(infos.size()); + boolean ordered = true; + while (true) { - for (Map.Entry e : infos.entrySet()) { - GridDhtCacheEntry entry = (GridDhtCacheEntry)cctx.cache().entryEx(e.getKey(), topVer); + Map uniqueEntries = new HashMap<>(); + + KeyCacheObject lastKey = null; + + for (CacheMapEntryInfo e : infos) { + KeyCacheObject key = e.key(); + CacheMapEntryInfo old = uniqueEntries.put(key, e); + + assert old == null || ATOMIC_VER_COMPARATOR.compare(old.version(), e.version()) < 0 : + "Version order mismatch: prev=" + old.version() + ", current=" + e.version(); + + if (ordered && lastKey != null && lastKey.hashCode() >= key.hashCode()) + ordered = false; + + GridDhtCacheEntry entry = (GridDhtCacheEntry)ctx.cctx.cache().entryEx(key, ctx.topVer()); locked.add(entry); - e.getValue().cacheEntry(entry); + e.init(ctx, entry); + + lastKey = key; } boolean retry = false; @@ -159,38 +183,41 @@ public void lock() { } } - if (!retry) - return; + if (!retry) { + ctx.sorted(ordered); + + return uniqueEntries.values(); + } } } /** * Releases java-level locks on cache entries. */ - public void unlock() { + private void unlock(BatchContext ctx, Collection infos) { // Process deleted entries before locks release. // todo - assert cctx.deferredDelete() : this; + assert ctx.cctx.deferredDelete() : this; // Entries to skip eviction manager notification for. // Enqueue entries while holding locks. int size = infos.size(); try { - for (Map.Entry e : infos.entrySet()) { - KeyCacheObject key = e.getKey(); - GridCacheMapEntry entry = e.getValue().cacheEntry(); + for (CacheMapEntryInfo e : infos) { + KeyCacheObject key = e.key(); + GridCacheMapEntry entry = e.cacheEntry(); - if (skipped.contains(key)) + if (ctx.skip(key)) continue; if (entry != null && entry.deleted()) - skipped.add(entry.key()); + ctx.markRemoved(entry.key()); try { - e.getValue().updateCacheEntry(); + e.updateCacheEntry(); } catch (IgniteCheckedException ex) { - skipped.add(entry.key()); + ctx.markRemoved(entry.key()); } } } @@ -198,7 +225,7 @@ public void unlock() { // At least RuntimeException can be thrown by the code above when GridCacheContext is cleaned and there is // an attempt to use cleaned resources. // That's why releasing locks in the finally block.. - for (CacheMapEntryInfo info : infos.values()) { + for (CacheMapEntryInfo info : infos) { GridCacheMapEntry entry = info.cacheEntry(); if (entry != null) @@ -207,263 +234,24 @@ public void unlock() { } // Try evict partitions. - for (CacheMapEntryInfo info : infos.values()) { + for (CacheMapEntryInfo info : infos) { GridDhtCacheEntry entry = info.cacheEntry(); if (entry != null) entry.onUnlock(); } - if (skipped.size() == size) + if (ctx.skipped.size() == size) // Optimization. return; // Must touch all entries since update may have deleted entries. // Eviction manager will remove empty entries. - for (CacheMapEntryInfo info : infos.values()) { + for (CacheMapEntryInfo info : infos) { GridCacheMapEntry entry = info.cacheEntry(); - if (entry != null && !skipped.contains(entry.key())) + if (entry != null && !ctx.skip(entry.key())) entry.touch(); } } - - /** */ - public void onRemove(KeyCacheObject key) { - skipped.add(key); - } - - /** */ - boolean preload() { - return preload; - } - - /** */ - boolean skip(KeyCacheObject key) { - return skipped.contains(key); - } - - /** */ - private static class UpdateAllClosure implements OffheapInvokeAllClosure { - /** */ - private final List> finalOps; - - /** */ - private final int cacheId; - - /** */ - private final CacheMapEntries entries; - - /** */ - public UpdateAllClosure(CacheMapEntries entries, GridCacheContext cctx) { - this.entries = entries; - finalOps = new ArrayList<>(entries.size()); - cacheId = cctx.group().storeCacheIdInDataPage() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; - } - - /** {@inheritDoc} */ - @Override public void call(Collection rows) throws IgniteCheckedException { - assert rows.size() == entries.size() : "size mismatch, expect=" + entries.size() + ", input=" + rows.size(); - - List newRows = new ArrayList<>(8); - - int partId = entries.part().id(); - GridCacheContext cctx = entries.context(); - - Iterator rowsIter = rows.iterator(); - - for (Map.Entry e : entries.infos.entrySet()) { - KeyCacheObject key = e.getKey(); - CacheMapEntryInfo newRowInfo = e.getValue(); - CacheDataRow oldRow = rowsIter.next(); - - try { - if (newRowInfo.needUpdate(oldRow)) { - CacheObject val = newRowInfo.value(); - - if (val != null) { - if (oldRow != null) { - CacheDataRow newRow = cctx.offheap().dataStore(entries.part()).createRow( - cctx, - key, - newRowInfo.value(), - newRowInfo.version(), - newRowInfo.expireTime(), - oldRow); - - finalOps.add(new T3<>(oldRow.link() == newRow.link() ? NOOP : PUT, oldRow, newRow)); - } - else { - CacheObjectContext coCtx = cctx.cacheObjectContext(); - - val.valueBytes(coCtx); - key.valueBytes(coCtx); - - if (key.partition() == -1) - key.partition(partId); - - DataRow newRow = new DataRow(key, val, newRowInfo.version(), partId, - newRowInfo.expireTime(), cacheId); - - newRows.add(newRow); - - finalOps.add(new T3<>(PUT, oldRow, newRow)); - } - } - else { - // todo we should pass key somehow to remove old row - // todo (in particular case oldRow should not contain key) - DataRow newRow = new DataRow(key, null, null, 0, 0, cacheId); - - finalOps.add(new T3<>(oldRow != null ? REMOVE : NOOP, oldRow, newRow)); - } - } - } - catch (GridCacheEntryRemovedException ex) { - entries.onRemove(key); - } - } - - if (!newRows.isEmpty()) { - // cctx.shared().database().ensureFreeSpace(cctx.dataRegion()); - - cctx.offheap().dataStore(entries.part()).rowStore(). - addRows(newRows, cctx.group().statisticsHolderData()); - - if (cacheId == CU.UNDEFINED_CACHE_ID) { - // Set cacheId before write keys into tree. - for (DataRow row : newRows) - row.cacheId(cctx.cacheId()); - - } - } - } - - /** {@inheritDoc} */ - @Override public Collection> result() { - return finalOps; - } - - /** {@inheritDoc} */ - @Override public boolean fastpath() { - return entries.ordered; - } - } - - /** */ - private static class CacheMapEntryInfo { - /** */ - private final CacheMapEntries batch; - - /** */ - private final CacheObject val; - - /** */ - private final long expireTime; - - /** */ - private final long ttl; - - /** */ - private final GridCacheVersion ver; - - /** */ - private final GridDrType drType; - - /** */ - private GridDhtCacheEntry entry; - - /** */ - private boolean update; - - /** */ - public CacheMapEntryInfo( - CacheMapEntries batch, - CacheObject val, - long expireTime, - long ttl, - GridCacheVersion ver, - GridDrType drType - ) { - this.batch = batch; - this.val = val; - this.expireTime = expireTime; - this.ver = ver; - this.drType = drType; - this.ttl = ttl; - } - - /** - * @return Version. - */ - public GridCacheVersion version() { - return ver; - } - - /** - * @return Value. - */ - public CacheObject value() { - return val; - } - - /** - * @return Expire time. - */ - public long expireTime() { - return expireTime; - } - - /** - * @param entry Cache entry. - */ - public void cacheEntry(GridDhtCacheEntry entry) { - this.entry = entry; - } - - /** - * @return Cache entry. - */ - public GridDhtCacheEntry cacheEntry() { - return entry; - } - - /** */ - public void updateCacheEntry() throws IgniteCheckedException { - if (!update) - return; - - entry.finishInitialUpdate(val, expireTime, ttl, ver, batch.topVer, drType, null, batch.preload); - } - - /** */ - public boolean needUpdate(CacheDataRow row) throws GridCacheEntryRemovedException { - GridCacheVersion currVer = row != null ? row.version() : entry.version(); - - GridCacheContext cctx = batch.context(); - - boolean isStartVer = cctx.versions().isStartVersion(currVer); - - boolean update0; - - if (cctx.group().persistenceEnabled()) { - if (!isStartVer) { - if (cctx.atomic()) - update0 = GridCacheMapEntry.ATOMIC_VER_COMPARATOR.compare(currVer, version()) < 0; - else - update0 = currVer.compareTo(version()) < 0; - } - else - update0 = true; - } - else - update0 = (isStartVer && row == null); - - update0 |= (!batch.preload() && entry.deletedUnlocked()); - - update = update0; - - return update0; - } - } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntryInfo.java new file mode 100644 index 0000000000000..b7a4883447375 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntryInfo.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.CacheMapEntries.BatchContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.dr.GridDrType; + +/** + * + */ +public class CacheMapEntryInfo { + /** */ + private BatchContext ctx; + + /** */ + private final KeyCacheObject key; + + /** */ + private final CacheObject val; + + /** */ + private final long expireTime; + + /** */ + private final long ttl; + + /** */ + private final GridCacheVersion ver; + + /** */ + private final GridDrType drType; + + /** */ + private GridDhtCacheEntry entry; + + /** */ + private boolean update; + + /** + * + */ + public CacheMapEntryInfo( + KeyCacheObject key, + CacheObject val, + long expireTime, + long ttl, + GridCacheVersion ver, + GridDrType drType + ) { + assert key != null; + + this.key = key; + this.val = val; + this.ver = ver; + this.ttl = ttl; + this.drType = drType; + this.expireTime = expireTime; + } + + /** + * + */ + public KeyCacheObject key() { + return key; + } + + /** + * @return Version. + */ + public GridCacheVersion version() { + return ver; + } + + /** + * @return Value. + */ + public CacheObject value() { + return val; + } + + /** + * @return Expire time. + */ + public long expireTime() { + return expireTime; + } + + /** + * + */ + public void onRemove() { + ctx.markRemoved(key); + } + + /** + * @return Cache entry. + */ + GridDhtCacheEntry cacheEntry() { + return entry; + } + + /** + * + */ + void init(BatchContext ctx, GridDhtCacheEntry entry) { + assert ctx != null; + assert entry != null; + + this.ctx = ctx; + this.entry = entry; + } + + /** + * + */ + void updateCacheEntry() throws IgniteCheckedException { + if (!update) + return; + + entry.finishInitialUpdate(val, expireTime, ttl, ver, ctx.topVer(), drType, null, ctx.preload()); + } + + /** + * + */ + boolean needUpdate(CacheDataRow row) throws GridCacheEntryRemovedException { + GridCacheVersion currVer = row != null ? row.version() : entry.version(); + + GridCacheContext cctx = ctx.context(); + + boolean isStartVer = cctx.versions().isStartVersion(currVer); + + boolean update0; + + if (cctx.group().persistenceEnabled()) { + if (!isStartVer) { + if (cctx.atomic()) + update0 = GridCacheMapEntry.ATOMIC_VER_COMPARATOR.compare(currVer, version()) < 0; + else + update0 = currVer.compareTo(version()) < 0; + } + else + update0 = true; + } + else + update0 = (isStartVer && row == null); + + update0 |= (!ctx.preload() && entry.deletedUnlocked()); + + update = update0; + + return update0; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java index 19c6eaa671ef2..dd33d91aa120f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java @@ -48,7 +48,6 @@ import org.apache.ignite.internal.util.lang.GridIterator; import org.apache.ignite.internal.util.lang.IgniteInClosure2X; import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; /** @@ -192,16 +191,16 @@ public void invoke(GridCacheContext cctx, KeyCacheObject key, GridDhtLocalPartit /** * @param cctx Cache context. - * @param keys Keys. * @param part Partition. - * @param c Tree batch update closure. + * @param entries Entries. + * @param sorted Sorted flag. * @throws IgniteCheckedException If failed. */ - public void invokeAll( + public void updateAll( GridCacheContext cctx, - Collection keys, GridDhtLocalPartition part, - OffheapInvokeAllClosure c + Collection entries, + boolean sorted ) throws IgniteCheckedException; /** @@ -595,13 +594,6 @@ interface OffheapInvokeClosure extends IgniteTree.InvokeClosure { @Nullable public CacheDataRow oldRow(); } - /** - * - */ - interface OffheapInvokeAllClosure extends IgniteTree.InvokeAllClosure { -// boolean preload(); - } - /** * */ @@ -741,6 +733,19 @@ void update( long expireTime, @Nullable CacheDataRow oldRow) throws IgniteCheckedException; + + /** + * @param cctx Cache context. + * @param entries Entries. + * @param sorted Sorted flag. + * @throws IgniteCheckedException If failed. + */ + public void updateAll( + GridCacheContext cctx, + Collection entries, + boolean sorted + ) throws IgniteCheckedException; + /** * @param cctx Cache context. * @param key Key. @@ -884,14 +889,6 @@ MvccUpdateResult mvccLock( */ public void invoke(GridCacheContext cctx, KeyCacheObject key, OffheapInvokeClosure c) throws IgniteCheckedException; - /** - * @param cctx Cache context. - * @param keys Keys. - * @param c Closure. - * @throws IgniteCheckedException If failed. - */ - public void invokeAll(GridCacheContext cctx, Collection keys, OffheapInvokeAllClosure c) throws IgniteCheckedException; - /** * * @param cctx Cache context. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index d181bd726f005..7c29dabd77356 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -37,7 +37,6 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageMvccMarkUpdatedRecord; @@ -136,6 +135,7 @@ import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.MVCC_INFO_SIZE; import static org.apache.ignite.internal.util.IgniteTree.OperationType.NOOP; import static org.apache.ignite.internal.util.IgniteTree.OperationType.PUT; +import static org.apache.ignite.internal.util.IgniteTree.OperationType.REMOVE; /** * @@ -437,13 +437,13 @@ private Iterator cacheData(boolean primary, boolean backup, Affi } /** {@inheritDoc} */ - @Override public void invokeAll( + @Override public void updateAll( GridCacheContext cctx, - Collection keys, GridDhtLocalPartition part, - OffheapInvokeAllClosure c) - throws IgniteCheckedException { - dataStore(part).invokeAll(cctx, keys, c); + Collection entries, + boolean sorted + ) throws IgniteCheckedException { + dataStore(part).updateAll(cctx, entries, sorted); } /** {@inheritDoc} */ @@ -1621,16 +1621,19 @@ private boolean canUpdateOldRow(GridCacheContext cctx, @Nullable CacheDataRow ol /** {@inheritDoc} */ - @Override public void invokeAll(GridCacheContext cctx, Collection keys, OffheapInvokeAllClosure c) - throws IgniteCheckedException { + @Override public void updateAll( + GridCacheContext cctx, + Collection entries, + boolean sorted + ) throws IgniteCheckedException { int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; - List searchRows = new ArrayList<>(keys.size()); + List searchRows = new ArrayList<>(entries.size()); - for (KeyCacheObject key : keys) - searchRows.add(new SearchRow(cacheId, key)); + for (CacheMapEntryInfo entry : entries) + searchRows.add(new SearchRow(cacheId, entry.key())); - invokeAll0(cctx, searchRows, c); + updateAll0(cctx, searchRows, sorted, entries); } /** @@ -1674,21 +1677,170 @@ private void invoke0(GridCacheContext cctx, CacheSearchRow row, OffheapInvokeClo /** * @param cctx Cache context. - * @param rows Search rows. - * @param c Closure. + * @param keys Search rows. + * @param sorted Sorted flag. + * @param entries Entries. * @throws IgniteCheckedException If failed. */ - private void invokeAll0(GridCacheContext cctx, List rows, OffheapInvokeAllClosure c) - throws IgniteCheckedException { + private void updateAll0( + GridCacheContext cctx, + List keys, + boolean sorted, + Collection entries + ) throws IgniteCheckedException { if (!busyLock.enterBusy()) throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); try { assert cctx.shared().database().checkpointLockIsHeldByThread(); - dataTree.invokeAll(rows, CacheDataRowAdapter.RowData.NO_KEY, c); + int keysCnt = keys.size(); + + List rows = new ArrayList<>(keysCnt); + + if (!sorted) { + for (CacheSearchRow row : keys) { + CacheDataRow old = dataTree.findOne(row, null, CacheDataRowAdapter.RowData.NO_KEY); + + rows.add(old); + } + } + else { + GridCursor cur = + dataTree.find(keys.get(0), keys.get(keysCnt - 1), CacheDataRowAdapter.RowData.FULL); + + Iterator keyItr = keys.iterator(); + + CacheSearchRow lastRow = null; + CacheSearchRow row = null; + KeyCacheObject key = null; + + CacheDataRow oldRow = null; + KeyCacheObject oldKey = null; + + while (cur.next()) { + oldRow = cur.get(); + oldKey = oldRow.key(); + + while (key == null || key.hashCode() <= oldKey.hashCode()) { + if (key != null && key.hashCode() == oldKey.hashCode()) { + while (key.hashCode() == oldKey.hashCode()) { + // todo test collision resolution + rows.add(key.equals(oldKey) ? oldRow : null); + + lastRow = null; + + if (!keyItr.hasNext()) + break; + + lastRow = row = keyItr.next(); + key = row.key(); + } + } + else { + if (row != null) + rows.add(null); + + lastRow = null; + + if (keyItr.hasNext()) { + lastRow = row = keyItr.next(); + key = lastRow.key(); + } + } + + if (!keyItr.hasNext()) + break; + } + } + + if (lastRow != null) + rows.add(key.equals(oldKey) ? oldRow : null); + + for (; keyItr.hasNext(); keyItr.next()) + rows.add(null); + } + + List newRows = new ArrayList<>(8); + + int cacheId = cctx.group().storeCacheIdInDataPage() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + Iterator rowsIter = rows.iterator(); + + List> finalOps = new ArrayList<>(keys.size()); + + for (CacheMapEntryInfo e : entries) { + KeyCacheObject key = e.key(); + CacheDataRow oldRow = rowsIter.next(); + + try { + if (e.needUpdate(oldRow)) { + CacheObject val = e.value(); + + if (val != null) { + if (oldRow != null) { + CacheDataRow newRow = createRow( + cctx, + key, + e.value(), + e.version(), + e.expireTime(), + oldRow); + + finalOps.add(new T3<>(oldRow.link() == newRow.link() ? NOOP : PUT, oldRow, newRow)); + } + else { + CacheObjectContext coCtx = cctx.cacheObjectContext(); + + val.valueBytes(coCtx); + key.valueBytes(coCtx); + + if (key.partition() == -1) + key.partition(partId); + + DataRow newRow = new DataRow(key, val, e.version(), partId, e.expireTime(), cacheId); + + newRows.add(newRow); + + finalOps.add(new T3<>(PUT, oldRow, newRow)); + } + } + else { + // todo we should pass key somehow to remove old row + // todo (in particular case oldRow should not contain key) + DataRow newRow = new DataRow(key, null, null, 0, 0, cacheId); + + finalOps.add(new T3<>(oldRow != null ? REMOVE : NOOP, oldRow, newRow)); + } + } + } + catch (GridCacheEntryRemovedException ex) { + e.onRemove(); + } + } + + if (!newRows.isEmpty()) { + // cctx.shared().database().ensureFreeSpace(cctx.dataRegion()); + + rowStore().addRows(newRows, cctx.group().statisticsHolderData()); + + if (cacheId == CU.UNDEFINED_CACHE_ID) { + // Set cacheId before write keys into tree. + for (DataRow row : newRows) + row.cacheId(cctx.cacheId()); + } + } + + for (T3 t3 : finalOps) { + IgniteTree.OperationType oper = t3.get1(); + + if (oper == IgniteTree.OperationType.PUT) + dataTree.putx(t3.get3()); + else if (oper == IgniteTree.OperationType.REMOVE) + dataTree.removex(t3.get2()); // old row + } - for (T3 finalOp : c.result()) { + for (T3 finalOp : finalOps) { IgniteTree.OperationType opType = finalOp.get1(); CacheDataRow oldRow = finalOp.get2(); CacheDataRow newRow = finalOp.get3(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index bcde7cdc3818d..5b33df13fe5a7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -47,11 +47,13 @@ import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheMapEntries; +import org.apache.ignite.internal.processors.cache.CacheMapEntryInfo; import org.apache.ignite.internal.processors.cache.CacheMetricsImpl; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; +import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; import org.apache.ignite.internal.processors.cache.GridCacheMvccEntryInfo; import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; @@ -878,19 +880,19 @@ public void handleSupplyMessage( */ private void preloadEntriesBatch(ClusterNode from, int p, - Collection entries, + Collection infos, AffinityTopologyVersion topVer ) throws IgniteCheckedException { - if (entries.isEmpty()) + if (infos.isEmpty()) return; grp.listenerLock().readLock().lock(); try { - Map cctxs = new HashMap<>(); + Map> cctxs = new HashMap<>(); // Map by context. - for (GridCacheEntryInfo e : entries) { + for (GridCacheEntryInfo e : infos) { try { GridCacheContext cctx0 = grp.sharedGroup() ? ctx.cacheContext(e.cacheId()) : grp.singleCacheContext(); @@ -905,10 +907,8 @@ private void preloadEntriesBatch(ClusterNode from, if (log.isTraceEnabled()) log.trace("Rebalancing key [key=" + e.key() + ", part=" + p + ", node=" + from.id() + ']'); - CacheMapEntries batch = - cctxs.computeIfAbsent(cctx.cacheId(), v -> new CacheMapEntries(topVer, p, cctx, true)); - - batch.add(e.key(), e.value(), e.expireTime(), e.ttl(), e.version(), DR_PRELOAD); + cctxs.computeIfAbsent(cctx.cacheId(), v -> new ArrayList<>()).add( + new CacheMapEntryInfo(e.key(), e.value(), e.expireTime(), e.ttl(), e.version(), DR_PRELOAD)); } catch (GridDhtInvalidPartitionException ignored) { if (log.isDebugEnabled()) @@ -916,21 +916,17 @@ private void preloadEntriesBatch(ClusterNode from, } } - for (CacheMapEntries batch : cctxs.values()) { - GridCacheContext cctx = batch.context(); - batch.lock(); + for (Map.Entry> e : cctxs.entrySet()) { + GridCacheContext cctx = ctx.cacheContext(e.getKey()); - try { - cctx.offheap().invokeAll(cctx, batch.keys(), batch.part(), batch.offheapUpdateClosure()); - } - finally { - batch.unlock(); + CacheMapEntries entries = new CacheMapEntries(); - for (GridCacheContext cctx0 : grp.caches()) { - if (cctx0.statisticsEnabled()) - cctx0.cache().metrics0().onRebalanceKeysReceived(batch.size()); - } + int initialized = entries.initialValues(e.getValue(), topVer, cctx, p, true); + + for (GridCacheContext cctx0 : grp.caches()) { + if (cctx0.statisticsEnabled()) + cctx0.cache().metrics0().onRebalanceKeysReceived(initialized); } } } finally { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java index 1c2cbedf11452..714690b703400 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java @@ -53,6 +53,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.CacheMapEntryInfo; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; @@ -2132,13 +2133,16 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { } /** {@inheritDoc} */ - @Override public void invokeAll(GridCacheContext cctx, Collection keys, OffheapInvokeAllClosure c) - throws IgniteCheckedException { + @Override public void updateAll( + GridCacheContext cctx, + Collection entries, + boolean sorted + ) throws IgniteCheckedException { assert ctx.database().checkpointLockIsHeldByThread(); CacheDataStore delegate = init0(false); - delegate.invokeAll(cctx, keys, c); + delegate.updateAll(cctx, entries, sorted); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java index 10b082a72af28..54d9816bcb7e2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java @@ -1822,11 +1822,6 @@ public final boolean removex(L row) throws IgniteCheckedException { } } - /** {@inheritDoc} */ - @Override public void invokeAll(List keys, Object z, InvokeAllClosure c) throws IgniteCheckedException { - throw new UnsupportedOperationException("Not implemented yet"); - } - /** * @param x Invoke operation. * @param pageId Page ID. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java index 12f59358441d8..b3c1c69e66319 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java @@ -17,9 +17,6 @@ package org.apache.ignite.internal.processors.cache.tree; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.PageUtils; import org.apache.ignite.internal.pagemem.store.PageStore; @@ -48,7 +45,6 @@ import org.apache.ignite.internal.stat.IoStatisticsHolder; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.lang.GridCursor; -import org.apache.ignite.internal.util.typedef.T3; import org.apache.ignite.internal.util.typedef.internal.CU; import static java.lang.Boolean.FALSE; @@ -331,90 +327,6 @@ public CacheDataRowStore rowStore() { return rowStore; } - /** - * todo Stub implementation. After implementing BPlusTree.invokeAll(...), this stub will be removed. - */ - @Override public void invokeAll(List keys, Object x, InvokeAllClosure c) - throws IgniteCheckedException { - checkDestroyed(); - - int keysCnt = keys.size(); - - List rows = new ArrayList<>(keysCnt); - - if (c.fastpath()) { - GridCursor cur = find(keys.get(0), keys.get(keysCnt - 1), CacheDataRowAdapter.RowData.FULL); - Iterator keyItr = keys.iterator(); - - CacheSearchRow lastRow = null; - CacheSearchRow row = null; - KeyCacheObject key = null; - - CacheDataRow oldRow = null; - KeyCacheObject oldKey = null; - - while (cur.next()) { - oldRow = cur.get(); - oldKey = oldRow.key(); - - while (key == null || key.hashCode() <= oldKey.hashCode()) { - if (key != null && key.hashCode() == oldKey.hashCode()) { - while (key.hashCode() == oldKey.hashCode()) { - // todo test collision resolution - rows.add(key.equals(oldKey) ? oldRow : null); - - lastRow = null; - - if (!keyItr.hasNext()) - break; - - lastRow = row = keyItr.next(); - key = row.key(); - } - } - else { - if (row != null) - rows.add(null); - - lastRow = null; - - if (keyItr.hasNext()) { - lastRow = row = keyItr.next(); - key = lastRow.key(); - } - } - - if (!keyItr.hasNext()) - break; - } - } - - if (lastRow != null) - rows.add(key.equals(oldKey) ? oldRow : null); - - for (; keyItr.hasNext(); keyItr.next()) - rows.add(null); - } else { - for (CacheSearchRow row : keys) { - // todo NO_KEY - CacheDataRow oldRow = findOne(row, null, CacheDataRowAdapter.RowData.FULL); - - rows.add(oldRow); - } - } - - c.call(rows); - - for (T3 t3 : c.result()) { - OperationType oper = t3.get1(); - - if (oper == OperationType.PUT) - putx(t3.get3()); - else if (oper == OperationType.REMOVE) - removex(t3.get2()); // old row - } - } - /** {@inheritDoc} */ @Override protected int compare(BPlusIO iox, long pageAddr, int idx, CacheSearchRow row) throws IgniteCheckedException { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java index 728b3d98ec576..9e854d28f6cb0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java @@ -17,11 +17,8 @@ package org.apache.ignite.internal.util; -import java.util.Collection; -import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.util.lang.GridCursor; -import org.apache.ignite.internal.util.typedef.T3; import org.jetbrains.annotations.Nullable; /** @@ -45,14 +42,6 @@ public interface IgniteTree { */ public void invoke(L key, Object x, InvokeClosure c) throws IgniteCheckedException; - /** - * @param keys Keys. - * @param x Implementation specific argument, {@code null} always means that we need a full detached data row. - * @param c Closure. - * @throws IgniteCheckedException If failed. - */ - public void invokeAll(List keys, Object x, InvokeAllClosure c) throws IgniteCheckedException; - /** * Returns the value to which the specified key is mapped, or {@code null} if this tree contains no mapping for the * key. @@ -141,31 +130,6 @@ interface InvokeClosure { OperationType operationType(); } - /** - * - */ - interface InvokeAllClosure { - /** - * - * @param rows Found/not found data rows. - * @throws IgniteCheckedException If failed. - */ - void call(Collection rows) throws IgniteCheckedException; - - /** - * - * @return List of final actions: operation, old row, new row. - */ - Collection> result(); - - /** - * @return Fast path flag. - * @deprecated Workaround to select B+ tree search strategy, should be removed in final implementation. - */ - @Deprecated - boolean fastpath(); - } - /** * */ From cdc63cde7ea58b21f1f69dc37b84894c5e4625b6 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 4 Apr 2019 17:24:10 +0300 Subject: [PATCH 07/24] IGNITE-11584 Update rows in batch. --- .../cache/IgniteCacheOffheapManagerImpl.java | 237 ++++++++---------- 1 file changed, 104 insertions(+), 133 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index 7c29dabd77356..a830f51512698 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -103,7 +103,7 @@ import org.apache.ignite.internal.util.lang.GridIterator; import org.apache.ignite.internal.util.lang.IgniteInClosure2X; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T3; +import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; @@ -135,7 +135,6 @@ import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.MVCC_INFO_SIZE; import static org.apache.ignite.internal.util.IgniteTree.OperationType.NOOP; import static org.apache.ignite.internal.util.IgniteTree.OperationType.PUT; -import static org.apache.ignite.internal.util.IgniteTree.OperationType.REMOVE; /** * @@ -1694,183 +1693,155 @@ private void updateAll0( try { assert cctx.shared().database().checkpointLockIsHeldByThread(); - int keysCnt = keys.size(); - - List rows = new ArrayList<>(keysCnt); + int cacheId = cctx.group().storeCacheIdInDataPage() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; - if (!sorted) { - for (CacheSearchRow row : keys) { - CacheDataRow old = dataTree.findOne(row, null, CacheDataRowAdapter.RowData.NO_KEY); + IoStatisticsHolder statHolder = grp.statisticsHolderData(); - rows.add(old); - } - } - else { - GridCursor cur = - dataTree.find(keys.get(0), keys.get(keysCnt - 1), CacheDataRowAdapter.RowData.FULL); + List oldRows = findAll(keys, sorted); - Iterator keyItr = keys.iterator(); + assert oldRows.size() == keys.size() : "Mismatch: expect=" + keys.size() + ", actual=" + oldRows.size(); - CacheSearchRow lastRow = null; - CacheSearchRow row = null; - KeyCacheObject key = null; + Iterator oldRowsIter = oldRows.iterator(); - CacheDataRow oldRow = null; - KeyCacheObject oldKey = null; + // Old to new rows mapping. + List> resMapping = new ArrayList<>(8); - while (cur.next()) { - oldRow = cur.get(); - oldKey = oldRow.key(); + for (CacheMapEntryInfo entry : entries) { + CacheDataRow oldRow = oldRowsIter.next(); - while (key == null || key.hashCode() <= oldKey.hashCode()) { - if (key != null && key.hashCode() == oldKey.hashCode()) { - while (key.hashCode() == oldKey.hashCode()) { - // todo test collision resolution - rows.add(key.equals(oldKey) ? oldRow : null); + KeyCacheObject key = entry.key(); - lastRow = null; - - if (!keyItr.hasNext()) - break; + try { + if (!entry.needUpdate(oldRow)) + continue; - lastRow = row = keyItr.next(); - key = row.key(); - } - } - else { - if (row != null) - rows.add(null); + CacheObject val = entry.value(); - lastRow = null; + if (val == null) { + dataTree.removex(new SearchRow(cacheId, key)); - if (keyItr.hasNext()) { - lastRow = row = keyItr.next(); - key = lastRow.key(); - } - } + finishRemove(cctx, key, oldRow); - if (!keyItr.hasNext()) - break; + continue; } - } - if (lastRow != null) - rows.add(key.equals(oldKey) ? oldRow : null); + CacheObjectContext coCtx = cctx.cacheObjectContext(); - for (; keyItr.hasNext(); keyItr.next()) - rows.add(null); - } - - List newRows = new ArrayList<>(8); - - int cacheId = cctx.group().storeCacheIdInDataPage() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + val.valueBytes(coCtx); + key.valueBytes(coCtx); - Iterator rowsIter = rows.iterator(); + DataRow row = makeDataRow(key, val, entry.version(), entry.expireTime(), cacheId); - List> finalOps = new ArrayList<>(keys.size()); + if (canUpdateOldRow(cctx, oldRow, row) && rowStore().updateRow(oldRow.link(), row, statHolder)) + continue; - for (CacheMapEntryInfo e : entries) { - KeyCacheObject key = e.key(); - CacheDataRow oldRow = rowsIter.next(); + resMapping.add(new T2<>(oldRow, row)); - try { - if (e.needUpdate(oldRow)) { - CacheObject val = e.value(); - - if (val != null) { - if (oldRow != null) { - CacheDataRow newRow = createRow( - cctx, - key, - e.value(), - e.version(), - e.expireTime(), - oldRow); - - finalOps.add(new T3<>(oldRow.link() == newRow.link() ? NOOP : PUT, oldRow, newRow)); - } - else { - CacheObjectContext coCtx = cctx.cacheObjectContext(); - - val.valueBytes(coCtx); - key.valueBytes(coCtx); - - if (key.partition() == -1) - key.partition(partId); - - DataRow newRow = new DataRow(key, val, e.version(), partId, e.expireTime(), cacheId); - - newRows.add(newRow); - - finalOps.add(new T3<>(PUT, oldRow, newRow)); - } - } - else { - // todo we should pass key somehow to remove old row - // todo (in particular case oldRow should not contain key) - DataRow newRow = new DataRow(key, null, null, 0, 0, cacheId); - - finalOps.add(new T3<>(oldRow != null ? REMOVE : NOOP, oldRow, newRow)); - } - } } catch (GridCacheEntryRemovedException ex) { - e.onRemove(); + entry.onRemove(); } } - if (!newRows.isEmpty()) { - // cctx.shared().database().ensureFreeSpace(cctx.dataRegion()); + if (!resMapping.isEmpty()) { + cctx.shared().database().ensureFreeSpace(cctx.dataRegion()); - rowStore().addRows(newRows, cctx.group().statisticsHolderData()); + Collection newRows = F.viewReadOnly(resMapping, IgniteBiTuple::get2); + + rowStore().addRows(newRows, statHolder); if (cacheId == CU.UNDEFINED_CACHE_ID) { // Set cacheId before write keys into tree. for (DataRow row : newRows) row.cacheId(cctx.cacheId()); } - } - for (T3 t3 : finalOps) { - IgniteTree.OperationType oper = t3.get1(); + for (T2 map : resMapping) { + CacheDataRow oldRow = map.get1(); + CacheDataRow newRow = map.get2(); + + assert newRow != null : map; + + dataTree.putx(newRow); - if (oper == IgniteTree.OperationType.PUT) - dataTree.putx(t3.get3()); - else if (oper == IgniteTree.OperationType.REMOVE) - dataTree.removex(t3.get2()); // old row + finishUpdate(cctx, newRow, oldRow); + } } + } + finally { + busyLock.leaveBusy(); + } + } - for (T3 finalOp : finalOps) { - IgniteTree.OperationType opType = finalOp.get1(); - CacheDataRow oldRow = finalOp.get2(); - CacheDataRow newRow = finalOp.get3(); + /** + * @param keys Keys to search. + * @param sorted Sorted flag. + * @return List of found (and not found) rows, each value of which corresponds to the input key. + * @throws IgniteCheckedException If failed. + */ + private List findAll(List keys, boolean sorted) throws IgniteCheckedException { + List rows = new ArrayList<>(keys.size()); - switch (opType) { - case PUT: { - assert newRow != null : finalOp; + if (!sorted) { + for (CacheSearchRow row : keys) + rows.add(dataTree.findOne(row, null, CacheDataRowAdapter.RowData.NO_KEY)); - finishUpdate(cctx, newRow, oldRow); + return rows; + } - break; - } + GridCursor cur = dataTree.find(keys.get(0), keys.get(keys.size() - 1)); + Iterator keyItr = keys.iterator(); - case REMOVE: { - finishRemove(cctx, newRow.key(), oldRow); + CacheSearchRow last = null; + CacheSearchRow row = null; + KeyCacheObject key = null; - break; + CacheDataRow oldRow = null; + KeyCacheObject oldKey = null; + + while (cur.next()) { + oldRow = cur.get(); + oldKey = oldRow.key(); + + while (key == null || key.hashCode() <= oldKey.hashCode()) { + if (key != null && key.hashCode() == oldKey.hashCode()) { + while (key.hashCode() == oldKey.hashCode()) { + // todo test collision resolution + rows.add(key.equals(oldKey) ? oldRow : null); + + last = null; + + if (!keyItr.hasNext()) + break; + + last = row = keyItr.next(); + key = row.key(); } + } + else { + if (row != null) + rows.add(null); - case NOOP: - break; + last = null; - default: - assert false : opType; + if (keyItr.hasNext()) { + last = row = keyItr.next(); + key = last.key(); + } } + + if (!keyItr.hasNext()) + break; } } - finally { - busyLock.leaveBusy(); - } + + if (last != null) + rows.add(key.equals(oldKey) ? oldRow : null); + + for (; keyItr.hasNext(); keyItr.next()) + rows.add(null); + + return rows; } /** {@inheritDoc} */ From 8a3ad85cf3d698659b8459f83a12d282981ab88e Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 4 Apr 2019 18:10:17 +0300 Subject: [PATCH 08/24] IGNITE-11584 Code cleanup. --- .../processors/cache/CacheMapEntries.java | 53 +++++++++++-------- .../processors/cache/CacheMapEntryInfo.java | 33 +++++++----- .../preloader/GridDhtPartitionDemander.java | 2 - .../tree/io/AbstractDataPageIO.java | 17 ++---- 4 files changed, 55 insertions(+), 50 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java index 873e9f4a0b9cd..49058da70f9cc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -38,7 +38,7 @@ public class CacheMapEntries { /** */ static class BatchContext { /** */ - private final GridCacheContext cctx; + private final GridCacheContext cctx; /** */ private final GridDhtLocalPartition part; @@ -50,13 +50,13 @@ static class BatchContext { private final AffinityTopologyVersion topVer; /** */ - private boolean sorted; + private final Set skipped = new HashSet<>(); /** */ - private final Set skipped = new HashSet<>(); + private boolean sorted; /** */ - BatchContext(GridCacheContext cctx, int partId, boolean preload, AffinityTopologyVersion topVer) { + BatchContext(GridCacheContext cctx, int partId, boolean preload, AffinityTopologyVersion topVer) { this.cctx = cctx; this.preload = preload; this.topVer = topVer; @@ -90,12 +90,12 @@ GridDhtLocalPartition part() { } /** */ - GridCacheContext context() { + GridCacheContext context() { return cctx; } /** */ - boolean skip(KeyCacheObject key) { + boolean skipped(KeyCacheObject key) { return skipped.contains(key); } @@ -115,19 +115,19 @@ public int initialValues( ) throws IgniteCheckedException { BatchContext ctx = new BatchContext(cctx, partId, preload, topVer); - Collection locked = lock(ctx, infos); + Collection locked = initialValuesLock(ctx, infos); try { cctx.offheap().updateAll(ctx.context(), ctx.part(), infos, ctx.sorted()); } finally { - unlock(ctx, locked); + initialValuesUnlock(ctx, locked); } return infos.size() - ctx.skipped.size(); } /** */ - private Collection lock(BatchContext ctx, Collection infos) { + private Collection initialValuesLock(BatchContext ctx, Collection infos) { List locked = new ArrayList<>(infos.size()); boolean ordered = true; @@ -191,10 +191,8 @@ private Collection lock(BatchContext ctx, Collection infos) { + /** */ + private void initialValuesUnlock(BatchContext ctx, Collection infos) { // Process deleted entries before locks release. // todo assert ctx.cctx.deferredDelete() : this; @@ -204,20 +202,31 @@ private void unlock(BatchContext ctx, Collection infos) { int size = infos.size(); try { - for (CacheMapEntryInfo e : infos) { - KeyCacheObject key = e.key(); - GridCacheMapEntry entry = e.cacheEntry(); + for (CacheMapEntryInfo info : infos) { + KeyCacheObject key = info.key(); + GridCacheMapEntry entry = info.cacheEntry(); + + assert entry != null : key; + + if (!info.needUpdate()) + continue; - if (ctx.skip(key)) + if (ctx.skipped(key)) continue; - if (entry != null && entry.deleted()) - ctx.markRemoved(entry.key()); + if (entry.deleted()) { + info.onRemove(); + + continue; + } try { - e.updateCacheEntry(); + entry.finishInitialUpdate(info.value(), info.expireTime(), info.ttl(), info.version(), ctx.topVer(), + info.drType(), null, ctx.preload()); } catch (IgniteCheckedException ex) { - ctx.markRemoved(entry.key()); + ctx.context().logger(getClass()).error("Unable to finish initial update, skip " + key, ex); + + info.onRemove(); } } } @@ -250,7 +259,7 @@ private void unlock(BatchContext ctx, Collection infos) { for (CacheMapEntryInfo info : infos) { GridCacheMapEntry entry = info.cacheEntry(); - if (entry != null && !ctx.skip(entry.key())) + if (entry != null && !ctx.skipped(entry.key())) entry.touch(); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntryInfo.java index b7a4883447375..d2aa52d3bb5ef 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntryInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntryInfo.java @@ -28,9 +28,6 @@ * */ public class CacheMapEntryInfo { - /** */ - private BatchContext ctx; - /** */ private final KeyCacheObject key; @@ -49,6 +46,9 @@ public class CacheMapEntryInfo { /** */ private final GridDrType drType; + /** */ + private BatchContext ctx; + /** */ private GridDhtCacheEntry entry; @@ -104,6 +104,18 @@ public long expireTime() { return expireTime; } + /** + * @return Expire time. + */ + public long ttl() { + return ttl; + } + + /** */ + public GridDrType drType() { + return drType; + } + /** * */ @@ -129,16 +141,6 @@ void init(BatchContext ctx, GridDhtCacheEntry entry) { this.entry = entry; } - /** - * - */ - void updateCacheEntry() throws IgniteCheckedException { - if (!update) - return; - - entry.finishInitialUpdate(val, expireTime, ttl, ver, ctx.topVer(), drType, null, ctx.preload()); - } - /** * */ @@ -170,4 +172,9 @@ boolean needUpdate(CacheDataRow row) throws GridCacheEntryRemovedException { return update0; } + + /** */ + boolean needUpdate() { + return update; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index 5b33df13fe5a7..04e845225c8ce 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -916,7 +916,6 @@ private void preloadEntriesBatch(ClusterNode from, } } - for (Map.Entry> e : cctxs.entrySet()) { GridCacheContext cctx = ctx.cacheContext(e.getKey()); @@ -1592,4 +1591,3 @@ private void sendRebalanceFinishedEvent() { } } } - diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java index d047d4714b150..96df8e3a2bb0a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java @@ -995,26 +995,19 @@ public void addRows( final int pageSize ) throws IgniteCheckedException { int maxPayloadSIze = pageSize - MIN_DATA_PAGE_OVERHEAD; - - int regularSizeFlags = SHOW_PAYLOAD_LEN | SHOW_ITEM; - int fragmentSizeFlags = regularSizeFlags | SHOW_LINK; - int dataOff = pageSize; - int cnt = 0; - int written = 0; + int cnt = 0; for (T row : rows) { int size = row.size(); - - boolean fragment = size > maxPayloadSIze; - int payloadSize = size % maxPayloadSIze; + boolean fragment = size > maxPayloadSIze; - int fullEntrySize = getPageEntrySize(payloadSize, fragment ? fragmentSizeFlags : regularSizeFlags); + int fullEntrySize = getPageEntrySize(payloadSize, fragment ? + SHOW_PAYLOAD_LEN | SHOW_ITEM | SHOW_LINK : SHOW_PAYLOAD_LEN | SHOW_ITEM); written += fullEntrySize; - dataOff -= (fullEntrySize - ITEM_SIZE); if (fragment) { @@ -1044,8 +1037,6 @@ public void addRows( setFirstEntryOffset(pageAddr, dataOff, pageSize); - // Update free space. If number of indirect items changed, then we were able to reuse an item slot. - // + (getIndirectCount(pageAddr) != indirectCnt ? ITEM_SIZE : 0) setRealFreeSpace(pageAddr, getRealFreeSpace(pageAddr) - written, pageSize); } From 86e83d416bca05b1b23f16de4664f1993e0a661a Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Tue, 9 Apr 2019 11:34:51 +0300 Subject: [PATCH 09/24] IGNITE-11584 Redactoring. --- .../processors/cache/CacheMapEntries.java | 164 ++++++++++++---- .../processors/cache/CacheMapEntryInfo.java | 180 ------------------ .../cache/IgniteCacheOffheapManager.java | 13 +- .../cache/IgniteCacheOffheapManagerImpl.java | 77 ++++---- .../preloader/GridDhtPartitionDemander.java | 18 +- .../persistence/GridCacheOffheapManager.java | 9 +- 6 files changed, 186 insertions(+), 275 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntryInfo.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java index 49058da70f9cc..dcc60bd2f3abd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -19,8 +19,8 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -28,6 +28,10 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.dr.GridDrType; +import org.apache.ignite.lang.IgniteBiPredicate; import static org.apache.ignite.internal.processors.cache.GridCacheMapEntry.ATOMIC_VER_COMPARATOR; @@ -35,8 +39,63 @@ * Batch of cache map entries. */ public class CacheMapEntries { + /** + * + */ + private static class GridCacheEntryInfoEx extends GridCacheEntryInfo { + /** */ + private final GridCacheEntryInfo delegate; + + /** */ + private GridDhtCacheEntry cacheEntry; + + /** */ + private boolean update; + + /** */ + private GridCacheEntryInfoEx(GridCacheEntryInfo info) { + delegate = info; + } + + /** + * @return Key. + */ + @Override public KeyCacheObject key() { + return delegate.key(); + } + + /** + * @return Entry value. + */ + @Override public CacheObject value() { + return delegate.value(); + } + + /** + * @return Expire time. + */ + @Override public long expireTime() { + return delegate.expireTime(); + } + + /** + * @return Time to live. + */ + @Override public long ttl() { + return delegate.ttl(); + } + + /** + * @return Time to live. + */ + @Override public GridCacheVersion version() { + return delegate.version(); + } + } + + /** */ - static class BatchContext { + private static class BatchContext { /** */ private final GridCacheContext cctx; @@ -107,53 +166,85 @@ void sorted(boolean sorted) { /** */ public int initialValues( - List infos, + List infos, AffinityTopologyVersion topVer, GridCacheContext cctx, int partId, - boolean preload + boolean preload, + GridDrType drType ) throws IgniteCheckedException { BatchContext ctx = new BatchContext(cctx, partId, preload, topVer); - Collection locked = initialValuesLock(ctx, infos); + Collection locked = initialValuesLock(ctx, infos); try { - cctx.offheap().updateAll(ctx.context(), ctx.part(), infos, ctx.sorted()); + IgniteBiPredicate pred = new IgniteBiPredicate() { + @Override public boolean apply(CacheDataRow row, GridCacheEntryInfo info) { + try { + GridCacheVersion currVer = row != null ? row.version() : + ((GridCacheEntryInfoEx)info).cacheEntry.version(); + + GridCacheContext cctx = ctx.context(); + + boolean isStartVer = cctx.versions().isStartVersion(currVer); + + boolean update0; + + if (cctx.group().persistenceEnabled()) { + if (!isStartVer) { + if (cctx.atomic()) + update0 = ATOMIC_VER_COMPARATOR.compare(currVer, info.version()) < 0; + else + update0 = currVer.compareTo(info.version()) < 0; + } + else + update0 = true; + } + else + update0 = (isStartVer && row == null); + + update0 |= (!ctx.preload() && ((GridCacheEntryInfoEx)info).cacheEntry.deletedUnlocked()); + + ((GridCacheEntryInfoEx)info).update = update0; + + return update0; + } + catch (GridCacheEntryRemovedException e) { + ctx.markRemoved(info.key()); + + return false; + } + } + }; + + cctx.offheap().updateAll(cctx, ctx.part(), locked, pred); } finally { - initialValuesUnlock(ctx, locked); + initialValuesUnlock(ctx, locked, drType); } return infos.size() - ctx.skipped.size(); } /** */ - private Collection initialValuesLock(BatchContext ctx, Collection infos) { + private Collection initialValuesLock(BatchContext ctx, Collection infos) { List locked = new ArrayList<>(infos.size()); - boolean ordered = true; - while (true) { - Map uniqueEntries = new HashMap<>(); - - KeyCacheObject lastKey = null; + Map uniqueEntries = new LinkedHashMap<>(); - for (CacheMapEntryInfo e : infos) { + for (GridCacheEntryInfo e : infos) { KeyCacheObject key = e.key(); - CacheMapEntryInfo old = uniqueEntries.put(key, e); + GridCacheEntryInfoEx entryEx = new GridCacheEntryInfoEx(e); + GridCacheEntryInfoEx old = uniqueEntries.put(key, entryEx); assert old == null || ATOMIC_VER_COMPARATOR.compare(old.version(), e.version()) < 0 : "Version order mismatch: prev=" + old.version() + ", current=" + e.version(); - if (ordered && lastKey != null && lastKey.hashCode() >= key.hashCode()) - ordered = false; - GridDhtCacheEntry entry = (GridDhtCacheEntry)ctx.cctx.cache().entryEx(key, ctx.topVer()); locked.add(entry); - e.init(ctx, entry); - - lastKey = key; + entryEx.cacheEntry = entry; } boolean retry = false; @@ -183,16 +274,13 @@ private Collection initialValuesLock(BatchContext ctx, Collec } } - if (!retry) { - ctx.sorted(ordered); - + if (!retry) return uniqueEntries.values(); - } } } /** */ - private void initialValuesUnlock(BatchContext ctx, Collection infos) { + private void initialValuesUnlock(BatchContext ctx, Collection infos, GridDrType drType) { // Process deleted entries before locks release. // todo assert ctx.cctx.deferredDelete() : this; @@ -202,31 +290,31 @@ private void initialValuesUnlock(BatchContext ctx, Collection int size = infos.size(); try { - for (CacheMapEntryInfo info : infos) { + for (GridCacheEntryInfoEx info : infos) { KeyCacheObject key = info.key(); - GridCacheMapEntry entry = info.cacheEntry(); + GridCacheMapEntry entry = info.cacheEntry; assert entry != null : key; - if (!info.needUpdate()) + if (!info.update) continue; if (ctx.skipped(key)) continue; if (entry.deleted()) { - info.onRemove(); + ctx.markRemoved(key); continue; } try { entry.finishInitialUpdate(info.value(), info.expireTime(), info.ttl(), info.version(), ctx.topVer(), - info.drType(), null, ctx.preload()); + drType, null, ctx.preload()); } catch (IgniteCheckedException ex) { ctx.context().logger(getClass()).error("Unable to finish initial update, skip " + key, ex); - info.onRemove(); + ctx.markRemoved(key); } } } @@ -234,8 +322,8 @@ private void initialValuesUnlock(BatchContext ctx, Collection // At least RuntimeException can be thrown by the code above when GridCacheContext is cleaned and there is // an attempt to use cleaned resources. // That's why releasing locks in the finally block.. - for (CacheMapEntryInfo info : infos) { - GridCacheMapEntry entry = info.cacheEntry(); + for (GridCacheEntryInfoEx info : infos) { + GridCacheMapEntry entry = info.cacheEntry; if (entry != null) entry.unlockEntry(); @@ -243,8 +331,8 @@ private void initialValuesUnlock(BatchContext ctx, Collection } // Try evict partitions. - for (CacheMapEntryInfo info : infos) { - GridDhtCacheEntry entry = info.cacheEntry(); + for (GridCacheEntryInfoEx info : infos) { + GridDhtCacheEntry entry = info.cacheEntry; if (entry != null) entry.onUnlock(); @@ -256,8 +344,8 @@ private void initialValuesUnlock(BatchContext ctx, Collection // Must touch all entries since update may have deleted entries. // Eviction manager will remove empty entries. - for (CacheMapEntryInfo info : infos) { - GridCacheMapEntry entry = info.cacheEntry(); + for (GridCacheEntryInfoEx info : infos) { + GridCacheMapEntry entry = info.cacheEntry; if (entry != null && !ctx.skipped(entry.key())) entry.touch(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntryInfo.java deleted file mode 100644 index d2aa52d3bb5ef..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntryInfo.java +++ /dev/null @@ -1,180 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.cache; - -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.cache.CacheMapEntries.BatchContext; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; -import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; -import org.apache.ignite.internal.processors.dr.GridDrType; - -/** - * - */ -public class CacheMapEntryInfo { - /** */ - private final KeyCacheObject key; - - /** */ - private final CacheObject val; - - /** */ - private final long expireTime; - - /** */ - private final long ttl; - - /** */ - private final GridCacheVersion ver; - - /** */ - private final GridDrType drType; - - /** */ - private BatchContext ctx; - - /** */ - private GridDhtCacheEntry entry; - - /** */ - private boolean update; - - /** - * - */ - public CacheMapEntryInfo( - KeyCacheObject key, - CacheObject val, - long expireTime, - long ttl, - GridCacheVersion ver, - GridDrType drType - ) { - assert key != null; - - this.key = key; - this.val = val; - this.ver = ver; - this.ttl = ttl; - this.drType = drType; - this.expireTime = expireTime; - } - - /** - * - */ - public KeyCacheObject key() { - return key; - } - - /** - * @return Version. - */ - public GridCacheVersion version() { - return ver; - } - - /** - * @return Value. - */ - public CacheObject value() { - return val; - } - - /** - * @return Expire time. - */ - public long expireTime() { - return expireTime; - } - - /** - * @return Expire time. - */ - public long ttl() { - return ttl; - } - - /** */ - public GridDrType drType() { - return drType; - } - - /** - * - */ - public void onRemove() { - ctx.markRemoved(key); - } - - /** - * @return Cache entry. - */ - GridDhtCacheEntry cacheEntry() { - return entry; - } - - /** - * - */ - void init(BatchContext ctx, GridDhtCacheEntry entry) { - assert ctx != null; - assert entry != null; - - this.ctx = ctx; - this.entry = entry; - } - - /** - * - */ - boolean needUpdate(CacheDataRow row) throws GridCacheEntryRemovedException { - GridCacheVersion currVer = row != null ? row.version() : entry.version(); - - GridCacheContext cctx = ctx.context(); - - boolean isStartVer = cctx.versions().isStartVersion(currVer); - - boolean update0; - - if (cctx.group().persistenceEnabled()) { - if (!isStartVer) { - if (cctx.atomic()) - update0 = GridCacheMapEntry.ATOMIC_VER_COMPARATOR.compare(currVer, version()) < 0; - else - update0 = currVer.compareTo(version()) < 0; - } - else - update0 = true; - } - else - update0 = (isStartVer && row == null); - - update0 |= (!ctx.preload() && entry.deletedUnlocked()); - - update = update0; - - return update0; - } - - /** */ - boolean needUpdate() { - return update; - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java index dd33d91aa120f..8024c4e303485 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java @@ -47,6 +47,7 @@ import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.internal.util.lang.GridIterator; import org.apache.ignite.internal.util.lang.IgniteInClosure2X; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.Nullable; @@ -193,14 +194,14 @@ public void invoke(GridCacheContext cctx, KeyCacheObject key, GridDhtLocalPartit * @param cctx Cache context. * @param part Partition. * @param entries Entries. - * @param sorted Sorted flag. + * @param pred Entry update predicate. * @throws IgniteCheckedException If failed. */ public void updateAll( GridCacheContext cctx, GridDhtLocalPartition part, - Collection entries, - boolean sorted + Collection entries, + IgniteBiPredicate pred ) throws IgniteCheckedException; /** @@ -737,13 +738,13 @@ void update( /** * @param cctx Cache context. * @param entries Entries. - * @param sorted Sorted flag. + * @param pred Entry update predicate. * @throws IgniteCheckedException If failed. */ public void updateAll( GridCacheContext cctx, - Collection entries, - boolean sorted + Collection entries, + IgniteBiPredicate pred ) throws IgniteCheckedException; /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index a830f51512698..2c9462bf345c4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -107,6 +107,7 @@ import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteInClosure; @@ -439,10 +440,10 @@ private Iterator cacheData(boolean primary, boolean backup, Affi @Override public void updateAll( GridCacheContext cctx, GridDhtLocalPartition part, - Collection entries, - boolean sorted + Collection entries, + IgniteBiPredicate pred ) throws IgniteCheckedException { - dataStore(part).updateAll(cctx, entries, sorted); + dataStore(part).updateAll(cctx, entries, pred); } /** {@inheritDoc} */ @@ -1622,17 +1623,25 @@ private boolean canUpdateOldRow(GridCacheContext cctx, @Nullable CacheDataRow ol /** {@inheritDoc} */ @Override public void updateAll( GridCacheContext cctx, - Collection entries, - boolean sorted + Collection entries, + IgniteBiPredicate pred ) throws IgniteCheckedException { int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; List searchRows = new ArrayList<>(entries.size()); - for (CacheMapEntryInfo entry : entries) + boolean ordered = true; + + KeyCacheObject last = null; + + for (GridCacheEntryInfo entry : entries) { searchRows.add(new SearchRow(cacheId, entry.key())); - updateAll0(cctx, searchRows, sorted, entries); + if (ordered && last != null && last.hashCode() >= entry.key().hashCode()) + ordered = false; + } + + updateAll0(cctx, searchRows, entries, pred, ordered); } /** @@ -1679,13 +1688,15 @@ private void invoke0(GridCacheContext cctx, CacheSearchRow row, OffheapInvokeClo * @param keys Search rows. * @param sorted Sorted flag. * @param entries Entries. + * @param pred Entry update predicate. * @throws IgniteCheckedException If failed. */ private void updateAll0( GridCacheContext cctx, List keys, - boolean sorted, - Collection entries + Collection entries, + IgniteBiPredicate pred, + boolean sorted ) throws IgniteCheckedException { if (!busyLock.enterBusy()) throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); @@ -1706,41 +1717,35 @@ private void updateAll0( // Old to new rows mapping. List> resMapping = new ArrayList<>(8); - for (CacheMapEntryInfo entry : entries) { + for (GridCacheEntryInfo entry : entries) { CacheDataRow oldRow = oldRowsIter.next(); KeyCacheObject key = entry.key(); - try { - if (!entry.needUpdate(oldRow)) - continue; + if (!pred.apply(oldRow, entry)) + continue; - CacheObject val = entry.value(); + CacheObject val = entry.value(); - if (val == null) { - dataTree.removex(new SearchRow(cacheId, key)); + if (val == null) { + dataTree.removex(new SearchRow(cacheId, key)); - finishRemove(cctx, key, oldRow); + finishRemove(cctx, key, oldRow); - continue; - } - - CacheObjectContext coCtx = cctx.cacheObjectContext(); + continue; + } - val.valueBytes(coCtx); - key.valueBytes(coCtx); + CacheObjectContext coCtx = cctx.cacheObjectContext(); - DataRow row = makeDataRow(key, val, entry.version(), entry.expireTime(), cacheId); + val.valueBytes(coCtx); + key.valueBytes(coCtx); - if (canUpdateOldRow(cctx, oldRow, row) && rowStore().updateRow(oldRow.link(), row, statHolder)) - continue; + DataRow row = makeDataRow(key, val, entry.version(), entry.expireTime(), cacheId); - resMapping.add(new T2<>(oldRow, row)); + if (canUpdateOldRow(cctx, oldRow, row) && rowStore().updateRow(oldRow.link(), row, statHolder)) + continue; - } - catch (GridCacheEntryRemovedException ex) { - entry.onRemove(); - } + resMapping.add(new T2<>(oldRow, row)); } if (!resMapping.isEmpty()) { @@ -1751,16 +1756,16 @@ private void updateAll0( rowStore().addRows(newRows, statHolder); if (cacheId == CU.UNDEFINED_CACHE_ID) { - // Set cacheId before write keys into tree. + // Set cacheId before store keys into tree. for (DataRow row : newRows) row.cacheId(cctx.cacheId()); } - for (T2 map : resMapping) { - CacheDataRow oldRow = map.get1(); - CacheDataRow newRow = map.get2(); + for (T2 mapping : resMapping) { + CacheDataRow oldRow = mapping.get1(); + CacheDataRow newRow = mapping.get2(); - assert newRow != null : map; + assert newRow != null : mapping; dataTree.putx(newRow); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index 04e845225c8ce..06eabc7f4e59e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -47,13 +47,11 @@ import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheMapEntries; -import org.apache.ignite.internal.processors.cache.CacheMapEntryInfo; import org.apache.ignite.internal.processors.cache.CacheMetricsImpl; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; -import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; import org.apache.ignite.internal.processors.cache.GridCacheMvccEntryInfo; import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; @@ -874,7 +872,7 @@ public void handleSupplyMessage( /** * @param from Node which sent entry. * @param p Partition id. - * @param entries Preloaded entries. + * @param infos Preloaded entries. * @param topVer Topology version. * @throws IgniteCheckedException If failed. */ @@ -889,7 +887,7 @@ private void preloadEntriesBatch(ClusterNode from, grp.listenerLock().readLock().lock(); try { - Map> cctxs = new HashMap<>(); + Map> cctxs = new HashMap<>(); // Map by context. for (GridCacheEntryInfo e : infos) { @@ -902,13 +900,10 @@ private void preloadEntriesBatch(ClusterNode from, if (cctx0.isNear()) cctx0 = cctx0.dhtCache().context(); - final GridCacheContext cctx = cctx0; - if (log.isTraceEnabled()) log.trace("Rebalancing key [key=" + e.key() + ", part=" + p + ", node=" + from.id() + ']'); - cctxs.computeIfAbsent(cctx.cacheId(), v -> new ArrayList<>()).add( - new CacheMapEntryInfo(e.key(), e.value(), e.expireTime(), e.ttl(), e.version(), DR_PRELOAD)); + cctxs.computeIfAbsent(cctx0.cacheId(), v -> new ArrayList<>(8)).add(e); } catch (GridDhtInvalidPartitionException ignored) { if (log.isDebugEnabled()) @@ -916,16 +911,17 @@ private void preloadEntriesBatch(ClusterNode from, } } - for (Map.Entry> e : cctxs.entrySet()) { + for (Map.Entry> e : cctxs.entrySet()) { GridCacheContext cctx = ctx.cacheContext(e.getKey()); CacheMapEntries entries = new CacheMapEntries(); - int initialized = entries.initialValues(e.getValue(), topVer, cctx, p, true); + entries.initialValues(e.getValue(), topVer, cctx, p, true, DR_PRELOAD); + //TODO: IGNITE-11330: Update metrics for touched cache only. for (GridCacheContext cctx0 : grp.caches()) { if (cctx0.statisticsEnabled()) - cctx0.cache().metrics0().onRebalanceKeysReceived(initialized); + cctx0.cache().metrics0().onRebalanceKeysReceived(e.getValue().size()); } } } finally { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java index 714690b703400..3d2cd4fa7c415 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java @@ -53,10 +53,10 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; import org.apache.ignite.internal.processors.cache.CacheGroupContext; -import org.apache.ignite.internal.processors.cache.CacheMapEntryInfo; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; +import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; import org.apache.ignite.internal.processors.cache.GridCacheMvccEntryInfo; import org.apache.ignite.internal.processors.cache.GridCacheTtlManager; import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl; @@ -98,6 +98,7 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.Nullable; @@ -2135,14 +2136,14 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void updateAll( GridCacheContext cctx, - Collection entries, - boolean sorted + Collection entries, + IgniteBiPredicate pred ) throws IgniteCheckedException { assert ctx.database().checkpointLockIsHeldByThread(); CacheDataStore delegate = init0(false); - delegate.updateAll(cctx, entries, sorted); + delegate.updateAll(cctx, entries, pred); } /** {@inheritDoc} */ From facd60d994d85b755e0fb1a75383738572bac9d2 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Tue, 9 Apr 2019 16:11:40 +0300 Subject: [PATCH 10/24] IGNITE-11584 Collision resolution fix. --- .../cache/IgniteCacheOffheapManagerImpl.java | 59 ++++++++----------- 1 file changed, 24 insertions(+), 35 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index 2c9462bf345c4..9424c7c78fc3c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -1785,21 +1785,20 @@ private void updateAll0( * @throws IgniteCheckedException If failed. */ private List findAll(List keys, boolean sorted) throws IgniteCheckedException { - List rows = new ArrayList<>(keys.size()); + List res = new ArrayList<>(keys.size()); if (!sorted) { for (CacheSearchRow row : keys) - rows.add(dataTree.findOne(row, null, CacheDataRowAdapter.RowData.NO_KEY)); + res.add(dataTree.findOne(row, null, CacheDataRowAdapter.RowData.NO_KEY)); - return rows; + return res; } GridCursor cur = dataTree.find(keys.get(0), keys.get(keys.size() - 1)); - Iterator keyItr = keys.iterator(); + Iterator itr = keys.iterator(); - CacheSearchRow last = null; - CacheSearchRow row = null; - KeyCacheObject key = null; + CacheSearchRow newRow = null; + KeyCacheObject newKey = null; CacheDataRow oldRow = null; KeyCacheObject oldKey = null; @@ -1808,45 +1807,35 @@ private List findAll(List keys, boolean sorted) th oldRow = cur.get(); oldKey = oldRow.key(); - while (key == null || key.hashCode() <= oldKey.hashCode()) { - if (key != null && key.hashCode() == oldKey.hashCode()) { - while (key.hashCode() == oldKey.hashCode()) { - // todo test collision resolution - rows.add(key.equals(oldKey) ? oldRow : null); - - last = null; + while (itr.hasNext() && (newKey == null || newKey.hashCode() <= oldKey.hashCode())) { + if (newKey != null) { + boolean keyFound = false; - if (!keyItr.hasNext()) - break; + if (newKey.hashCode() == oldKey.hashCode()) { + while (!(keyFound = newKey.equals(oldKey)) && cur.next()) { + oldRow = cur.get(); + oldKey = oldRow.key(); - last = row = keyItr.next(); - key = row.key(); + if (newKey.hashCode() != oldKey.hashCode()) + break; + } } - } - else { - if (row != null) - rows.add(null); - last = null; - - if (keyItr.hasNext()) { - last = row = keyItr.next(); - key = last.key(); - } + res.add(keyFound ? oldRow : null); } - if (!keyItr.hasNext()) - break; + newRow = itr.next(); + newKey = newRow.key(); } } - if (last != null) - rows.add(key.equals(oldKey) ? oldRow : null); + if (newRow != null) + res.add(newKey.equals(oldKey) ? oldRow : null); - for (; keyItr.hasNext(); keyItr.next()) - rows.add(null); + for (; itr.hasNext(); itr.next()) + res.add(null); - return rows; + return res; } /** {@inheritDoc} */ From 233765d6ae2f0222f0ddd959f4c7e0d95105b7f1 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Tue, 9 Apr 2019 18:24:19 +0300 Subject: [PATCH 11/24] IGNITE-11584 Code cleanup (wip). --- .../ignite/internal/processors/cache/CacheMapEntries.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java index dcc60bd2f3abd..57e750d3e92ed 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -43,6 +43,9 @@ public class CacheMapEntries { * */ private static class GridCacheEntryInfoEx extends GridCacheEntryInfo { + /** */ + private static final long serialVersionUID = 0L; + /** */ private final GridCacheEntryInfo delegate; From e293e3b2a27e1771b7e1a9869d2b140faa1c7379 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Tue, 9 Apr 2019 19:40:09 +0300 Subject: [PATCH 12/24] IGNITE-11584 Code cleanup. --- .../processors/cache/CacheMapEntries.java | 16 +++++--- .../cache/IgniteCacheOffheapManagerImpl.java | 40 ++++++++----------- .../preloader/GridDhtPartitionDemander.java | 2 +- 3 files changed, 27 insertions(+), 31 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java index 57e750d3e92ed..3a5858e5ca7af 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -118,12 +118,11 @@ private static class BatchContext { private boolean sorted; /** */ - BatchContext(GridCacheContext cctx, int partId, boolean preload, AffinityTopologyVersion topVer) { + BatchContext(GridCacheContext cctx, GridDhtLocalPartition part, boolean preload, AffinityTopologyVersion topVer) { this.cctx = cctx; this.preload = preload; this.topVer = topVer; - - part = cctx.topology().localPartition(partId, topVer, true, true); + this.part = part; } /** */ @@ -176,12 +175,15 @@ public int initialValues( boolean preload, GridDrType drType ) throws IgniteCheckedException { - BatchContext ctx = new BatchContext(cctx, partId, preload, topVer); + GridDhtLocalPartition part = cctx.topology().localPartition(partId, topVer, true, true); + + BatchContext ctx = new BatchContext(cctx, part, preload, topVer); Collection locked = initialValuesLock(ctx, infos); try { - IgniteBiPredicate pred = new IgniteBiPredicate() { + IgniteBiPredicate pred = + new IgniteBiPredicate() { @Override public boolean apply(CacheDataRow row, GridCacheEntryInfo info) { try { GridCacheVersion currVer = row != null ? row.version() : @@ -220,7 +222,7 @@ public int initialValues( } }; - cctx.offheap().updateAll(cctx, ctx.part(), locked, pred); + cctx.offheap().updateAll(cctx, part, locked, pred); } finally { initialValuesUnlock(ctx, locked, drType); } @@ -237,7 +239,9 @@ private Collection initialValuesLock(BatchContext ctx, Col for (GridCacheEntryInfo e : infos) { KeyCacheObject key = e.key(); + GridCacheEntryInfoEx entryEx = new GridCacheEntryInfoEx(e); + GridCacheEntryInfoEx old = uniqueEntries.put(key, entryEx); assert old == null || ATOMIC_VER_COMPARATOR.compare(old.version(), e.version()) < 0 : diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index 9424c7c78fc3c..189fedc24d040 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -1785,6 +1785,8 @@ private void updateAll0( * @throws IgniteCheckedException If failed. */ private List findAll(List keys, boolean sorted) throws IgniteCheckedException { + assert keys != null && !keys.isEmpty(); + List res = new ArrayList<>(keys.size()); if (!sorted) { @@ -1797,40 +1799,30 @@ private List findAll(List keys, boolean sorted) th GridCursor cur = dataTree.find(keys.get(0), keys.get(keys.size() - 1)); Iterator itr = keys.iterator(); - CacheSearchRow newRow = null; - KeyCacheObject newKey = null; + CacheDataRow foundRow = null; + KeyCacheObject foundKey = null; - CacheDataRow oldRow = null; - KeyCacheObject oldKey = null; + KeyCacheObject key = itr.next().key(); while (cur.next()) { - oldRow = cur.get(); - oldKey = oldRow.key(); - - while (itr.hasNext() && (newKey == null || newKey.hashCode() <= oldKey.hashCode())) { - if (newKey != null) { - boolean keyFound = false; - - if (newKey.hashCode() == oldKey.hashCode()) { - while (!(keyFound = newKey.equals(oldKey)) && cur.next()) { - oldRow = cur.get(); - oldKey = oldRow.key(); + foundRow = cur.get(); + foundKey = foundRow.key(); - if (newKey.hashCode() != oldKey.hashCode()) - break; - } - } + while (itr.hasNext() && key.hashCode() <= foundKey.hashCode()) { + boolean keyFound = false; - res.add(keyFound ? oldRow : null); + while (key.hashCode() == foundKey.hashCode() && !(keyFound = key.equals(foundKey)) && cur.next()) { + foundRow = cur.get(); + foundKey = foundRow.key(); } - newRow = itr.next(); - newKey = newRow.key(); + res.add(keyFound ? foundRow : null); + + key = itr.next().key(); } } - if (newRow != null) - res.add(newKey.equals(oldKey) ? oldRow : null); + res.add(key.equals(foundKey) ? foundRow : null); for (; itr.hasNext(); itr.next()) res.add(null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index 06eabc7f4e59e..d57cbdcf7b05e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -889,7 +889,7 @@ private void preloadEntriesBatch(ClusterNode from, try { Map> cctxs = new HashMap<>(); - // Map by context. + // Map by cache id. for (GridCacheEntryInfo e : infos) { try { GridCacheContext cctx0 = grp.sharedGroup() ? ctx.cacheContext(e.cacheId()) : grp.singleCacheContext(); From 58989af9c4cc1017485321ae073aa74298c6ce9c Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Wed, 10 Apr 2019 11:28:52 +0300 Subject: [PATCH 13/24] IGNITE-11584 Removed batch context. --- .../processors/cache/CacheMapEntries.java | 123 +++++------------- 1 file changed, 29 insertions(+), 94 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java index 3a5858e5ca7af..8cfaeccab80bf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -96,76 +96,6 @@ private GridCacheEntryInfoEx(GridCacheEntryInfo info) { } } - - /** */ - private static class BatchContext { - /** */ - private final GridCacheContext cctx; - - /** */ - private final GridDhtLocalPartition part; - - /** */ - private final boolean preload; - - /** */ - private final AffinityTopologyVersion topVer; - - /** */ - private final Set skipped = new HashSet<>(); - - /** */ - private boolean sorted; - - /** */ - BatchContext(GridCacheContext cctx, GridDhtLocalPartition part, boolean preload, AffinityTopologyVersion topVer) { - this.cctx = cctx; - this.preload = preload; - this.topVer = topVer; - this.part = part; - } - - /** */ - void markRemoved(KeyCacheObject key) { - skipped.add(key); - } - - /** */ - boolean preload() { - return preload; - } - - /** */ - boolean sorted() { - return sorted; - } - - /** */ - AffinityTopologyVersion topVer() { - return topVer; - } - - /** */ - GridDhtLocalPartition part() { - return part; - } - - /** */ - GridCacheContext context() { - return cctx; - } - - /** */ - boolean skipped(KeyCacheObject key) { - return skipped.contains(key); - } - - /** */ - void sorted(boolean sorted) { - this.sorted = sorted; - } - } - /** */ public int initialValues( List infos, @@ -177,9 +107,9 @@ public int initialValues( ) throws IgniteCheckedException { GridDhtLocalPartition part = cctx.topology().localPartition(partId, topVer, true, true); - BatchContext ctx = new BatchContext(cctx, part, preload, topVer); + Set skippedKeys = new HashSet<>(); - Collection locked = initialValuesLock(ctx, infos); + Collection locked = initialValuesLock(cctx, topVer, infos); try { IgniteBiPredicate pred = @@ -189,8 +119,6 @@ public int initialValues( GridCacheVersion currVer = row != null ? row.version() : ((GridCacheEntryInfoEx)info).cacheEntry.version(); - GridCacheContext cctx = ctx.context(); - boolean isStartVer = cctx.versions().isStartVersion(currVer); boolean update0; @@ -208,14 +136,14 @@ public int initialValues( else update0 = (isStartVer && row == null); - update0 |= (!ctx.preload() && ((GridCacheEntryInfoEx)info).cacheEntry.deletedUnlocked()); + update0 |= (!preload && ((GridCacheEntryInfoEx)info).cacheEntry.deletedUnlocked()); ((GridCacheEntryInfoEx)info).update = update0; return update0; } catch (GridCacheEntryRemovedException e) { - ctx.markRemoved(info.key()); + skippedKeys.add(info.key()); return false; } @@ -224,14 +152,18 @@ public int initialValues( cctx.offheap().updateAll(cctx, part, locked, pred); } finally { - initialValuesUnlock(ctx, locked, drType); + initialValuesUnlock(cctx, topVer, preload, drType, locked, skippedKeys); } - return infos.size() - ctx.skipped.size(); + return infos.size() - skippedKeys.size(); } /** */ - private Collection initialValuesLock(BatchContext ctx, Collection infos) { + private Collection initialValuesLock( + GridCacheContext cctx, + AffinityTopologyVersion topVer, + Collection infos + ) { List locked = new ArrayList<>(infos.size()); while (true) { @@ -247,7 +179,7 @@ private Collection initialValuesLock(BatchContext ctx, Col assert old == null || ATOMIC_VER_COMPARATOR.compare(old.version(), e.version()) < 0 : "Version order mismatch: prev=" + old.version() + ", current=" + e.version(); - GridDhtCacheEntry entry = (GridDhtCacheEntry)ctx.cctx.cache().entryEx(key, ctx.topVer()); + GridDhtCacheEntry entry = (GridDhtCacheEntry)cctx.cache().entryEx(key, topVer); locked.add(entry); @@ -287,14 +219,17 @@ private Collection initialValuesLock(BatchContext ctx, Col } /** */ - private void initialValuesUnlock(BatchContext ctx, Collection infos, GridDrType drType) { + private void initialValuesUnlock( + GridCacheContext cctx, + AffinityTopologyVersion topVer, + boolean preload, + GridDrType drType, + Collection infos, + Set skippedKeys + ) { // Process deleted entries before locks release. // todo - assert ctx.cctx.deferredDelete() : this; - - // Entries to skip eviction manager notification for. - // Enqueue entries while holding locks. - int size = infos.size(); + assert cctx.deferredDelete() : this; try { for (GridCacheEntryInfoEx info : infos) { @@ -306,22 +241,22 @@ private void initialValuesUnlock(BatchContext ctx, Collection Date: Wed, 10 Apr 2019 13:18:17 +0300 Subject: [PATCH 14/24] IGNITE-11584 Record preload events. --- .../processors/cache/CacheMapEntries.java | 44 ++++++------------- .../preloader/GridDhtPartitionDemander.java | 36 +++++++++++---- 2 files changed, 42 insertions(+), 38 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java index 8cfaeccab80bf..f0c7de5569b2f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -26,11 +26,12 @@ import java.util.Set; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.dr.GridDrType; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.lang.IgniteBiPredicate; import static org.apache.ignite.internal.processors.cache.GridCacheMapEntry.ATOMIC_VER_COMPARATOR; @@ -50,7 +51,7 @@ private static class GridCacheEntryInfoEx extends GridCacheEntryInfo { private final GridCacheEntryInfo delegate; /** */ - private GridDhtCacheEntry cacheEntry; + private GridCacheMapEntry cacheEntry; /** */ private boolean update; @@ -97,7 +98,7 @@ private GridCacheEntryInfoEx(GridCacheEntryInfo info) { } /** */ - public int initialValues( + public Collection> initialValues( List infos, AffinityTopologyVersion topVer, GridCacheContext cctx, @@ -107,9 +108,9 @@ public int initialValues( ) throws IgniteCheckedException { GridDhtLocalPartition part = cctx.topology().localPartition(partId, topVer, true, true); - Set skippedKeys = new HashSet<>(); + Set skipped = new HashSet<>(); - Collection locked = initialValuesLock(cctx, topVer, infos); + Collection locked = lockEntries(cctx, topVer, infos); try { IgniteBiPredicate pred = @@ -143,7 +144,7 @@ public int initialValues( return update0; } catch (GridCacheEntryRemovedException e) { - skippedKeys.add(info.key()); + skipped.add(info.key()); return false; } @@ -152,19 +153,19 @@ public int initialValues( cctx.offheap().updateAll(cctx, part, locked, pred); } finally { - initialValuesUnlock(cctx, topVer, preload, drType, locked, skippedKeys); + unlockEntries(cctx, topVer, preload, drType, locked, skipped); } - return infos.size() - skippedKeys.size(); + return F.viewReadOnly(locked, v -> new T2<>(v, v.cacheEntry), v -> !skipped.contains(v.key())); } /** */ - private Collection initialValuesLock( + private Collection lockEntries( GridCacheContext cctx, AffinityTopologyVersion topVer, Collection infos ) { - List locked = new ArrayList<>(infos.size()); + List locked = new ArrayList<>(infos.size()); while (true) { Map uniqueEntries = new LinkedHashMap<>(); @@ -179,7 +180,7 @@ private Collection initialValuesLock( assert old == null || ATOMIC_VER_COMPARATOR.compare(old.version(), e.version()) < 0 : "Version order mismatch: prev=" + old.version() + ", current=" + e.version(); - GridDhtCacheEntry entry = (GridDhtCacheEntry)cctx.cache().entryEx(key, topVer); + GridCacheMapEntry entry = (GridCacheMapEntry)cctx.cache().entryEx(key, topVer); locked.add(entry); @@ -219,7 +220,7 @@ private Collection initialValuesLock( } /** */ - private void initialValuesUnlock( + private void unlockEntries( GridCacheContext cctx, AffinityTopologyVersion topVer, boolean preload, @@ -227,10 +228,6 @@ private void initialValuesUnlock( Collection infos, Set skippedKeys ) { - // Process deleted entries before locks release. - // todo - assert cctx.deferredDelete() : this; - try { for (GridCacheEntryInfoEx info : infos) { KeyCacheObject key = info.key(); @@ -274,23 +271,10 @@ private void initialValuesUnlock( // Try evict partitions. for (GridCacheEntryInfoEx info : infos) { - GridDhtCacheEntry entry = info.cacheEntry; + GridCacheMapEntry entry = info.cacheEntry; if (entry != null) entry.onUnlock(); } - - if (skippedKeys.size() == infos.size()) - // Optimization. - return; - - // Must touch all entries since update may have deleted entries. - // Eviction manager will remove empty entries. - for (GridCacheEntryInfoEx info : infos) { - GridCacheMapEntry entry = info.cacheEntry; - - if (entry != null && !skippedKeys.contains(entry.key())) - entry.touch(); - } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index d57cbdcf7b05e..619bbdcff4923 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -52,6 +52,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; +import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; import org.apache.ignite.internal.processors.cache.GridCacheMvccEntryInfo; import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; @@ -66,10 +67,12 @@ import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.lang.IgniteInClosure2X; import org.apache.ignite.internal.util.lang.IgniteInClosureX; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; @@ -911,17 +914,34 @@ private void preloadEntriesBatch(ClusterNode from, } } - for (Map.Entry> e : cctxs.entrySet()) { - GridCacheContext cctx = ctx.cacheContext(e.getKey()); + CacheMapEntries cacheEntries = new CacheMapEntries(); - CacheMapEntries entries = new CacheMapEntries(); + for (Map.Entry> cctxEntry : cctxs.entrySet()) { + GridCacheContext cctx = ctx.cacheContext(cctxEntry.getKey()); + List cctxInfos = cctxEntry.getValue(); - entries.initialValues(e.getValue(), topVer, cctx, p, true, DR_PRELOAD); + try { + Iterable> cachedEntries = + cacheEntries.initialValues(cctxInfos, topVer, cctx, p, true, DR_PRELOAD); + + for (Map.Entry e : cachedEntries) { + GridCacheMapEntry cached = e.getValue(); + + cached.touch(); + + if (!cctx.events().isRecordable(EVT_CACHE_REBALANCE_OBJECT_LOADED) || cached.isInternal()) + continue; - //TODO: IGNITE-11330: Update metrics for touched cache only. - for (GridCacheContext cctx0 : grp.caches()) { - if (cctx0.statisticsEnabled()) - cctx0.cache().metrics0().onRebalanceKeysReceived(e.getValue().size()); + cctx.events().addEvent(p, cached.key(), cctx.localNodeId(), null, null, null, + EVT_CACHE_REBALANCE_OBJECT_LOADED, e.getKey().value(), true, null, + false, null, null, null, true); + } + } finally { + //TODO: IGNITE-11330: Update metrics for touched cache only. + for (GridCacheContext cctx0 : grp.caches()) { + if (cctx0.statisticsEnabled()) + cctx0.cache().metrics0().onRebalanceKeysReceived(cctxInfos.size()); + } } } } finally { From 5a8ffe018a73d36f754d56b3edb5365abd3c221b Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 11 Apr 2019 11:14:33 +0300 Subject: [PATCH 15/24] IGNITE-11584 Near cache context fix. --- .../processors/cache/CacheMapEntries.java | 23 ++++++++--- .../preloader/GridDhtPartitionDemander.java | 41 +++++++++---------- 2 files changed, 37 insertions(+), 27 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java index f0c7de5569b2f..ad1d0ad78a2cc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -26,6 +26,7 @@ import java.util.Set; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -51,7 +52,7 @@ private static class GridCacheEntryInfoEx extends GridCacheEntryInfo { private final GridCacheEntryInfo delegate; /** */ - private GridCacheMapEntry cacheEntry; + private GridDhtCacheEntry cacheEntry; /** */ private boolean update; @@ -163,9 +164,9 @@ public Collection> initialValue private Collection lockEntries( GridCacheContext cctx, AffinityTopologyVersion topVer, - Collection infos + List infos ) { - List locked = new ArrayList<>(infos.size()); + List locked = new ArrayList<>(infos.size()); while (true) { Map uniqueEntries = new LinkedHashMap<>(); @@ -180,23 +181,29 @@ private Collection lockEntries( assert old == null || ATOMIC_VER_COMPARATOR.compare(old.version(), e.version()) < 0 : "Version order mismatch: prev=" + old.version() + ", current=" + e.version(); - GridCacheMapEntry entry = (GridCacheMapEntry)cctx.cache().entryEx(key, topVer); + GridCacheEntryEx entry = cctx.cache().entryEx(key, topVer); locked.add(entry); - entryEx.cacheEntry = entry; + assert entry instanceof GridDhtCacheEntry; + + entryEx.cacheEntry = (GridDhtCacheEntry)entry; } boolean retry = false; for (int i = 0; i < locked.size(); i++) { - GridCacheMapEntry entry = locked.get(i); + GridCacheEntryEx entry = locked.get(i); if (entry == null) continue; entry.lockEntry(); + GridCacheEntryInfo info = infos.get(i); + + info.value(cctx.kernalContext().cacheObjects().prepareForCache(info.value(), cctx)); + if (entry.obsolete()) { // Unlock all locked. for (int j = 0; j <= i; j++) { @@ -228,6 +235,10 @@ private void unlockEntries( Collection infos, Set skippedKeys ) { + // Process deleted entries before locks release. + // todo + assert cctx.deferredDelete() : this; + try { for (GridCacheEntryInfoEx info : infos) { KeyCacheObject key = info.key(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index 619bbdcff4923..07b7e702e991d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -892,34 +892,27 @@ private void preloadEntriesBatch(ClusterNode from, try { Map> cctxs = new HashMap<>(); - // Map by cache id. + // Group by cache id. for (GridCacheEntryInfo e : infos) { - try { - GridCacheContext cctx0 = grp.sharedGroup() ? ctx.cacheContext(e.cacheId()) : grp.singleCacheContext(); - - if (cctx0 == null) - return; - - if (cctx0.isNear()) - cctx0 = cctx0.dhtCache().context(); - - if (log.isTraceEnabled()) - log.trace("Rebalancing key [key=" + e.key() + ", part=" + p + ", node=" + from.id() + ']'); + if (log.isTraceEnabled()) + log.trace("Rebalancing key [key=" + e.key() + ", part=" + p + ", node=" + from.id() + ']'); - cctxs.computeIfAbsent(cctx0.cacheId(), v -> new ArrayList<>(8)).add(e); - } - catch (GridDhtInvalidPartitionException ignored) { - if (log.isDebugEnabled()) - log.debug("Partition became invalid during rebalancing (will ignore): " + p); - } + cctxs.computeIfAbsent(e.cacheId(), v -> new ArrayList<>(8)).add(e); } CacheMapEntries cacheEntries = new CacheMapEntries(); for (Map.Entry> cctxEntry : cctxs.entrySet()) { - GridCacheContext cctx = ctx.cacheContext(cctxEntry.getKey()); + GridCacheContext cctx = grp.sharedGroup() ? ctx.cacheContext(cctxEntry.getKey()) : grp.singleCacheContext(); + + if (cctx == null) + return; + List cctxInfos = cctxEntry.getValue(); + if (cctx.isNear()) + cctx = cctx.dhtCache().context(); + try { Iterable> cachedEntries = cacheEntries.initialValues(cctxInfos, topVer, cctx, p, true, DR_PRELOAD); @@ -936,7 +929,12 @@ private void preloadEntriesBatch(ClusterNode from, EVT_CACHE_REBALANCE_OBJECT_LOADED, e.getKey().value(), true, null, false, null, null, null, true); } - } finally { + } + catch (GridDhtInvalidPartitionException ignored) { + if (log.isDebugEnabled()) + log.debug("Partition became invalid during rebalancing (will ignore): " + p); + } + finally { //TODO: IGNITE-11330: Update metrics for touched cache only. for (GridCacheContext cctx0 : grp.caches()) { if (cctx0.statisticsEnabled()) @@ -944,7 +942,8 @@ private void preloadEntriesBatch(ClusterNode from, } } } - } finally { + } + finally { grp.listenerLock().readLock().unlock(); } } From e104e6ecebe97ed3b8a6137b3bb8bc889151d1cf Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 11 Apr 2019 11:24:59 +0300 Subject: [PATCH 16/24] IGNITE_11584 Code cleanup. --- .../processors/cache/CacheMapEntries.java | 35 ++------- .../processors/cache/GridCacheMapEntry.java | 74 ++++++++++++------- .../preloader/GridDhtPartitionDemander.java | 2 - 3 files changed, 53 insertions(+), 58 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java index ad1d0ad78a2cc..21e6a73394d7f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -34,6 +34,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgnitePredicate; import static org.apache.ignite.internal.processors.cache.GridCacheMapEntry.ATOMIC_VER_COMPARATOR; @@ -117,38 +118,12 @@ public Collection> initialValue IgniteBiPredicate pred = new IgniteBiPredicate() { @Override public boolean apply(CacheDataRow row, GridCacheEntryInfo info) { - try { - GridCacheVersion currVer = row != null ? row.version() : - ((GridCacheEntryInfoEx)info).cacheEntry.version(); + GridCacheEntryInfoEx infoEx = (GridCacheEntryInfoEx)info; - boolean isStartVer = cctx.versions().isStartVersion(currVer); + IgnitePredicate p = + new GridCacheMapEntry.InitialValuePredicate(infoEx.cacheEntry, info.version(), preload); - boolean update0; - - if (cctx.group().persistenceEnabled()) { - if (!isStartVer) { - if (cctx.atomic()) - update0 = ATOMIC_VER_COMPARATOR.compare(currVer, info.version()) < 0; - else - update0 = currVer.compareTo(info.version()) < 0; - } - else - update0 = true; - } - else - update0 = (isStartVer && row == null); - - update0 |= (!preload && ((GridCacheEntryInfoEx)info).cacheEntry.deletedUnlocked()); - - ((GridCacheEntryInfoEx)info).update = update0; - - return update0; - } - catch (GridCacheEntryRemovedException e) { - skipped.add(info.key()); - - return false; - } + return infoEx.update = p.apply(row); } }; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 39c9a6cfb1624..f3351ecb53aa1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -3307,6 +3307,53 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { } } + /** */ + protected static class InitialValuePredicate implements IgnitePredicate { + /** */ + private final GridCacheMapEntry entry;; + + /** */ + private final boolean preload; + + /** */ + private final GridCacheVersion newVer; + + /** */ + InitialValuePredicate(GridCacheMapEntry entry, GridCacheVersion newVer, boolean preload) { + this.entry = entry; + this.preload = preload; + this.newVer = newVer; + } + + /** {@inheritDoc} */ + @Override public boolean apply(@Nullable CacheDataRow row) { + boolean update0; + + GridCacheVersion currentVer = row != null ? row.version() : entry.ver; + + GridCacheContext cctx = entry.cctx; + + boolean isStartVer = cctx.shared().versions().isStartVersion(currentVer); + + if (cctx.group().persistenceEnabled()) { + if (!isStartVer) { + if (cctx.atomic()) + update0 = ATOMIC_VER_COMPARATOR.compare(currentVer, newVer) < 0; + else + update0 = currentVer.compareTo(newVer) < 0; + } + else + update0 = true; + } + else + update0 = isStartVer; + + update0 |= (!preload && entry.deletedUnlocked()); + + return update0; + } + }; + /** {@inheritDoc} */ @Override public boolean initialValue( CacheObject val, @@ -3345,32 +3392,7 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { boolean update; - IgnitePredicate p = new IgnitePredicate() { - @Override public boolean apply(@Nullable CacheDataRow row) { - boolean update0; - - GridCacheVersion currentVer = row != null ? row.version() : GridCacheMapEntry.this.ver; - - boolean isStartVer = cctx.shared().versions().isStartVersion(currentVer); - - if (cctx.group().persistenceEnabled()) { - if (!isStartVer) { - if (cctx.atomic()) - update0 = ATOMIC_VER_COMPARATOR.compare(currentVer, ver) < 0; - else - update0 = currentVer.compareTo(ver) < 0; - } - else - update0 = true; - } - else - update0 = isStartVer; - - update0 |= (!preload && deletedUnlocked()); - - return update0; - } - }; + IgnitePredicate p = new InitialValuePredicate(this, ver, preload); if (unswapped) { update = p.apply(null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index 07b7e702e991d..88b961eab92c1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -67,12 +67,10 @@ import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; -import org.apache.ignite.internal.util.lang.IgniteInClosure2X; import org.apache.ignite.internal.util.lang.IgniteInClosureX; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; From c9200421528d9f7ac00743ebaa4fda561c7261b2 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 11 Apr 2019 17:36:50 +0300 Subject: [PATCH 17/24] IGNITE-11584 Bench fix. --- ...k.java => JmhBatchUpdatesInPreloadBenchmark.java} | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) rename modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/{JmhBatchUpdatesBenchmark.java => JmhBatchUpdatesInPreloadBenchmark.java} (98%) diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesInPreloadBenchmark.java similarity index 98% rename from modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java rename to modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesInPreloadBenchmark.java index 17a9080f1276a..a25571a206215 100644 --- a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesBenchmark.java +++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/pagemem/JmhBatchUpdatesInPreloadBenchmark.java @@ -78,14 +78,14 @@ */ @BenchmarkMode(Mode.AverageTime) @Fork(value = 1, jvmArgsAppend = {"-Xms3g", "-Xmx3g", "-server", "-XX:+AggressiveOpts", "-XX:MaxMetaspaceSize=256m"}) -@Measurement(iterations = 11) +@Measurement(iterations = 5) @OutputTimeUnit(TimeUnit.MICROSECONDS) @State(Scope.Benchmark) @Threads(1) -@Warmup(iterations = 15) -public class JmhBatchUpdatesBenchmark { +@Warmup(iterations = 10) +public class JmhBatchUpdatesInPreloadBenchmark { /** */ - private static final long DEF_REG_SIZE = 3 * 1024 * 1024 * 1024L; + private static final long DEF_REG_SIZE = 20 * 1024 * 1024 * 1024L; /** */ private static final int BATCH_SIZE = 500; @@ -325,7 +325,7 @@ public void setup() { /** * Prepare collection. */ - @Setup(Level.Iteration) + @Setup(Level.Invocation) public void prepare() { int iter = iteration++; int off = iter * BATCH_SIZE; @@ -536,7 +536,7 @@ private static void setFieldValue(Object obj, Class cls, String fieldName, Objec */ public static void main(String[] args) throws RunnerException { final Options options = new OptionsBuilder() - .include(JmhBatchUpdatesBenchmark.class.getSimpleName()) + .include(JmhBatchUpdatesInPreloadBenchmark.class.getSimpleName()) .build(); new Runner(options).run(); From f8b7c236302646e704399885fadec7997cabb1d2 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 11 Apr 2019 19:39:23 +0300 Subject: [PATCH 18/24] wip --- .../processors/cache/CacheMapEntries.java | 2 +- .../cache/GridCacheEntryProcessor.java | 455 ++++++++++++++++++ .../processors/cache/GridCacheMapEntry.java | 184 +------ .../preloader/GridDhtPartitionDemander.java | 2 +- 4 files changed, 479 insertions(+), 164 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryProcessor.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java index 21e6a73394d7f..23c8dae0f4fc8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -121,7 +121,7 @@ public Collection> initialValue GridCacheEntryInfoEx infoEx = (GridCacheEntryInfoEx)info; IgnitePredicate p = - new GridCacheMapEntry.InitialValuePredicate(infoEx.cacheEntry, info.version(), preload); + new GridCacheEntryProcessor.InitialValuePredicate(infoEx.cacheEntry, info.version(), preload); return infoEx.update = p.apply(row); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryProcessor.java new file mode 100644 index 0000000000000..5b64b4bc0cd8d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryProcessor.java @@ -0,0 +1,455 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.record.DataEntry; +import org.apache.ignite.internal.pagemem.wal.record.DataRecord; +import org.apache.ignite.internal.pagemem.wal.record.MvccDataEntry; +import org.apache.ignite.internal.pagemem.wal.record.MvccDataRecord; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; +import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; +import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; +import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.DataRegion; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.dr.GridDrType; +import org.apache.ignite.internal.util.IgniteTree; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgnitePredicate; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_EXPIRED; +import static org.apache.ignite.internal.processors.cache.GridCacheMapEntry.ATOMIC_VER_COMPARATOR; +import static org.apache.ignite.internal.processors.cache.GridCacheMapEntry.IS_UNSWAPPED_MASK; +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE; +import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE; + +/** */ +public class GridCacheEntryProcessor { + /** */ + private final GridCacheContext cctx; + + /** */ + GridCacheEntryProcessor(GridCacheContext cctx) { + this.cctx = cctx; + } + + /** {@inheritDoc} */ + public boolean initialValue( + GridCacheMapEntry entry, + CacheObject val, + GridCacheVersion ver, + MvccVersion mvccVer, + MvccVersion newMvccVer, + byte mvccTxState, + byte newMvccTxState, + long ttl, + long expireTime, + boolean preload, + AffinityTopologyVersion topVer, + GridDrType drType, + boolean fromStore + ) throws IgniteCheckedException, GridCacheEntryRemovedException { + ensureFreeSpace(entry); + + boolean deferred = false; + boolean obsolete = false; + + GridCacheVersion oldVer = null; + + entry.lockListenerReadLock(); + entry.lockEntry(); + + try { + entry.checkObsolete(); + + long expTime = expireTime < 0 ? CU.toExpireTime(ttl) : expireTime; + + val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx); + + final boolean unswapped = ((entry.flags & IS_UNSWAPPED_MASK) != 0); + + boolean update; + + IgnitePredicate p = new InitialValuePredicate(entry, ver, preload); + + if (unswapped) { + update = p.apply(null); + + if (update) { + // If entry is already unswapped and we are modifying it, we must run deletion callbacks for old value. + long oldExpTime = entry.expireTimeUnlocked(); + + if (oldExpTime > 0 && oldExpTime < U.currentTimeMillis()) { + if (onExpired(entry, entry.val, null)) { + if (cctx.deferredDelete()) { + deferred = true; + oldVer = entry.ver; + } + else if (val == null) + obsolete = true; + } + } + + if (cctx.mvccEnabled()) { + assert !preload; + + cctx.offheap().mvccInitialValue(entry, val, ver, expTime, mvccVer, newMvccVer); + } + else + storeValue(entry, val, expTime, ver, null); + } + } + else { + if (cctx.mvccEnabled()) { + // cannot identify whether the entry is exist on the fly + entry.unswap(false); + + if (update = p.apply(null)) { + // If entry is already unswapped and we are modifying it, we must run deletion callbacks for old value. + long oldExpTime = entry.expireTimeUnlocked(); + long delta = (oldExpTime == 0 ? 0 : oldExpTime - U.currentTimeMillis()); + + if (delta < 0) { + if (onExpired(entry, entry.val, null)) { + if (cctx.deferredDelete()) { + deferred = true; + oldVer = entry.ver; + } + else if (val == null) + obsolete = true; + } + } + + assert !preload; + + cctx.offheap().mvccInitialValue(entry, val, ver, expTime, mvccVer, newMvccVer); + } + } + else + // Optimization to access storage only once. + update = storeValue(entry, val, expTime, ver, p); + } + + if (update) { + finishInitialUpdate(entry, val, expireTime, ttl, ver, topVer, drType, mvccVer, preload); + + return true; + } + + return false; + } + finally { + entry.unlockEntry(); + entry.unlockListenerReadLock(); + + // It is necessary to execute these callbacks outside of lock to avoid deadlocks. + + if (obsolete) { + entry.onMarkedObsolete(); + + cctx.cache().removeEntry(entry); + } + + if (deferred) { + assert oldVer != null; + + cctx.onDeferredDelete(entry, oldVer); + } + } + } + + /** + * todo explain this and remove code duplication + * @param val New value. + * @param expireTime Expiration time. + * @param ttl Time to live. + * @param ver Version to use. + * @param topVer Topology version. + * @param drType DR type. + * @param mvccVer Mvcc version. + * @param preload Flag indicating whether entry is being preloaded. + * @throws IgniteCheckedException In case of error. + */ + protected void finishInitialUpdate( + GridCacheMapEntry entry, + @Nullable CacheObject val, + long expireTime, + long ttl, + GridCacheVersion ver, + AffinityTopologyVersion topVer, + GridDrType drType, + MvccVersion mvccVer, + boolean preload + ) throws IgniteCheckedException { + boolean fromStore = false; + boolean walEnabled = !cctx.isNear() && cctx.group().persistenceEnabled() && cctx.group().walEnabled(); + + entry.update(val, expireTime, ttl, ver, true); + + boolean skipQryNtf = false; + + if (val == null) { + skipQryNtf = true; + + if (cctx.deferredDelete() && !entry.deletedUnlocked() && !entry.isInternal()) + entry.deletedUnlocked(true); + } + else if (entry.deletedUnlocked()) + entry.deletedUnlocked(false); + + long updateCntr = 0; + + if (!preload) + // todo update counters is not applicable to cache entry and should be moved + updateCntr = entry.nextPartitionCounter(topVer, true, null); + + if (walEnabled) { + if (cctx.mvccEnabled()) { + cctx.shared().wal().log(new MvccDataRecord(new MvccDataEntry( + cctx.cacheId(), + entry.key, + val, + val == null ? DELETE : GridCacheOperation.CREATE, + null, + ver, + expireTime, + entry.partition(), + updateCntr, + mvccVer == null ? MvccUtils.INITIAL_VERSION : mvccVer + ))); + } else { + cctx.shared().wal().log(new DataRecord(new DataEntry( + cctx.cacheId(), + entry.key, + val, + val == null ? DELETE : GridCacheOperation.CREATE, + null, + ver, + expireTime, + entry.partition(), + updateCntr + ))); + } + } + + drReplicate(entry, drType, val, ver, topVer); + + if (!skipQryNtf) { + cctx.continuousQueries().onEntryUpdated( + entry.key, + val, + null, + entry.isInternal() || !entry.context().userCache(), + entry.partition(), + true, + preload, + updateCntr, + null, + topVer); + } + + entry.onUpdateFinished(updateCntr); + + if (!fromStore && cctx.store().isLocal()) { + if (val != null) + cctx.store().put(null, entry.key, val, ver); + } + } + + /** + * Stores value in off-heap. + * + * @param val Value. + * @param expireTime Expire time. + * @param ver New entry version. + * @param predicate Optional predicate. + * @return {@code True} if storage was modified. + * @throws IgniteCheckedException If update failed. + */ + protected boolean storeValue( + GridCacheMapEntry entry, + @Nullable CacheObject val, + long expireTime, + GridCacheVersion ver, + @Nullable IgnitePredicate predicate) throws IgniteCheckedException { + assert entry.lock.isHeldByCurrentThread(); + + GridCacheMapEntry.UpdateClosure closure = new GridCacheMapEntry.UpdateClosure(entry, val, ver, expireTime, predicate); + + cctx.offheap().invoke(cctx, entry.key(), entry.localPartition(), closure); + + return closure.treeOp != IgniteTree.OperationType.NOOP; + } + + /** + * Perform DR if needed. + * + * @param drType DR type. + * @param val Value. + * @param ver Version. + * @param topVer Topology version. + * @throws IgniteCheckedException In case of exception. + */ + private void drReplicate(GridCacheMapEntry entry, GridDrType drType, @Nullable CacheObject val, GridCacheVersion ver, AffinityTopologyVersion topVer) + throws IgniteCheckedException { + if (cctx.isDrEnabled() && drType != DR_NONE && !entry.isInternal()) + cctx.dr().replicate(entry.key, val, entry.rawTtl(), entry.rawExpireTime(), ver.conflictVersion(), drType, topVer); + } + + /** + * Evicts necessary number of data pages if per-page eviction is configured in current {@link DataRegion}. + */ + private void ensureFreeSpace(GridCacheMapEntry entry) throws IgniteCheckedException { + // Deadlock alert: evicting data page causes removing (and locking) all entries on the page one by one. + assert !entry.lock.isHeldByCurrentThread(); + + cctx.shared().database().ensureFreeSpace(cctx.dataRegion()); + } + + /** + * @param expiredVal Expired value. + * @param obsoleteVer Version. + * @return {@code True} if entry was marked as removed. + * @throws IgniteCheckedException If failed. + */ + private boolean onExpired(GridCacheMapEntry entry, CacheObject expiredVal, GridCacheVersion obsoleteVer) throws IgniteCheckedException { + assert expiredVal != null; + + boolean rmvd = false; + + if (entry.mvccExtras() != null) + return false; + + if (cctx.deferredDelete() && !entry.detached() && !entry.isInternal()) { + if (!entry.deletedUnlocked() && !entry.isStartVersion()) { + entry.update(null, 0L, 0L, entry.ver, true); + + entry.deletedUnlocked(true); + + rmvd = true; + } + } + else { + if (obsoleteVer == null) + obsoleteVer = cctx.versions().next(entry.ver); + + if (entry.markObsolete0(obsoleteVer, true, null)) + rmvd = true; + } + +// if (log.isTraceEnabled()) +// log.trace("onExpired clear [key=" + key + ", entry=" + System.identityHashCode(this) + ']'); + + cctx.shared().database().checkpointReadLock(); + + try { + if (cctx.mvccEnabled()) + cctx.offheap().mvccRemoveAll(entry); + else + removeValue(entry); + } + finally { + cctx.shared().database().checkpointReadUnlock(); + } + + if (cctx.events().isRecordable(EVT_CACHE_OBJECT_EXPIRED)) { + cctx.events().addEvent(entry.partition(), + entry.key, + cctx.localNodeId(), + null, + EVT_CACHE_OBJECT_EXPIRED, + null, + false, + expiredVal, + expiredVal != null, + null, + null, + null, + true); + } + + cctx.continuousQueries().onEntryExpired(entry, entry.key, expiredVal); + + return rmvd; + } + + /** + * Removes value from offheap. + * + * @throws IgniteCheckedException If failed. + */ + protected void removeValue(GridCacheMapEntry entry) throws IgniteCheckedException { + assert entry.lock.isHeldByCurrentThread(); + + cctx.offheap().remove(cctx, entry.key, entry.partition(), entry.localPartition()); + } + + /** */ + protected static class InitialValuePredicate implements IgnitePredicate { + /** */ + private final GridCacheMapEntry entry;; + + /** */ + private final boolean preload; + + /** */ + private final GridCacheVersion newVer; + + /** */ + InitialValuePredicate(GridCacheMapEntry entry, GridCacheVersion newVer, boolean preload) { + this.entry = entry; + this.preload = preload; + this.newVer = newVer; + } + + /** {@inheritDoc} */ + @Override public boolean apply(@Nullable CacheDataRow row) { + boolean update0; + + GridCacheVersion currentVer = row != null ? row.version() : entry.ver; + + GridCacheContext cctx = entry.cctx; + + boolean isStartVer = cctx.shared().versions().isStartVersion(currentVer); + + if (cctx.group().persistenceEnabled()) { + if (!isStartVer) { + if (cctx.atomic()) + update0 = ATOMIC_VER_COMPARATOR.compare(currentVer, newVer) < 0; + else + update0 = currentVer.compareTo(newVer) < 0; + } + else + update0 = true; + } + else + update0 = isStartVer; + + update0 |= (!preload && entry.deletedUnlocked()); + + return update0; + } + }; +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index f3351ecb53aa1..838747d95012e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -133,7 +133,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme private static final byte IS_DELETED_MASK = 0x01; /** */ - private static final byte IS_UNSWAPPED_MASK = 0x02; + static final byte IS_UNSWAPPED_MASK = 0x02; /** */ private static final byte IS_EVICT_DISABLED = 0x04; @@ -225,12 +225,14 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme /** */ @GridToStringExclude - private final ReentrantLock lock = new ReentrantLock(); + final ReentrantLock lock = new ReentrantLock(); /** Read Lock for continuous query listener */ @GridToStringExclude private final ReadWriteLock listenerLock; + private final GridCacheEntryProcessor proc; + /** * Flags: *
    @@ -254,6 +256,8 @@ protected GridCacheMapEntry( key = (KeyCacheObject)cctx.kernalContext().cacheObjects().prepareForCache(key, cctx); + proc = new GridCacheEntryProcessor(cctx); + assert key != null; this.key = key; @@ -3307,53 +3311,6 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { } } - /** */ - protected static class InitialValuePredicate implements IgnitePredicate { - /** */ - private final GridCacheMapEntry entry;; - - /** */ - private final boolean preload; - - /** */ - private final GridCacheVersion newVer; - - /** */ - InitialValuePredicate(GridCacheMapEntry entry, GridCacheVersion newVer, boolean preload) { - this.entry = entry; - this.preload = preload; - this.newVer = newVer; - } - - /** {@inheritDoc} */ - @Override public boolean apply(@Nullable CacheDataRow row) { - boolean update0; - - GridCacheVersion currentVer = row != null ? row.version() : entry.ver; - - GridCacheContext cctx = entry.cctx; - - boolean isStartVer = cctx.shared().versions().isStartVersion(currentVer); - - if (cctx.group().persistenceEnabled()) { - if (!isStartVer) { - if (cctx.atomic()) - update0 = ATOMIC_VER_COMPARATOR.compare(currentVer, newVer) < 0; - else - update0 = currentVer.compareTo(newVer) < 0; - } - else - update0 = true; - } - else - update0 = isStartVer; - - update0 |= (!preload && entry.deletedUnlocked()); - - return update0; - } - }; - /** {@inheritDoc} */ @Override public boolean initialValue( CacheObject val, @@ -3369,115 +3326,18 @@ protected static class InitialValuePredicate implements IgnitePredicate p = new InitialValuePredicate(this, ver, preload); - - if (unswapped) { - update = p.apply(null); - - if (update) { - // If entry is already unswapped and we are modifying it, we must run deletion callbacks for old value. - long oldExpTime = expireTimeUnlocked(); - - if (oldExpTime > 0 && oldExpTime < U.currentTimeMillis()) { - if (onExpired(this.val, null)) { - if (cctx.deferredDelete()) { - deferred = true; - oldVer = this.ver; - } - else if (val == null) - obsolete = true; - } - } - - if (cctx.mvccEnabled()) { - assert !preload; - - cctx.offheap().mvccInitialValue(this, val, ver, expTime, mvccVer, newMvccVer); - } - else - storeValue(val, expTime, ver); - } - } - else { - if (cctx.mvccEnabled()) { - // cannot identify whether the entry is exist on the fly - unswap(false); - - if (update = p.apply(null)) { - // If entry is already unswapped and we are modifying it, we must run deletion callbacks for old value. - long oldExpTime = expireTimeUnlocked(); - long delta = (oldExpTime == 0 ? 0 : oldExpTime - U.currentTimeMillis()); - - if (delta < 0) { - if (onExpired(this.val, null)) { - if (cctx.deferredDelete()) { - deferred = true; - oldVer = this.ver; - } - else if (val == null) - obsolete = true; - } - } - - assert !preload; - - cctx.offheap().mvccInitialValue(this, val, ver, expTime, mvccVer, newMvccVer); - } - } - else - // Optimization to access storage only once. - update = storeValue(val, expTime, ver, p); - } - - if (update) { - finishInitialUpdate(val, expireTime, ttl, ver, topVer, drType, mvccVer, preload); - - return true; - } - - return false; - } - finally { - unlockEntry(); - unlockListenerReadLock(); - - // It is necessary to execute these callbacks outside of lock to avoid deadlocks. - - if (obsolete) { - onMarkedObsolete(); - - cctx.cache().removeEntry(this); - } - - if (deferred) { - assert oldVer != null; - - cctx.onDeferredDelete(this, oldVer); - } - } + return proc.initialValue(this, val, + ver, + mvccVer, + newMvccVer, + mvccTxState, + newMvccTxState, + ttl, + expireTime, + preload, + topVer, + drType, + fromStore); } /** @@ -5067,7 +4927,7 @@ private int extrasSize() { * in order to ensure that the entry update is completed and existing continuous * query notified before the next cache listener update */ - private void lockListenerReadLock() { + void lockListenerReadLock() { listenerLock.readLock().lock(); } @@ -5076,7 +4936,7 @@ private void lockListenerReadLock() { * * @see #lockListenerReadLock() */ - private void unlockListenerReadLock() { + void unlockListenerReadLock() { listenerLock.readLock().unlock(); } @@ -5734,7 +5594,7 @@ private LazyValueEntry(KeyCacheObject key, boolean keepBinary) { /** * */ - private static class UpdateClosure implements IgniteCacheOffheapManager.OffheapInvokeClosure { + static class UpdateClosure implements IgniteCacheOffheapManager.OffheapInvokeClosure { /** */ private final GridCacheMapEntry entry; @@ -5757,7 +5617,7 @@ private static class UpdateClosure implements IgniteCacheOffheapManager.OffheapI private CacheDataRow oldRow; /** */ - private IgniteTree.OperationType treeOp = IgniteTree.OperationType.PUT; + IgniteTree.OperationType treeOp = IgniteTree.OperationType.PUT; /** * @param entry Entry. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index 88b961eab92c1..621bd63d313a5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -97,7 +97,7 @@ public class GridDhtPartitionDemander { /** */ private static final boolean BATCH_PAGE_WRITE_ENABLED = - IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, true); + IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, false); /** */ private final GridCacheSharedContext ctx; From aec36bcaa362dda6cea41cb0e70a65a774598dcc Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Fri, 12 Apr 2019 12:17:27 +0300 Subject: [PATCH 19/24] IGNITE-11584 Simplify initialValue. --- .../processors/cache/CacheMapEntries.java | 4 +- .../processors/cache/GridCacheEntryEx.java | 8 +- .../cache/GridCacheEntryProcessor.java | 50 +++------ .../processors/cache/GridCacheMapEntry.java | 102 +----------------- .../distributed/dht/GridDhtLockFuture.java | 3 - .../dht/GridDhtTxPrepareFuture.java | 4 - .../dht/preloader/GridDhtForceKeysFuture.java | 3 - .../preloader/GridDhtPartitionDemander.java | 3 - .../cache/GridCacheTestEntryEx.java | 2 - 9 files changed, 17 insertions(+), 162 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java index 23c8dae0f4fc8..ca8fd2d5e0706 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -234,8 +234,8 @@ private void unlockEntries( } try { - entry.finishInitialUpdate(info.value(), info.expireTime(), info.ttl(), info.version(), topVer, - drType, null, preload); + entry.proc.finishInitialUpdate(entry, info.value(), info.expireTime(), info.ttl(), info.version(), topVer, + drType, null, preload, false); } catch (IgniteCheckedException ex) { cctx.logger(getClass()).error("Unable to finish initial update, skip " + key, ex); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java index 9aec3996c3204..8ab00495775c9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java @@ -33,7 +33,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; -import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -769,8 +768,7 @@ default boolean initialValue(CacheObject val, AffinityTopologyVersion topVer, GridDrType drType, boolean fromStore) throws IgniteCheckedException, GridCacheEntryRemovedException { - return initialValue(val, ver, null, null, TxState.NA, TxState.NA, - ttl, expireTime, preload, topVer, drType, fromStore); + return initialValue(val, ver, null, null, ttl, expireTime, preload, topVer, drType, fromStore); } /** @@ -780,8 +778,6 @@ default boolean initialValue(CacheObject val, * @param ver Version to use. * @param mvccVer Mvcc version. * @param newMvccVer New mvcc version. - * @param mvccTxState Tx state hint for mvcc version. - * @param newMvccTxState Tx state hint for new mvcc version. * @param ttl Time to live. * @param expireTime Expiration time. * @param preload Flag indicating whether entry is being preloaded. @@ -796,8 +792,6 @@ public boolean initialValue(CacheObject val, GridCacheVersion ver, @Nullable MvccVersion mvccVer, @Nullable MvccVersion newMvccVer, - byte mvccTxState, - byte newMvccTxState, long ttl, long expireTime, boolean preload, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryProcessor.java index 5b64b4bc0cd8d..e61ade6e9e278 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryProcessor.java @@ -62,8 +62,6 @@ public boolean initialValue( GridCacheVersion ver, MvccVersion mvccVer, MvccVersion newMvccVer, - byte mvccTxState, - byte newMvccTxState, long ttl, long expireTime, boolean preload, @@ -90,14 +88,15 @@ public boolean initialValue( final boolean unswapped = ((entry.flags & IS_UNSWAPPED_MASK) != 0); - boolean update; + boolean update = false; IgnitePredicate p = new InitialValuePredicate(entry, ver, preload); - if (unswapped) { - update = p.apply(null); + if (unswapped || cctx.mvccEnabled()) { + if (!unswapped) + entry.unswap(false); - if (update) { + if (update = p.apply(null)) { // If entry is already unswapped and we are modifying it, we must run deletion callbacks for old value. long oldExpTime = entry.expireTimeUnlocked(); @@ -118,42 +117,19 @@ else if (val == null) cctx.offheap().mvccInitialValue(entry, val, ver, expTime, mvccVer, newMvccVer); } else - storeValue(entry, val, expTime, ver, null); + p = null; } } - else { - if (cctx.mvccEnabled()) { - // cannot identify whether the entry is exist on the fly - entry.unswap(false); - - if (update = p.apply(null)) { - // If entry is already unswapped and we are modifying it, we must run deletion callbacks for old value. - long oldExpTime = entry.expireTimeUnlocked(); - long delta = (oldExpTime == 0 ? 0 : oldExpTime - U.currentTimeMillis()); - - if (delta < 0) { - if (onExpired(entry, entry.val, null)) { - if (cctx.deferredDelete()) { - deferred = true; - oldVer = entry.ver; - } - else if (val == null) - obsolete = true; - } - } - assert !preload; + if (!cctx.mvccEnabled() && (!unswapped || (unswapped && p == null))) { + boolean update0 = storeValue(entry, val, expTime, ver, p); - cctx.offheap().mvccInitialValue(entry, val, ver, expTime, mvccVer, newMvccVer); - } - } - else - // Optimization to access storage only once. - update = storeValue(entry, val, expTime, ver, p); + if (p != null) + update = update0; } if (update) { - finishInitialUpdate(entry, val, expireTime, ttl, ver, topVer, drType, mvccVer, preload); + finishInitialUpdate(entry, val, expireTime, ttl, ver, topVer, drType, mvccVer, preload, fromStore); return true; } @@ -201,9 +177,9 @@ protected void finishInitialUpdate( AffinityTopologyVersion topVer, GridDrType drType, MvccVersion mvccVer, - boolean preload + boolean preload, + boolean fromStore ) throws IgniteCheckedException { - boolean fromStore = false; boolean walEnabled = !cctx.isNear() && cctx.group().persistenceEnabled() && cctx.group().walEnabled(); entry.update(val, expireTime, ttl, ver, true); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 838747d95012e..1907e23a645fb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -231,7 +231,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme @GridToStringExclude private final ReadWriteLock listenerLock; - private final GridCacheEntryProcessor proc; + protected final GridCacheEntryProcessor proc; /** * Flags: @@ -3317,8 +3317,6 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { GridCacheVersion ver, MvccVersion mvccVer, MvccVersion newMvccVer, - byte mvccTxState, - byte newMvccTxState, long ttl, long expireTime, boolean preload, @@ -3330,8 +3328,6 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { ver, mvccVer, newMvccVer, - mvccTxState, - newMvccTxState, ttl, expireTime, preload, @@ -3340,102 +3336,6 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { fromStore); } - /** - * todo explain this and remove code duplication - * @param val New value. - * @param expireTime Expiration time. - * @param ttl Time to live. - * @param ver Version to use. - * @param topVer Topology version. - * @param drType DR type. - * @param mvccVer Mvcc version. - * @param preload Flag indicating whether entry is being preloaded. - * @throws IgniteCheckedException In case of error. - */ - protected void finishInitialUpdate( - @Nullable CacheObject val, - long expireTime, - long ttl, - GridCacheVersion ver, - AffinityTopologyVersion topVer, - GridDrType drType, - MvccVersion mvccVer, - boolean preload - ) throws IgniteCheckedException { - boolean fromStore = false; - boolean walEnabled = !cctx.isNear() && cctx.group().persistenceEnabled() && cctx.group().walEnabled(); - - update(val, expireTime, ttl, ver, true); - - boolean skipQryNtf = false; - - if (val == null) { - skipQryNtf = true; - - if (cctx.deferredDelete() && !deletedUnlocked() && !isInternal()) - deletedUnlocked(true); - } - else if (deletedUnlocked()) - deletedUnlocked(false); - - long updateCntr = 0; - - if (!preload) - updateCntr = nextPartitionCounter(topVer, true, null); - - if (walEnabled) { - if (cctx.mvccEnabled()) { - cctx.shared().wal().log(new MvccDataRecord(new MvccDataEntry( - cctx.cacheId(), - key, - val, - val == null ? DELETE : GridCacheOperation.CREATE, - null, - ver, - expireTime, - partition(), - updateCntr, - mvccVer == null ? MvccUtils.INITIAL_VERSION : mvccVer - ))); - } else { - cctx.shared().wal().log(new DataRecord(new DataEntry( - cctx.cacheId(), - key, - val, - val == null ? DELETE : GridCacheOperation.CREATE, - null, - ver, - expireTime, - partition(), - updateCntr - ))); - } - } - - drReplicate(drType, val, ver, topVer); - - if (!skipQryNtf) { - cctx.continuousQueries().onEntryUpdated( - key, - val, - null, - this.isInternal() || !this.context().userCache(), - this.partition(), - true, - preload, - updateCntr, - null, - topVer); - } - - onUpdateFinished(updateCntr); - - if (!fromStore && cctx.store().isLocal()) { - if (val != null) - cctx.store().put(null, key, val, ver); - } - } - /** * @param cntr Updated partition counter. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java index 4f3581acd1c52..003c0851465a7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java @@ -57,7 +57,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; import org.apache.ignite.internal.processors.cache.mvcc.MvccUpdateVersionAware; import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; -import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -1380,8 +1379,6 @@ void onResult(GridDhtLockResponse res) { info.version(), cctx.mvccEnabled() ? ((MvccVersionAware)info).mvccVersion() : null, cctx.mvccEnabled() ? ((MvccUpdateVersionAware)info).newMvccVersion() : null, - cctx.mvccEnabled() ? ((MvccVersionAware)entry).mvccTxState() : TxState.NA, - cctx.mvccEnabled() ? ((MvccUpdateVersionAware)entry).newMvccTxState() : TxState.NA, info.ttl(), info.expireTime(), true, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java index 97a1f03da25db..87e17521e61b6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java @@ -67,7 +67,6 @@ import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot; import org.apache.ignite.internal.processors.cache.mvcc.MvccUpdateVersionAware; import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; -import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -75,7 +74,6 @@ import org.apache.ignite.internal.processors.dr.GridDrType; import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException; -import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -1910,8 +1908,6 @@ void onResult(GridDhtTxPrepareResponse res) { info.version(), cacheCtx.mvccEnabled() ? ((MvccVersionAware)info).mvccVersion() : null, cacheCtx.mvccEnabled() ? ((MvccUpdateVersionAware)info).newMvccVersion() : null, - cacheCtx.mvccEnabled() ? ((MvccVersionAware)info).mvccTxState() : TxState.NA, - cacheCtx.mvccEnabled() ? ((MvccUpdateVersionAware)info).newMvccTxState() : TxState.NA, info.ttl(), info.expireTime(), true, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java index fba1d9da58b1b..a440e20a0e558 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java @@ -44,7 +44,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology; import org.apache.ignite.internal.processors.cache.mvcc.MvccUpdateVersionAware; import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; -import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.internal.util.future.GridCompoundFuture; @@ -542,8 +541,6 @@ void onResult(GridDhtForceKeysResponse res) { info.version(), cctx.mvccEnabled() ? ((MvccVersionAware)info).mvccVersion() : null, cctx.mvccEnabled() ? ((MvccUpdateVersionAware)info).newMvccVersion() : null, - cctx.mvccEnabled() ? ((MvccVersionAware)entry).mvccTxState() : TxState.NA, - cctx.mvccEnabled() ? ((MvccUpdateVersionAware)entry).newMvccTxState() : TxState.NA, info.ttl(), info.expireTime(), true, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index 621bd63d313a5..079e4eb39e800 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -61,7 +61,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology; import org.apache.ignite.internal.processors.cache.mvcc.MvccUpdateVersionAware; import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware; -import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; import org.apache.ignite.internal.util.future.GridCompoundFuture; @@ -1135,8 +1134,6 @@ private boolean preloadEntry( entry.version(), cctx.mvccEnabled() ? ((MvccVersionAware)entry).mvccVersion() : null, cctx.mvccEnabled() ? ((MvccUpdateVersionAware)entry).newMvccVersion() : null, - cctx.mvccEnabled() ? ((MvccVersionAware)entry).mvccTxState() : TxState.NA, - cctx.mvccEnabled() ? ((MvccUpdateVersionAware)entry).newMvccTxState() : TxState.NA, entry.ttl(), entry.expireTime(), true, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index 8d1ab878f2970..4d46ba6f5ad89 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -704,8 +704,6 @@ void recheckLock() { GridCacheVersion ver, MvccVersion mvccVer, MvccVersion newMvccVer, - byte mvccTxState, - byte newMvccTxState, long ttl, long expireTime, boolean preload, From 54aa2c6a2ed8273e9ad958f9e57dfba0c3c0175f Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Fri, 12 Apr 2019 14:02:10 +0300 Subject: [PATCH 20/24] batch (wip). --- ...java => CacheEntryInitialValuesBatch.java} | 231 +++++++----------- .../processors/cache/CacheMapEntries.java | 20 +- .../processors/cache/GridCacheMapEntry.java | 15 +- 3 files changed, 102 insertions(+), 164 deletions(-) rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/{GridCacheEntryProcessor.java => CacheEntryInitialValuesBatch.java} (64%) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java similarity index 64% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryProcessor.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java index e61ade6e9e278..93b4ad51b034b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java @@ -17,46 +17,74 @@ package org.apache.ignite.internal.processors.cache; +import java.util.ArrayList; +import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; import org.apache.ignite.internal.pagemem.wal.record.DataRecord; import org.apache.ignite.internal.pagemem.wal.record.MvccDataEntry; import org.apache.ignite.internal.pagemem.wal.record.MvccDataRecord; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.CacheObject; -import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; -import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; -import org.apache.ignite.internal.processors.cache.persistence.DataRegion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.dr.GridDrType; -import org.apache.ignite.internal.util.IgniteTree; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; -import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_EXPIRED; import static org.apache.ignite.internal.processors.cache.GridCacheMapEntry.ATOMIC_VER_COMPARATOR; import static org.apache.ignite.internal.processors.cache.GridCacheMapEntry.IS_UNSWAPPED_MASK; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE; -import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE; /** */ -public class GridCacheEntryProcessor { +public class CacheEntryInitialValuesBatch { /** */ private final GridCacheContext cctx; /** */ - GridCacheEntryProcessor(GridCacheContext cctx) { + public CacheEntryInitialValuesBatch(GridCacheContext cctx) { this.cctx = cctx; } - /** {@inheritDoc} */ - public boolean initialValue( + /** */ + private List initialValues = new ArrayList<>(1); + + class InitialValue { + private final GridCacheMapEntry entry; + private final CacheObject val; + private final GridCacheVersion ver; + private final MvccVersion mvccVer; + private final MvccVersion newMvccVer; + private final long ttl; + private final long expireTime; + private final boolean preload; + private final AffinityTopologyVersion topVer; + private final GridDrType drType; + private final boolean fromStore; + + public InitialValue(GridCacheMapEntry entry, CacheObject val, + GridCacheVersion ver, MvccVersion mvccVer, MvccVersion newMvccVer, long ttl, long expireTime, + boolean preload, + AffinityTopologyVersion topVer, GridDrType drType, boolean fromStore) { + this.entry = entry; + this.val = val; + this.ver = ver; + this.mvccVer = mvccVer; + this.newMvccVer = newMvccVer; + this.ttl = ttl; + this.expireTime = expireTime; + this.preload = preload; + this.topVer = topVer; + this.drType = drType; + this.fromStore = fromStore; + } + } + + /** */ + public CacheEntryInitialValuesBatch add( GridCacheMapEntry entry, CacheObject val, GridCacheVersion ver, @@ -67,9 +95,49 @@ public boolean initialValue( boolean preload, AffinityTopologyVersion topVer, GridDrType drType, - boolean fromStore - ) throws IgniteCheckedException, GridCacheEntryRemovedException { - ensureFreeSpace(entry); + boolean fromStore) { + + initialValues.add(new InitialValue(entry, + val, + ver, + mvccVer, + newMvccVer, + ttl, + expireTime, + preload, + topVer, + drType, + fromStore)); + + return this; + } + + /** */ + public int initValues() throws IgniteCheckedException, GridCacheEntryRemovedException { + int initCnt = 0; + + for (InitialValue val : initialValues) { + if (initialValue(val)) + ++initCnt; + } + + return initCnt; + } + + private boolean initialValue(InitialValue iv) throws IgniteCheckedException, GridCacheEntryRemovedException { + GridCacheMapEntry entry = iv.entry; + CacheObject val = iv.val; + GridCacheVersion ver = iv.ver; + MvccVersion mvccVer = iv.mvccVer; + MvccVersion newMvccVer = iv.newMvccVer; + long ttl = iv.ttl; + long expireTime = iv.expireTime; + boolean preload = iv.preload; + AffinityTopologyVersion topVer = iv.topVer; + GridDrType drType = iv.drType; + boolean fromStore = iv.fromStore; + + entry.ensureFreeSpace(); boolean deferred = false; boolean obsolete = false; @@ -101,7 +169,7 @@ public boolean initialValue( long oldExpTime = entry.expireTimeUnlocked(); if (oldExpTime > 0 && oldExpTime < U.currentTimeMillis()) { - if (onExpired(entry, entry.val, null)) { + if (entry.onExpired(entry.val, null)) { if (cctx.deferredDelete()) { deferred = true; oldVer = entry.ver; @@ -122,7 +190,7 @@ else if (val == null) } if (!cctx.mvccEnabled() && (!unswapped || (unswapped && p == null))) { - boolean update0 = storeValue(entry, val, expTime, ver, p); + boolean update0 = entry.storeValue(val, expTime, ver, p); if (p != null) update = update0; @@ -230,7 +298,7 @@ else if (entry.deletedUnlocked()) } } - drReplicate(entry, drType, val, ver, topVer); + entry.drReplicate(drType, val, ver, topVer); if (!skipQryNtf) { cctx.continuousQueries().onEntryUpdated( @@ -254,133 +322,6 @@ else if (entry.deletedUnlocked()) } } - /** - * Stores value in off-heap. - * - * @param val Value. - * @param expireTime Expire time. - * @param ver New entry version. - * @param predicate Optional predicate. - * @return {@code True} if storage was modified. - * @throws IgniteCheckedException If update failed. - */ - protected boolean storeValue( - GridCacheMapEntry entry, - @Nullable CacheObject val, - long expireTime, - GridCacheVersion ver, - @Nullable IgnitePredicate predicate) throws IgniteCheckedException { - assert entry.lock.isHeldByCurrentThread(); - - GridCacheMapEntry.UpdateClosure closure = new GridCacheMapEntry.UpdateClosure(entry, val, ver, expireTime, predicate); - - cctx.offheap().invoke(cctx, entry.key(), entry.localPartition(), closure); - - return closure.treeOp != IgniteTree.OperationType.NOOP; - } - - /** - * Perform DR if needed. - * - * @param drType DR type. - * @param val Value. - * @param ver Version. - * @param topVer Topology version. - * @throws IgniteCheckedException In case of exception. - */ - private void drReplicate(GridCacheMapEntry entry, GridDrType drType, @Nullable CacheObject val, GridCacheVersion ver, AffinityTopologyVersion topVer) - throws IgniteCheckedException { - if (cctx.isDrEnabled() && drType != DR_NONE && !entry.isInternal()) - cctx.dr().replicate(entry.key, val, entry.rawTtl(), entry.rawExpireTime(), ver.conflictVersion(), drType, topVer); - } - - /** - * Evicts necessary number of data pages if per-page eviction is configured in current {@link DataRegion}. - */ - private void ensureFreeSpace(GridCacheMapEntry entry) throws IgniteCheckedException { - // Deadlock alert: evicting data page causes removing (and locking) all entries on the page one by one. - assert !entry.lock.isHeldByCurrentThread(); - - cctx.shared().database().ensureFreeSpace(cctx.dataRegion()); - } - - /** - * @param expiredVal Expired value. - * @param obsoleteVer Version. - * @return {@code True} if entry was marked as removed. - * @throws IgniteCheckedException If failed. - */ - private boolean onExpired(GridCacheMapEntry entry, CacheObject expiredVal, GridCacheVersion obsoleteVer) throws IgniteCheckedException { - assert expiredVal != null; - - boolean rmvd = false; - - if (entry.mvccExtras() != null) - return false; - - if (cctx.deferredDelete() && !entry.detached() && !entry.isInternal()) { - if (!entry.deletedUnlocked() && !entry.isStartVersion()) { - entry.update(null, 0L, 0L, entry.ver, true); - - entry.deletedUnlocked(true); - - rmvd = true; - } - } - else { - if (obsoleteVer == null) - obsoleteVer = cctx.versions().next(entry.ver); - - if (entry.markObsolete0(obsoleteVer, true, null)) - rmvd = true; - } - -// if (log.isTraceEnabled()) -// log.trace("onExpired clear [key=" + key + ", entry=" + System.identityHashCode(this) + ']'); - - cctx.shared().database().checkpointReadLock(); - - try { - if (cctx.mvccEnabled()) - cctx.offheap().mvccRemoveAll(entry); - else - removeValue(entry); - } - finally { - cctx.shared().database().checkpointReadUnlock(); - } - - if (cctx.events().isRecordable(EVT_CACHE_OBJECT_EXPIRED)) { - cctx.events().addEvent(entry.partition(), - entry.key, - cctx.localNodeId(), - null, - EVT_CACHE_OBJECT_EXPIRED, - null, - false, - expiredVal, - expiredVal != null, - null, - null, - null, - true); - } - - cctx.continuousQueries().onEntryExpired(entry, entry.key, expiredVal); - - return rmvd; - } - - /** - * Removes value from offheap. - * - * @throws IgniteCheckedException If failed. - */ - protected void removeValue(GridCacheMapEntry entry) throws IgniteCheckedException { - assert entry.lock.isHeldByCurrentThread(); - - cctx.offheap().remove(cctx, entry.key, entry.partition(), entry.localPartition()); - } /** */ protected static class InitialValuePredicate implements IgnitePredicate { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java index ca8fd2d5e0706..8e7788b8da528 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMapEntries.java @@ -121,7 +121,7 @@ public Collection> initialValue GridCacheEntryInfoEx infoEx = (GridCacheEntryInfoEx)info; IgnitePredicate p = - new GridCacheEntryProcessor.InitialValuePredicate(infoEx.cacheEntry, info.version(), preload); + new CacheEntryInitialValuesBatch.InitialValuePredicate(infoEx.cacheEntry, info.version(), preload); return infoEx.update = p.apply(row); } @@ -232,15 +232,15 @@ private void unlockEntries( continue; } - - try { - entry.proc.finishInitialUpdate(entry, info.value(), info.expireTime(), info.ttl(), info.version(), topVer, - drType, null, preload, false); - } catch (IgniteCheckedException ex) { - cctx.logger(getClass()).error("Unable to finish initial update, skip " + key, ex); - - skippedKeys.add(key); - } +// todo +// try { +// entry.proc.finishInitialUpdate(entry, info.value(), info.expireTime(), info.ttl(), info.version(), topVer, +// drType, null, preload, false); +// } catch (IgniteCheckedException ex) { +// cctx.logger(getClass()).error("Unable to finish initial update, skip " + key, ex); +// +// skippedKeys.add(key); +// } } } finally { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 1907e23a645fb..9b4f46004853c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -231,8 +231,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme @GridToStringExclude private final ReadWriteLock listenerLock; - protected final GridCacheEntryProcessor proc; - /** * Flags: *
      @@ -256,8 +254,6 @@ protected GridCacheMapEntry( key = (KeyCacheObject)cctx.kernalContext().cacheObjects().prepareForCache(key, cctx); - proc = new GridCacheEntryProcessor(cctx); - assert key != null; this.key = key; @@ -2627,7 +2623,7 @@ private GridTuple3 ttlAndExpireTime(IgniteCacheExpiryPolicy * @param topVer Topology version. * @throws IgniteCheckedException In case of exception. */ - private void drReplicate(GridDrType drType, @Nullable CacheObject val, GridCacheVersion ver, AffinityTopologyVersion topVer) + protected void drReplicate(GridDrType drType, @Nullable CacheObject val, GridCacheVersion ver, AffinityTopologyVersion topVer) throws IgniteCheckedException { if (cctx.isDrEnabled() && drType != DR_NONE && !isInternal()) cctx.dr().replicate(key, val, rawTtl(), rawExpireTime(), ver.conflictVersion(), drType, topVer); @@ -3324,7 +3320,8 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { GridDrType drType, boolean fromStore ) throws IgniteCheckedException, GridCacheEntryRemovedException { - return proc.initialValue(this, val, + + return new CacheEntryInitialValuesBatch(cctx).add(this, val, ver, mvccVer, newMvccVer, @@ -3333,7 +3330,7 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { preload, topVer, drType, - fromStore); + fromStore).initValues() == 1; } /** @@ -3864,7 +3861,7 @@ private GridCacheVersion nextVersion() { * @return {@code True} if entry was marked as removed. * @throws IgniteCheckedException If failed. */ - private boolean onExpired(CacheObject expiredVal, GridCacheVersion obsoleteVer) throws IgniteCheckedException { + protected boolean onExpired(CacheObject expiredVal, GridCacheVersion obsoleteVer) throws IgniteCheckedException { assert expiredVal != null; boolean rmvd = false; @@ -4312,7 +4309,7 @@ protected void removeValue() throws IgniteCheckedException { /** * Evicts necessary number of data pages if per-page eviction is configured in current {@link DataRegion}. */ - private void ensureFreeSpace() throws IgniteCheckedException { + protected void ensureFreeSpace() throws IgniteCheckedException { // Deadlock alert: evicting data page causes removing (and locking) all entries on the page one by one. assert !lock.isHeldByCurrentThread(); From 5e4bd188bbfcb496771902c50da4ce46befacf6e Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Fri, 12 Apr 2019 15:01:19 +0300 Subject: [PATCH 21/24] w --- .../cache/CacheEntryInitialValuesBatch.java | 94 +++++++++++++------ 1 file changed, 63 insertions(+), 31 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java index 93b4ad51b034b..e8a5596d5f99c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java @@ -65,6 +65,9 @@ class InitialValue { private final GridDrType drType; private final boolean fromStore; + private Runnable unlockCb; +// private boolean obsolete; + public InitialValue(GridCacheMapEntry entry, CacheObject val, GridCacheVersion ver, MvccVersion mvccVer, MvccVersion newMvccVer, long ttl, long expireTime, boolean preload, @@ -116,14 +119,49 @@ public CacheEntryInitialValuesBatch add( public int initValues() throws IgniteCheckedException, GridCacheEntryRemovedException { int initCnt = 0; - for (InitialValue val : initialValues) { - if (initialValue(val)) - ++initCnt; + cctx.shared().database().ensureFreeSpace(cctx.dataRegion()); + + cctx.group().listenerLock().readLock().lock(); + + lockEntries(); + + try { + for (InitialValue val : initialValues) { + val.entry.lockEntry(); + + try { + if (initialValue(val)) + ++initCnt; + } finally { + val.entry.unlockEntry(); + } + } + } finally { + unlockEntries(); + + cctx.group().listenerLock().readLock().unlock(); } return initCnt; } + /** */ + private void unlockEntries() { + for (InitialValue val : initialValues) { + val.entry.unlockEntry(); + + if (val.unlockCb != null) + val.unlockCb.run(); + } + } + + /** */ + private void lockEntries() { + // todo improve by copy-pasting from atomic cache + for (InitialValue val : initialValues) + val.entry.lockEntry(); + } + private boolean initialValue(InitialValue iv) throws IgniteCheckedException, GridCacheEntryRemovedException { GridCacheMapEntry entry = iv.entry; CacheObject val = iv.val; @@ -136,18 +174,18 @@ private boolean initialValue(InitialValue iv) throws IgniteCheckedException, Gri AffinityTopologyVersion topVer = iv.topVer; GridDrType drType = iv.drType; boolean fromStore = iv.fromStore; - - entry.ensureFreeSpace(); - boolean deferred = false; - boolean obsolete = false; +// entry.ensureFreeSpace(); + +// boolean deferred = false; +// boolean obsolete = false; GridCacheVersion oldVer = null; - entry.lockListenerReadLock(); - entry.lockEntry(); +// entry.lockListenerReadLock(); +// entry.lockEntry(); - try { +// try { entry.checkObsolete(); long expTime = expireTime < 0 ? CU.toExpireTime(ttl) : expireTime; @@ -171,11 +209,17 @@ private boolean initialValue(InitialValue iv) throws IgniteCheckedException, Gri if (oldExpTime > 0 && oldExpTime < U.currentTimeMillis()) { if (entry.onExpired(entry.val, null)) { if (cctx.deferredDelete()) { - deferred = true; - oldVer = entry.ver; + iv.unlockCb = () -> { + iv.entry.onMarkedObsolete(); + + cctx.cache().removeEntry(iv.entry); + }; + } + else if (val == null) { + assert oldVer != null; + + iv.unlockCb = () -> cctx.onDeferredDelete(iv.entry, oldVer); } - else if (val == null) - obsolete = true; } } @@ -203,25 +247,13 @@ else if (val == null) } return false; - } - finally { - entry.unlockEntry(); - entry.unlockListenerReadLock(); - - // It is necessary to execute these callbacks outside of lock to avoid deadlocks. +// } +// finally { +// entry.unlockEntry(); +// entry.unlockListenerReadLock(); - if (obsolete) { - entry.onMarkedObsolete(); - - cctx.cache().removeEntry(entry); - } - if (deferred) { - assert oldVer != null; - - cctx.onDeferredDelete(entry, oldVer); - } - } +// } } /** From e2843e39948126666295d46ff8f1353a4b3a9740 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Fri, 12 Apr 2019 15:52:55 +0300 Subject: [PATCH 22/24] wip --- .../cache/CacheEntryInitialValuesBatch.java | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java index e8a5596d5f99c..b479ca403516a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java @@ -98,8 +98,8 @@ public CacheEntryInitialValuesBatch add( boolean preload, AffinityTopologyVersion topVer, GridDrType drType, - boolean fromStore) { - + boolean fromStore + ) { initialValues.add(new InitialValue(entry, val, ver, @@ -180,7 +180,7 @@ private boolean initialValue(InitialValue iv) throws IgniteCheckedException, Gri // boolean deferred = false; // boolean obsolete = false; - GridCacheVersion oldVer = null; +// GridCacheVersion oldVer = null; // entry.lockListenerReadLock(); // entry.lockEntry(); @@ -209,31 +209,31 @@ private boolean initialValue(InitialValue iv) throws IgniteCheckedException, Gri if (oldExpTime > 0 && oldExpTime < U.currentTimeMillis()) { if (entry.onExpired(entry.val, null)) { if (cctx.deferredDelete()) { - iv.unlockCb = () -> { - iv.entry.onMarkedObsolete(); + final GridCacheVersion oldVer = entry.ver; - cctx.cache().removeEntry(iv.entry); - }; + iv.unlockCb = () -> cctx.onDeferredDelete(entry, oldVer); } else if (val == null) { - assert oldVer != null; + iv.unlockCb = () -> { + entry.onMarkedObsolete(); - iv.unlockCb = () -> cctx.onDeferredDelete(iv.entry, oldVer); + cctx.cache().removeEntry(entry); + }; } } } - if (cctx.mvccEnabled()) { - assert !preload; - - cctx.offheap().mvccInitialValue(entry, val, ver, expTime, mvccVer, newMvccVer); - } - else - p = null; +// if (!cctx.mvccEnabled()) + p = null; } } - if (!cctx.mvccEnabled() && (!unswapped || (unswapped && p == null))) { + if (cctx.mvccEnabled()) { + assert !preload; + + cctx.offheap().mvccInitialValue(entry, val, ver, expTime, mvccVer, newMvccVer); + } + else { // !cctx.mvccEnabled() && (!unswapped || (unswapped && p == null)) boolean update0 = entry.storeValue(val, expTime, ver, p); if (p != null) From 67be3fb33cc12e60615553be8ee30b4d26bf9136 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Fri, 12 Apr 2019 17:00:19 +0300 Subject: [PATCH 23/24] mega wip --- .../cache/CacheEntryInitialValuesBatch.java | 216 ++++++++++++------ .../processors/cache/GridCacheMapEntry.java | 2 +- .../preloader/GridDhtPartitionDemander.java | 4 +- 3 files changed, 155 insertions(+), 67 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java index b479ca403516a..404c9e338af12 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java @@ -18,13 +18,16 @@ package org.apache.ignite.internal.processors.cache; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; import org.apache.ignite.internal.pagemem.wal.record.DataRecord; import org.apache.ignite.internal.pagemem.wal.record.MvccDataEntry; import org.apache.ignite.internal.pagemem.wal.record.MvccDataRecord; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; @@ -32,6 +35,7 @@ import org.apache.ignite.internal.processors.dr.GridDrType; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; @@ -45,40 +49,65 @@ public class CacheEntryInitialValuesBatch { private final GridCacheContext cctx; /** */ - public CacheEntryInitialValuesBatch(GridCacheContext cctx) { + private final GridDhtLocalPartition part; + + /** */ + public CacheEntryInitialValuesBatch(GridCacheContext cctx, int partId) { this.cctx = cctx; + + part = cctx.topology().localPartition(partId, + cctx.topology().readyTopologyVersion(), true, true); } /** */ private List initialValues = new ArrayList<>(1); - class InitialValue { + class InitialValue extends GridCacheMvccEntryInfo { private final GridCacheMapEntry entry; - private final CacheObject val; - private final GridCacheVersion ver; - private final MvccVersion mvccVer; - private final MvccVersion newMvccVer; - private final long ttl; - private final long expireTime; +// private CacheObject val; +// private final GridCacheVersion ver; +// private final MvccVersion mvccVer; +// private final MvccVersion newMvccVer; +// private final long ttl; +// private final long expireTime; private final boolean preload; private final AffinityTopologyVersion topVer; private final GridDrType drType; private final boolean fromStore; private Runnable unlockCb; + private boolean update; // private boolean obsolete; - - public InitialValue(GridCacheMapEntry entry, CacheObject val, - GridCacheVersion ver, MvccVersion mvccVer, MvccVersion newMvccVer, long ttl, long expireTime, + private long expTime; + private IgnitePredicate p; + + public InitialValue( + GridCacheMapEntry entry, + CacheObject val, + GridCacheVersion ver, + MvccVersion mvccVer, + MvccVersion newMvccVer, + long ttl, + long expireTime, boolean preload, - AffinityTopologyVersion topVer, GridDrType drType, boolean fromStore) { + AffinityTopologyVersion topVer, + GridDrType drType, + boolean fromStore + ) { this.entry = entry; - this.val = val; - this.ver = ver; - this.mvccVer = mvccVer; - this.newMvccVer = newMvccVer; - this.ttl = ttl; - this.expireTime = expireTime; + + key(entry.key); + value(val); + version(ver); + ttl(ttl); + expireTime(expireTime); + + if (mvccVer != null) + mvccVersion(mvccVer); + + if (newMvccVer != null) + newMvccVersion(newMvccVer); + this.preload = preload; this.topVer = topVer; this.drType = drType; @@ -116,26 +145,110 @@ public CacheEntryInitialValuesBatch add( } /** */ - public int initValues() throws IgniteCheckedException, GridCacheEntryRemovedException { + public int initValues() throws IgniteCheckedException { int initCnt = 0; cctx.shared().database().ensureFreeSpace(cctx.dataRegion()); cctx.group().listenerLock().readLock().lock(); - lockEntries(); +// lockEntries(); + + boolean mvcc = cctx.mvccEnabled(); try { - for (InitialValue val : initialValues) { - val.entry.lockEntry(); + Set skipped = new HashSet<>(); + + // lock stage + for (InitialValue v : initialValues) { + v.entry.lockEntry(); + + v.expTime = v.expireTime() < 0 ? CU.toExpireTime(v.ttl()) : v.expireTime(); try { - if (initialValue(val)) - ++initCnt; - } finally { - val.entry.unlockEntry(); + if (!(v.update = prepareInitialValue(v))) + v.p = new InitialValuePredicate(v.entry, v.version(), v.preload); + + if (mvcc) { + assert !v.preload; + + cctx.offheap().mvccInitialValue(v.entry, v.value(), v.version(), v.expTime, v.mvccVersion(), v.newMvccVersion()); + } + } + catch (GridCacheEntryRemovedException e) { + skipped.add(v.entry.key); + } + } + + if (!mvcc) { + List infos = new ArrayList<>(initialValues.size()); + + for (InitialValue v : initialValues) { +// GridCacheEntryInfo info = new GridCacheEntryInfo(); +// +// info.version(v.ver); +// info.value(v.val); +// info.key(v.entry.key); +// info.ttl(v.ttl); +// info.expireTime(v.expTime); + if (!skipped.contains(v.entry.key)) + infos.add(v); + } + +// IgniteBiPredicate pred = update ? +// new IgniteBiPredicate() { +// @Override public boolean apply(CacheDataRow row, GridCacheEntryInfo info) { +// InitialValue infoEx = (InitialValue)info; +// +// IgnitePredicate p = +// new InitialValuePredicate(infoEx.entry, info.version(), infoEx.preload); +// +// return infoEx.update = p.apply(row); +// } +// }; + + cctx.offheap().updateAll(cctx, part, infos, (r, i) -> { + InitialValue iv = ((InitialValue)i); + + if (iv.p != null) + iv.update = iv.p.apply(r); + + return iv.p == null || iv.update; }); + + } + + for (InitialValue v : initialValues) { + if (v.update) { + finishInitialUpdate(v.entry, v.value(), v.expireTime(), v.ttl(), v.version(), v.topVer, v.drType, v.mvccVersion(), v.preload, v.fromStore); + + ++initCnt; } } + + + +// try { +// boolean update = prepareInitialValue(v); +// +// IgnitePredicate p = update ? null : new InitialValuePredicate(v.entry, v.ver, v.preload); +// +// +// if (!mvcc) { // !cctx.mvccEnabled() && (!unswapped || (unswapped && p == null)) +// boolean update0 = v.entry.storeValue(v.val, expTime, v.ver, p); +// +// if (p != null) +// update = update0; +// } +// +// if (update) { +// finishInitialUpdate(v.entry, v.val, v.expireTime, v.ttl, v.ver, v.topVer, v.drType, v.mvccVer, v.preload, v.fromStore); +// +// ++initCnt; +// } +// } finally { +// v.entry.unlockEntry(); +// } +// } } finally { unlockEntries(); @@ -156,24 +269,17 @@ private void unlockEntries() { } /** */ - private void lockEntries() { - // todo improve by copy-pasting from atomic cache - for (InitialValue val : initialValues) - val.entry.lockEntry(); - } +// private void lockEntries() { +// // todo improve by copy-pasting from atomic cache +// for (InitialValue val : initialValues) +// val.entry.lockEntry(); +// } - private boolean initialValue(InitialValue iv) throws IgniteCheckedException, GridCacheEntryRemovedException { + private boolean prepareInitialValue(InitialValue iv) throws IgniteCheckedException, GridCacheEntryRemovedException { GridCacheMapEntry entry = iv.entry; - CacheObject val = iv.val; - GridCacheVersion ver = iv.ver; - MvccVersion mvccVer = iv.mvccVer; - MvccVersion newMvccVer = iv.newMvccVer; - long ttl = iv.ttl; - long expireTime = iv.expireTime; + CacheObject val = iv.value(); + GridCacheVersion ver = iv.version(); boolean preload = iv.preload; - AffinityTopologyVersion topVer = iv.topVer; - GridDrType drType = iv.drType; - boolean fromStore = iv.fromStore; // entry.ensureFreeSpace(); @@ -188,21 +294,19 @@ private boolean initialValue(InitialValue iv) throws IgniteCheckedException, Gri // try { entry.checkObsolete(); - long expTime = expireTime < 0 ? CU.toExpireTime(ttl) : expireTime; - val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx); + + iv.value(cctx.kernalContext().cacheObjects().prepareForCache(val, cctx)); final boolean unswapped = ((entry.flags & IS_UNSWAPPED_MASK) != 0); boolean update = false; - IgnitePredicate p = new InitialValuePredicate(entry, ver, preload); - if (unswapped || cctx.mvccEnabled()) { if (!unswapped) entry.unswap(false); - if (update = p.apply(null)) { + if (update = new InitialValuePredicate(entry, ver, preload).apply(null)) { // If entry is already unswapped and we are modifying it, we must run deletion callbacks for old value. long oldExpTime = entry.expireTimeUnlocked(); @@ -224,29 +328,13 @@ else if (val == null) { } // if (!cctx.mvccEnabled()) - p = null; +// p = null; } } - if (cctx.mvccEnabled()) { - assert !preload; - - cctx.offheap().mvccInitialValue(entry, val, ver, expTime, mvccVer, newMvccVer); - } - else { // !cctx.mvccEnabled() && (!unswapped || (unswapped && p == null)) - boolean update0 = entry.storeValue(val, expTime, ver, p); - - if (p != null) - update = update0; - } - if (update) { - finishInitialUpdate(entry, val, expireTime, ttl, ver, topVer, drType, mvccVer, preload, fromStore); - - return true; - } - return false; + return update; // } // finally { // entry.unlockEntry(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 9b4f46004853c..ca72e2c43257a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -3321,7 +3321,7 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { boolean fromStore ) throws IgniteCheckedException, GridCacheEntryRemovedException { - return new CacheEntryInitialValuesBatch(cctx).add(this, val, + return new CacheEntryInitialValuesBatch(cctx, partition()).add(this, val, ver, mvccVer, newMvccVer, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index 079e4eb39e800..36de8d5c813de 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -95,8 +95,8 @@ public class GridDhtPartitionDemander { private static final int CHECKPOINT_THRESHOLD = 100; /** */ - private static final boolean BATCH_PAGE_WRITE_ENABLED = - IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, false); + private static final boolean BATCH_PAGE_WRITE_ENABLED = false; + //IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, true); /** */ private final GridCacheSharedContext ctx; From 7902db25f886fce19f8e4b032263c0691baf19f4 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Fri, 12 Apr 2019 19:37:10 +0300 Subject: [PATCH 24/24] draft ver --- .../cache/CacheEntryInitialValuesBatch.java | 200 ++++++------------ .../preloader/GridDhtPartitionDemander.java | 33 ++- 2 files changed, 91 insertions(+), 142 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java index 404c9e338af12..bbe67006266f8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInitialValuesBatch.java @@ -35,7 +35,6 @@ import org.apache.ignite.internal.processors.dr.GridDrType; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; @@ -62,12 +61,14 @@ public CacheEntryInitialValuesBatch(GridCacheContext cctx, int partId) { /** */ private List initialValues = new ArrayList<>(1); - class InitialValue extends GridCacheMvccEntryInfo { + /** */ + class InitialValue extends GridCacheEntryInfo { private final GridCacheMapEntry entry; // private CacheObject val; // private final GridCacheVersion ver; -// private final MvccVersion mvccVer; -// private final MvccVersion newMvccVer; + // todo unable to use version aware + private final MvccVersion mvccVer; + private final MvccVersion newMvccVer; // private final long ttl; // private final long expireTime; private final boolean preload; @@ -102,11 +103,9 @@ public InitialValue( ttl(ttl); expireTime(expireTime); - if (mvccVer != null) - mvccVersion(mvccVer); + this.mvccVer = mvccVer; - if (newMvccVer != null) - newMvccVersion(newMvccVer); + this.newMvccVer = newMvccVer; this.preload = preload; this.topVer = topVer; @@ -152,15 +151,15 @@ public int initValues() throws IgniteCheckedException { cctx.group().listenerLock().readLock().lock(); -// lockEntries(); - boolean mvcc = cctx.mvccEnabled(); try { Set skipped = new HashSet<>(); // lock stage - for (InitialValue v : initialValues) { + for (int i = 0; i < initialValues.size(); i++) { + InitialValue v = initialValues.get(i); + v.entry.lockEntry(); v.expTime = v.expireTime() < 0 ? CU.toExpireTime(v.ttl()) : v.expireTime(); @@ -168,11 +167,11 @@ public int initValues() throws IgniteCheckedException { try { if (!(v.update = prepareInitialValue(v))) v.p = new InitialValuePredicate(v.entry, v.version(), v.preload); - + else if (mvcc) { assert !v.preload; - cctx.offheap().mvccInitialValue(v.entry, v.value(), v.version(), v.expTime, v.mvccVersion(), v.newMvccVersion()); + cctx.offheap().mvccInitialValue(v.entry, v.value(), v.version(), v.expTime, v.mvccVer, v.newMvccVer); } } catch (GridCacheEntryRemovedException e) { @@ -181,76 +180,52 @@ public int initValues() throws IgniteCheckedException { } if (!mvcc) { - List infos = new ArrayList<>(initialValues.size()); - - for (InitialValue v : initialValues) { -// GridCacheEntryInfo info = new GridCacheEntryInfo(); -// -// info.version(v.ver); -// info.value(v.val); -// info.key(v.entry.key); -// info.ttl(v.ttl); -// info.expireTime(v.expTime); - if (!skipped.contains(v.entry.key)) - infos.add(v); + if (initialValues.size() == 1) { + InitialValue v = initialValues.get(0); + + boolean update0 = v.entry.storeValue(v.value(), v.expTime, v.version(), v.p); + + if (update0 && v.p != null) + v.update = update0; } + else { + List infos = new ArrayList<>(initialValues.size()); -// IgniteBiPredicate pred = update ? -// new IgniteBiPredicate() { -// @Override public boolean apply(CacheDataRow row, GridCacheEntryInfo info) { -// InitialValue infoEx = (InitialValue)info; -// -// IgnitePredicate p = -// new InitialValuePredicate(infoEx.entry, info.version(), infoEx.preload); -// -// return infoEx.update = p.apply(row); -// } -// }; + for (int i = 0; i < initialValues.size(); i++) { + InitialValue v = initialValues.get(i); + + if (!skipped.contains(v.entry.key)) + infos.add(v); + } - cctx.offheap().updateAll(cctx, part, infos, (r, i) -> { - InitialValue iv = ((InitialValue)i); + cctx.offheap().updateAll(cctx, part, infos, (r, i) -> { + InitialValue iv = ((InitialValue)i); - if (iv.p != null) - iv.update = iv.p.apply(r); + if (iv.p != null) + iv.update = iv.p.apply(r); - return iv.p == null || iv.update; }); + return iv.p == null || iv.update; + }); + } } - for (InitialValue v : initialValues) { + for (int i = 0; i < initialValues.size(); i++) { + InitialValue v = initialValues.get(i); + if (v.update) { - finishInitialUpdate(v.entry, v.value(), v.expireTime(), v.ttl(), v.version(), v.topVer, v.drType, v.mvccVersion(), v.preload, v.fromStore); + finishInitialUpdate(v.entry, v.value(), v.expireTime(), v.ttl(), v.version(), v.topVer, v.drType, v.mvccVer, v.preload, v.fromStore); ++initCnt; } } - - - -// try { -// boolean update = prepareInitialValue(v); -// -// IgnitePredicate p = update ? null : new InitialValuePredicate(v.entry, v.ver, v.preload); -// -// -// if (!mvcc) { // !cctx.mvccEnabled() && (!unswapped || (unswapped && p == null)) -// boolean update0 = v.entry.storeValue(v.val, expTime, v.ver, p); -// -// if (p != null) -// update = update0; -// } -// -// if (update) { -// finishInitialUpdate(v.entry, v.val, v.expireTime, v.ttl, v.ver, v.topVer, v.drType, v.mvccVer, v.preload, v.fromStore); -// -// ++initCnt; -// } -// } finally { -// v.entry.unlockEntry(); -// } -// } } finally { - unlockEntries(); + for (InitialValue val : initialValues) { + val.entry.unlockEntry(); + + if (val.unlockCb != null) + val.unlockCb.run(); + } cctx.group().listenerLock().readLock().unlock(); } @@ -259,89 +234,48 @@ public int initValues() throws IgniteCheckedException { } /** */ - private void unlockEntries() { - for (InitialValue val : initialValues) { - val.entry.unlockEntry(); - - if (val.unlockCb != null) - val.unlockCb.run(); - } - } - - /** */ -// private void lockEntries() { -// // todo improve by copy-pasting from atomic cache -// for (InitialValue val : initialValues) -// val.entry.lockEntry(); -// } - private boolean prepareInitialValue(InitialValue iv) throws IgniteCheckedException, GridCacheEntryRemovedException { GridCacheMapEntry entry = iv.entry; CacheObject val = iv.value(); GridCacheVersion ver = iv.version(); boolean preload = iv.preload; -// entry.ensureFreeSpace(); - -// boolean deferred = false; -// boolean obsolete = false; - -// GridCacheVersion oldVer = null; - -// entry.lockListenerReadLock(); -// entry.lockEntry(); - -// try { - entry.checkObsolete(); - + entry.checkObsolete(); + iv.value(cctx.kernalContext().cacheObjects().prepareForCache(val, cctx)); - iv.value(cctx.kernalContext().cacheObjects().prepareForCache(val, cctx)); + final boolean unswapped = ((entry.flags & IS_UNSWAPPED_MASK) != 0); - final boolean unswapped = ((entry.flags & IS_UNSWAPPED_MASK) != 0); + boolean update = false; - boolean update = false; + if (unswapped || cctx.mvccEnabled()) { + if (!unswapped) + entry.unswap(false); - if (unswapped || cctx.mvccEnabled()) { - if (!unswapped) - entry.unswap(false); + if (update = new InitialValuePredicate(entry, ver, preload).apply(null)) { + // If entry is already unswapped and we are modifying it, we must run deletion callbacks for old value. + long oldExpTime = entry.expireTimeUnlocked(); - if (update = new InitialValuePredicate(entry, ver, preload).apply(null)) { - // If entry is already unswapped and we are modifying it, we must run deletion callbacks for old value. - long oldExpTime = entry.expireTimeUnlocked(); + if (oldExpTime > 0 && oldExpTime < U.currentTimeMillis()) { + if (entry.onExpired(entry.val, null)) { + if (cctx.deferredDelete()) { + final GridCacheVersion oldVer = entry.ver; - if (oldExpTime > 0 && oldExpTime < U.currentTimeMillis()) { - if (entry.onExpired(entry.val, null)) { - if (cctx.deferredDelete()) { - final GridCacheVersion oldVer = entry.ver; - - iv.unlockCb = () -> cctx.onDeferredDelete(entry, oldVer); - } - else if (val == null) { - iv.unlockCb = () -> { - entry.onMarkedObsolete(); + iv.unlockCb = () -> cctx.onDeferredDelete(entry, oldVer); + } + else if (val == null) { + iv.unlockCb = () -> { + entry.onMarkedObsolete(); - cctx.cache().removeEntry(entry); - }; - } + cctx.cache().removeEntry(entry); + }; } } - -// if (!cctx.mvccEnabled()) -// p = null; } } + } - - - return update; -// } -// finally { -// entry.unlockEntry(); -// entry.unlockListenerReadLock(); - - -// } + return update; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index 36de8d5c813de..75070a6a0c7fa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -45,8 +45,8 @@ import org.apache.ignite.internal.processors.affinity.AffinityAssignment; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection; +import org.apache.ignite.internal.processors.cache.CacheEntryInitialValuesBatch; import org.apache.ignite.internal.processors.cache.CacheGroupContext; -import org.apache.ignite.internal.processors.cache.CacheMapEntries; import org.apache.ignite.internal.processors.cache.CacheMetricsImpl; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; @@ -70,6 +70,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; @@ -95,8 +96,8 @@ public class GridDhtPartitionDemander { private static final int CHECKPOINT_THRESHOLD = 100; /** */ - private static final boolean BATCH_PAGE_WRITE_ENABLED = false; - //IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, true); + private static final boolean BATCH_PAGE_WRITE_ENABLED = + IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, true); /** */ private final GridCacheSharedContext ctx; @@ -897,7 +898,8 @@ private void preloadEntriesBatch(ClusterNode from, cctxs.computeIfAbsent(e.cacheId(), v -> new ArrayList<>(8)).add(e); } - CacheMapEntries cacheEntries = new CacheMapEntries(); + //CacheMapEntries cacheEntries = new CacheMapEntries(); + for (Map.Entry> cctxEntry : cctxs.entrySet()) { GridCacheContext cctx = grp.sharedGroup() ? ctx.cacheContext(cctxEntry.getKey()) : grp.singleCacheContext(); @@ -910,12 +912,25 @@ private void preloadEntriesBatch(ClusterNode from, if (cctx.isNear()) cctx = cctx.dhtCache().context(); + CacheEntryInitialValuesBatch cacheEntries = new CacheEntryInitialValuesBatch(cctx, p); + + List> res = new ArrayList<>(cctxInfos.size()); + + for (GridCacheEntryInfo info : cctxInfos) { + GridCacheMapEntry cached = (GridCacheMapEntry)cctx.cache().entryEx(info.key(), topVer); + + cacheEntries.add(cached, info.value(), info.version(), null, null, info.ttl(), info.expireTime(), true, topVer, DR_PRELOAD, false); + + res.add(new T2<>(cached, info)); + } + try { - Iterable> cachedEntries = - cacheEntries.initialValues(cctxInfos, topVer, cctx, p, true, DR_PRELOAD); +// Iterable> cachedEntries = +// cacheEntries.initialValues(cctxInfos, topVer, cctx, p, true, DR_PRELOAD); + cacheEntries.initValues(); - for (Map.Entry e : cachedEntries) { - GridCacheMapEntry cached = e.getValue(); + for (Map.Entry e : res) { + GridCacheMapEntry cached = e.getKey(); cached.touch(); @@ -923,7 +938,7 @@ private void preloadEntriesBatch(ClusterNode from, continue; cctx.events().addEvent(p, cached.key(), cctx.localNodeId(), null, null, null, - EVT_CACHE_REBALANCE_OBJECT_LOADED, e.getKey().value(), true, null, + EVT_CACHE_REBALANCE_OBJECT_LOADED, e.getValue().value(), true, null, false, null, null, null, true); } }