From 18bf3e30eef66ef9ed310274a1d4d3a83b051b1a Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 28 Mar 2019 14:05:34 +0300 Subject: [PATCH 1/6] 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 00000000000000..225e702bfea747 --- /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 3097a050c31f25..6fdf2968f7747a 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 00000000000000..a7e583c398daf1 --- /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 8ce21c59de3d12..aa88b2e1bb04e9 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 2428290d38a203..726686cc5232c1 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 @@ -3407,75 +3407,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; } @@ -3502,6 +3434,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 b7e8ec717fc38a..19c6eaa671ef23 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 04443be2cf7d76..3f9d2ccbbfb519 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 61f1e06f9dccbc..852f2347bbc432 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 @@ -30,9 +30,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; @@ -42,6 +44,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; @@ -86,6 +89,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; @@ -766,10 +779,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); @@ -851,6 +864,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. * @@ -942,48 +1060,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(); @@ -1390,7 +1503,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 35dd3c46ee4316..6bbe1316b8ca58 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 ab48540f6dfc54..1c2cbedf114525 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 ee19aed7284b34..f7afa694ac5c07 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 91fd2070cc0481..5b721710bde890 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 60aefb927ce6f1..bd81d757f03ac9 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 e28d421bdf063c..894c1aa64faca0 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 54d9816bcb7e23..10b082a72af28c 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 78752bbfefc848..d047d4714b150e 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 5ab1bf38dbc18c..07467f42c166e0 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 b3c1c69e66319c..1ce8e560886e4b 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 9e854d28f6cb03..728b3d98ec576a 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 00000000000000..539c0c4fc7e7bb --- /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 c02442656e192c17d616a4c5b12c118b2349f4fe Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 28 Mar 2019 14:56:02 +0300 Subject: [PATCH 2/6] 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 3f9d2ccbbfb519..d181bd726f0053 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 1ce8e560886e4b..12f59358441d8b 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 b25a7ea9ccc8767fb9e0862ce0a57ee4d0d3bd2e Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 28 Mar 2019 19:53:21 +0300 Subject: [PATCH 3/6] 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 225e702bfea747..9402a79b70ba71 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 a4b8b422a8ffb36d3a2685c20d1b165495de1ef7 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 28 Mar 2019 20:29:51 +0300 Subject: [PATCH 4/6] 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 852f2347bbc432..ad7a64768c9f24 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 @@ -89,14 +89,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); /** */ @@ -779,10 +776,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); @@ -864,42 +861,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. @@ -907,7 +868,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 @@ -1060,22 +1021,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. @@ -1083,20 +1038,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(); @@ -1500,11 +1480,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); } @@ -1608,3 +1588,4 @@ private void sendRebalanceFinishedEvent() { } } } + From 6ada564dacea22f22f8590eb47e387dc6f9c88ab Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Tue, 2 Apr 2019 14:55:47 +0300 Subject: [PATCH 5/6] 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 9402a79b70ba71..17a9080f1276a1 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 539c0c4fc7e7bb..bbe6f59a7258d3 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 0cbe07ba4cbfbad0f8e9b08d02ce43e480eb2582 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Tue, 2 Apr 2019 19:58:13 +0300 Subject: [PATCH 6/6] IGNITE-11584 Rework. --- .../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 a7e583c398daf1..873e9f4a0b9cdc 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 00000000000000..b7a48834473759 --- /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 19c6eaa671ef23..dd33d91aa120fc 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 d181bd726f0053..7c29dabd77356a 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 ad7a64768c9f24..f0d6842445d344 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,11 +45,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; @@ -870,19 +872,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(); @@ -897,10 +899,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()) @@ -908,21 +908,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 1c2cbedf114525..714690b703400c 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 10b082a72af28c..54d9816bcb7e23 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 12f59358441d8b..b3c1c69e66319c 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 728b3d98ec576a..9e854d28f6cb03 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(); - } - /** * */