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 f58f1aa37b531..0d131b1bae61b 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -1081,6 +1081,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"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java index 691fe373f4e8e..80b971c7f76b5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java @@ -34,6 +34,7 @@ import org.apache.ignite.internal.managers.indexing.GridIndexingManager; import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager; import org.apache.ignite.internal.processors.cache.mvcc.MvccProcessor; +import org.apache.ignite.internal.processors.diag.DiagnosticProcessor; import org.apache.ignite.internal.stat.IoStatisticsManager; import org.apache.ignite.internal.processors.compress.CompressionProcessor; import org.apache.ignite.internal.processors.service.ServiceProcessorAdapter; @@ -467,6 +468,9 @@ public interface GridKernalContext extends Iterable { */ public FailureProcessor failure(); + /** */ + public DiagnosticProcessor diagnostic(); + /** * Print grid kernal memory stats (sizes of internal structures, etc.). * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java index 1219d00b9860a..de21296ae7b4c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java @@ -48,6 +48,7 @@ import org.apache.ignite.internal.managers.failover.GridFailoverManager; import org.apache.ignite.internal.managers.indexing.GridIndexingManager; import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager; +import org.apache.ignite.internal.processors.diag.DiagnosticProcessor; import org.apache.ignite.internal.processors.service.ServiceProcessorAdapter; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor; import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor; @@ -423,6 +424,9 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable /** Failure processor. */ private FailureProcessor failureProc; + /** */ + private DiagnosticProcessor diagProc; + /** Recovery mode flag. Flag is set to {@code false} when discovery manager started. */ private boolean recoveryMode = true; @@ -585,9 +589,10 @@ else if (comp instanceof GridEncryptionManager) * Processors. * ========== */ - else if (comp instanceof FailureProcessor) failureProc = (FailureProcessor)comp; + else if (comp instanceof DiagnosticProcessor) + diagProc = (DiagnosticProcessor)comp; else if (comp instanceof GridTaskProcessor) taskProc = (GridTaskProcessor)comp; else if (comp instanceof GridJobProcessor) @@ -1196,6 +1201,11 @@ void disconnected(boolean disconnected) { return failureProc; } + /** {@inheritDoc} */ + @Override public DiagnosticProcessor diagnostic() { + return diagProc; + } + /** {@inheritDoc} */ @Override public Thread.UncaughtExceptionHandler uncaughtExceptionHandler() { return hnd; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 3a3af8e72e157..2a6680b826218 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -141,6 +141,7 @@ import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor; import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor; import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor; +import org.apache.ignite.internal.processors.diag.DiagnosticProcessor; import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.processors.hadoop.Hadoop; import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter; @@ -967,6 +968,8 @@ public void start( startProcessor(new FailureProcessor(ctx)); + startProcessor(new DiagnosticProcessor(ctx)); + startProcessor(new PoolProcessor(ctx)); // Closure processor should be started before all others diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/BatchedCacheEntries.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/BatchedCacheEntries.java new file mode 100644 index 0000000000000..14d1251a7077c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/BatchedCacheEntries.java @@ -0,0 +1,482 @@ +/* + * 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.LinkedHashMap; +import java.util.List; +import java.util.Set; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtInvalidPartitionException; +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.persistence.CacheSearchRow; +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.T2; +import org.apache.ignite.internal.util.typedef.T3; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheMapEntry.ATOMIC_VER_COMPARATOR; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_TREE_FINISH_UPDATE1; + +/** + * Batch of cache entries to optimize page memory processing. + */ +public class BatchedCacheEntries { + /** */ +// private final int partId; + + /** */ + private final GridDhtLocalPartition part; + + /** */ + private final GridCacheContext cctx; + + /** */ + private final LinkedHashMap infos = new LinkedHashMap<>(); + + /** */ + private final AffinityTopologyVersion topVer; + + /** */ + private final boolean preload; + + /** */ + private List entries; + + /** */ + private int skipped; + + /** */ + public BatchedCacheEntries(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 addEntry(KeyCacheObject key, CacheObject val, long expTime, long ttl, GridCacheVersion ver, GridDrType drType) { + // todo remove `key` duplication (Map keys() { + return infos.keySet(); + } + + /** */ + public Collection values() { + return infos.values(); + } + + /** */ +// public int part() { +// return partId; +// } + + /** */ + public GridDhtLocalPartition part() { + return part; + } + + /** */ + public GridCacheContext context() { + return cctx; + } + + /** */ + public BatchedCacheMapEntryInfo get(KeyCacheObject key) { + return infos.get(key); + } + + /** */ + public boolean preload() { + return preload; + } + + /** */ + public boolean needUpdate(KeyCacheObject key, CacheDataRow row) throws GridCacheEntryRemovedException { + BatchedCacheMapEntryInfo info = infos.get(key); + + return info.needUpdate(row); + } + + public void onRemove(KeyCacheObject key) { + // todo - remove from original collection + ++skipped; + } + + public void onError(KeyCacheObject key, IgniteCheckedException e) { + // todo - remove from original collection + ++skipped; + } + + public boolean skip(KeyCacheObject key) { + // todo + return false; + } + + public List lock() { + entries = lockEntries(infos.values(), topVer); + + return entries; + } + + public void unlock() { + unlockEntries(infos.values(), topVer); + } + + public int size() { + return infos.size() - skipped; + } + + private List lockEntries(Collection list, AffinityTopologyVersion topVer) + throws GridDhtInvalidPartitionException { +// if (req.size() == 1) { +// KeyCacheObject key = req.key(0); +// +// while (true) { +// GridDhtCacheEntry entry = entryExx(key, topVer); +// +// entry.lockEntry(); +// +// if (entry.obsolete()) +// entry.unlockEntry(); +// else +// return Collections.singletonList(entry); +// } +// } +// else { + List locked = new ArrayList<>(list.size()); + + while (true) { + for (BatchedCacheMapEntryInfo info : list) { + GridDhtCacheEntry entry = (GridDhtCacheEntry)cctx.cache().entryEx(info.key(), topVer); + + locked.add(entry); + + info.cacheEntry(entry); + } + + boolean retry = false; + + for (int i = 0; i < locked.size(); i++) { + GridCacheMapEntry entry = locked.get(i); + + if (entry == null) + continue; + + // todo ensure free space + // todo check obsolete + + 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 locked; + } +// } + } + + /** + * Releases java-level locks on cache entries + * todo carefully think about possible reorderings in locking/unlocking. + * + * @param locked Locked entries. + * @param topVer Topology version. + */ + private void unlockEntries(Collection locked, AffinityTopologyVersion topVer) { + // Process deleted entries before locks release. + assert cctx.deferredDelete() : this; + + // Entries to skip eviction manager notification for. + // Enqueue entries while holding locks. + // todo Common skip list. + Collection skip = null; + + int size = locked.size(); + + try { + for (BatchedCacheMapEntryInfo info : locked) { + GridCacheMapEntry entry = info.cacheEntry(); + + if (entry != null && entry.deleted()) { + if (skip == null) + skip = U.newHashSet(locked.size()); + + skip.add(entry.key()); + } + + try { + info.updateCacheEntry(); + } catch (IgniteCheckedException e) { + skip.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 (BatchedCacheMapEntryInfo info : locked) { + GridCacheMapEntry entry = info.cacheEntry(); + if (entry != null) + entry.unlockEntry(); + } + } + + // Try evict partitions. + for (BatchedCacheMapEntryInfo info : locked) { + GridDhtCacheEntry entry = info.cacheEntry(); + if (entry != null) + entry.onUnlock(); + } + + if (skip != null && skip.size() == size) + // Optimization. + return; + + // Must touch all entries since update may have deleted entries. + // Eviction manager will remove empty entries. + for (BatchedCacheMapEntryInfo info : locked) { + GridCacheMapEntry entry = info.cacheEntry(); + if (entry != null && (skip == null || !skip.contains(entry.key()))) + entry.touch(topVer); + } + } + +// public KeyCacheObject lastKey() { +// return lastKey; +// } + + public class UpdateClosure implements IgniteCacheOffheapManager.OffheapInvokeAllClosure { + + private List> resBatch = new ArrayList<>(entries.size()); + + /** {@inheritDoc} */ + @Override public void call(@Nullable Collection> rows) throws IgniteCheckedException { + List newRows = new ArrayList<>(16); + + final int cacheId = cctx.group().storeCacheIdInDataPage() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + for (T2 t2 : rows) { + CacheDataRow oldRow = t2.get1(); + + KeyCacheObject key = t2.get2().key(); + + BatchedCacheMapEntryInfo newRowInfo = get(key); + + // todo +// if (key.partition() == -1) +// key.partition(partId); + + try { + if (newRowInfo.needUpdate(oldRow)) { + CacheDataRow newRow = null; + + boolean noop = false; + + if (oldRow != null) { + // todo think about batch updates + newRow = context().offheap().dataStore(part()).createRow( + cctx, + key, + newRowInfo.value(), + newRowInfo.version(), + newRowInfo.expireTime(), + oldRow); + + noop = oldRow.link() == newRow.link(); + } + else { + + CacheObjectContext coCtx = cctx.cacheObjectContext(); + + CacheObject val = newRowInfo.value(); + + val.valueBytes(coCtx); + key.valueBytes(coCtx); + + if (key.partition() == -1) + key.partition(part().id()); + + newRow = new DataRow(key, val, newRowInfo.version(), part().id(), newRowInfo.expireTime(), cacheId); + + newRows.add(newRow); +// newRow = context().offheap().dataStore(part()).createRow( +// cctx, +// key, +// newRowInfo.value(), +// newRowInfo.version(), +// newRowInfo.expireTime(), +// oldRow); +// +// newRow = context().offheap().dataStore(part()).makeDataRow(key, val, newRowInfo.version(), newRowInfo.expireTime(), cacheId); +//// +// context().offheap().dataStore(part()).rowStore().addRow(newRow, cctx.group().statisticsHolderData()); + } + + resBatch.add(new T3<>(noop ? IgniteTree.OperationType.NOOP : IgniteTree.OperationType.PUT, oldRow, newRow)); + } + } + catch (GridCacheEntryRemovedException e) { + onRemove(key); + } + } + + // todo add addRows to rowstore + //insertDataRows + context().offheap().dataStore(part()).rowStore().addRows(newRows, cctx.group().statisticsHolderData()); + } + + @Override public Collection> result() { + return resBatch; + } + + @Override public boolean apply(CacheDataRow row) { + return false; + } + } + + public static class BatchedCacheMapEntryInfo { + // todo think about remove + private final BatchedCacheEntries batch; + private final KeyCacheObject key; + private final CacheObject val; + private final long expTime; + private final long ttl; + private final GridCacheVersion ver; + private final GridDrType drType; + + private GridDhtCacheEntry entry; + + private boolean update; + + public BatchedCacheMapEntryInfo( + BatchedCacheEntries batch, + KeyCacheObject key, + CacheObject val, + long expTime, + long ttl, + GridCacheVersion ver, + GridDrType drType + ) { + this.batch = batch; + this.key = key; + this.val = val; + this.expTime = expTime; + this.ver = ver; + this.drType = drType; + this.ttl = ttl; + } + + public KeyCacheObject key() { + return key; + } + + public GridCacheVersion version() { + return ver; + } + + public CacheObject value() { + return val; + } + + public long expireTime() { + return expTime; + } + + public GridDhtCacheEntry cacheEntry() { + return entry; + } + + public void cacheEntry(GridDhtCacheEntry entry) { + this.entry = entry; + } + + public void updateCacheEntry() throws IgniteCheckedException { + if (!update) + return; + +// batch.context().kernalContext().diagnostic().beginTrack(PRELOAD_TREE_FINISH_UPDATE1); + + entry.finishPreload(val, expTime, ttl, ver, batch.topVer, drType, null, batch.preload); + +// batch.context().kernalContext().diagnostic().endTrack(PRELOAD_TREE_FINISH_UPDATE1); + } + +// public void update(boolean update) { +// this.update = update; +// } + + 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 = ATOMIC_VER_COMPARATOR.compare(currVer, version()) < 0; + else + update0 = currVer.compareTo(version()) < 0; + } + else + update0 = true; + } + else + update0 = (isStartVer && row == null); + + // todo update0 |= (!preload && 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 77d8a86e64550..3be061c2c57c7 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 @@ -1167,6 +1167,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/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java index 8cef1763af868..2d1ceead27479 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java @@ -808,6 +808,18 @@ public boolean initialValue(CacheObject val, GridDrType drType, boolean fromStore) throws IgniteCheckedException, GridCacheEntryRemovedException; + + public void finishPreload( + @Nullable CacheObject val, + long expTime, + long ttl, + GridCacheVersion ver, + AffinityTopologyVersion topVer, + GridDrType drType, + MvccVersion mvccVer, + boolean preload + ) throws IgniteCheckedException; + /** * Create versioned entry for this cache entry. * @@ -1071,7 +1083,6 @@ public void updateIndex(SchemaIndexCacheFilter filter, SchemaIndexCacheVisitorCl */ @Nullable public CacheObject unswap(CacheDataRow row) throws IgniteCheckedException, GridCacheEntryRemovedException; - /** * Unswap ignoring flags. * 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 899417d7cf59d..8cbb7ff8236c6 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 @@ -77,6 +77,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx; +import org.apache.ignite.internal.processors.diag.DiagnosticTopics; import org.apache.ignite.internal.processors.dr.GridDrType; import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheFilter; @@ -122,6 +123,7 @@ import static org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.RowData.NO_KEY; import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.DUPLICATE_KEY; import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.TRANSACTION_SERIALIZATION_ERROR; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_UPDATED; import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE; /** @@ -2979,6 +2981,8 @@ protected final void update(@Nullable CacheObject val, long expireTime, long ttl assert lock.isHeldByCurrentThread(); assert ttl != CU.TTL_ZERO && ttl != CU.TTL_NOT_CHANGED && ttl >= 0 : ttl; +// cctx.kernalContext().diagnostic().beginTrack(PRELOAD_UPDATED); + boolean trackNear = addTracked && isNear() && cctx.config().isEagerTtl(); long oldExpireTime = expireTimeExtras(); @@ -2995,6 +2999,8 @@ protected final void update(@Nullable CacheObject val, long expireTime, long ttl if (trackNear && expireTime != 0 && (expireTime != oldExpireTime || isStartVersion())) cctx.ttl().addTrackedEntry((GridNearCacheEntry)this); + +// cctx.kernalContext().diagnostic().endTrack(PRELOAD_UPDATED); } /** @@ -3474,6 +3480,8 @@ else if (val == null) update = storeValue(val, expTime, ver, p); } +// log.info("update=" + update + " key=" + keyValue(false)); + if (update) { update(val, expTime, ttl, ver, true); @@ -3508,6 +3516,7 @@ else if (deletedUnlocked()) mvccVer == null ? MvccUtils.INITIAL_VERSION : mvccVer ))); } else { +// cctx.kernalContext().diagnostic().beginTrack(DiagnosticTopics.PRELOAD_ON_WAL_LOG); cctx.shared().wal().log(new DataRecord(new DataEntry( cctx.cacheId(), key, @@ -3519,12 +3528,14 @@ else if (deletedUnlocked()) partition(), updateCntr ))); +// cctx.kernalContext().diagnostic().endTrack(DiagnosticTopics.PRELOAD_ON_WAL_LOG); } } drReplicate(drType, val, ver, topVer); if (!skipQryNtf) { +// cctx.kernalContext().diagnostic().beginTrack(DiagnosticTopics.PRELOAD_ON_ENTRY_UPDATED); cctx.continuousQueries().onEntryUpdated( key, val, @@ -3536,6 +3547,7 @@ else if (deletedUnlocked()) updateCntr, null, topVer); +// cctx.kernalContext().diagnostic().endTrack(DiagnosticTopics.PRELOAD_ON_ENTRY_UPDATED); } onUpdateFinished(updateCntr); @@ -3570,6 +3582,91 @@ else if (deletedUnlocked()) } } + /** {@inheritDoc} */ + @Override public void finishPreload( + @Nullable CacheObject val, + long expTime, + 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, 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, + expTime, + 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, + expTime, + partition(), + updateCntr + ))); + } + } + + drReplicate(drType, val, ver, topVer); + + if (!skipQryNtf) { + cctx.continuousQueries().onEntryUpdated( + key, + val, + null, + this.isInternal() || !this.context().userCache(), + this.partition(), + true, + true, + 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. */ @@ -4324,10 +4421,14 @@ protected boolean storeValue( @Nullable IgnitePredicate predicate) throws IgniteCheckedException { assert lock.isHeldByCurrentThread(); +// cctx.kernalContext().diagnostic().beginTrack(DiagnosticTopics.PRELOAD_OFFHEAP_INVOKE); + UpdateClosure closure = new UpdateClosure(this, val, ver, expireTime, predicate); cctx.offheap().invoke(cctx, key, localPartition(), closure); +// cctx.kernalContext().diagnostic().endTrack(DiagnosticTopics.PRELOAD_OFFHEAP_INVOKE); + return closure.treeOp != IgniteTree.OperationType.NOOP; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index d2304d44bd2e9..c4ccbac8fccb6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -138,6 +138,7 @@ import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap.PARTIAL_COUNTERS_MAP_SINCE; import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.nextDumpTimeout; import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader.DFLT_PRELOAD_RESEND_TIMEOUT; +import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.TOTAL; /** * Partition exchange manager. @@ -3041,6 +3042,8 @@ else if (task instanceof ForceRebalanceExchangeTask) { if (task instanceof ForceRebalanceExchangeTask) forcedRebFut = ((ForceRebalanceExchangeTask)task).forcedRebalanceFuture(); +// cctx.kernalContext().diagnostic().beginTrack(TOTAL); + for (Integer order : orderMap.descendingKeySet()) { for (Integer grpId : orderMap.get(order)) { CacheGroupContext grp = cctx.cache().cacheGroup(grpId); 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 7f0fc3096d9a0..f13ed23edd226 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; @@ -34,6 +35,7 @@ import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId; import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionRecoverState; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; +import org.apache.ignite.internal.processors.cache.tree.DataRow; import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree; import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateResult; import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow; @@ -47,6 +49,8 @@ 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.NotNull; import org.jetbrains.annotations.Nullable; /** @@ -188,6 +192,16 @@ public boolean expire(GridCacheContext cctx, IgniteInClosure2X keys, GridDhtLocalPartition part, + OffheapInvokeAllClosure c) throws IgniteCheckedException; + /** * @param cctx Cache context. * @param key Key. @@ -421,6 +435,19 @@ public void update( @Nullable CacheDataRow oldRow ) throws IgniteCheckedException; + /** todo + */ + public void updateBatch( + BatchedCacheEntries batchEntries + ) throws IgniteCheckedException; + + public void updateBatch( + GridCacheContext cctx, + List keys, + GridDhtLocalPartition part, + Map items + ) throws IgniteCheckedException; + /** * @param cctx Cache context. * @param key Key. @@ -640,10 +667,20 @@ interface OffheapInvokeClosure extends IgniteTree.InvokeClosure { @Nullable public CacheDataRow oldRow(); } + /** + * + */ + interface OffheapInvokeAllClosure extends IgniteTree.InvokeAllClosure, IgnitePredicate { +// boolean preload(); + } + /** * */ interface CacheDataStore { + + @NotNull public DataRow makeDataRow(KeyCacheObject key, CacheObject val, GridCacheVersion ver, long expireTime, int cacheId); + /** * @return Partition ID. */ @@ -779,6 +816,23 @@ void update( long expireTime, @Nullable CacheDataRow oldRow) throws IgniteCheckedException; + /** + * @param cctx Cache context. + * @param keys Sorted keys. + * @param items todo + * @throws IgniteCheckedException If failed. + */ + public void updateBatch( + GridCacheContext cctx, + List keys, + Map items) throws IgniteCheckedException; + + /** todo + */ + public void updateBatch( + BatchedCacheEntries batchEntries + ) throws IgniteCheckedException; + /** * @param cctx Cache context. * @param key Key. @@ -960,6 +1014,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 0976f637a32b1..cc9d8bca731aa 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,10 +18,12 @@ 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; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; @@ -102,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.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; @@ -128,6 +131,17 @@ import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.unexpectedStateException; import static org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.EMPTY_CURSOR; import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.MVCC_INFO_SIZE; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_OFFHEAP_BATCH_FIND; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_OFFHEAP_BATCH_INSERT; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_OFFHEAP_BATCH_TREE_INSERT; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_PENDING_TREE_PUT; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_PENDING_TREE_REMOVE; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_TREE_ADD_ROW; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_TREE_FINISH_UPDATE; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_TREE_INVOKE; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_OFFHEAP_BATCH_FIND; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_OFFHEAP_BATCH_INSERT; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_OFFHEAP_BATCH_TREE_INSERT; import static org.apache.ignite.internal.util.IgniteTree.OperationType.NOOP; import static org.apache.ignite.internal.util.IgniteTree.OperationType.PUT; @@ -440,6 +454,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, @@ -455,6 +479,23 @@ private Iterator cacheData(boolean primary, boolean backup, Affi dataStore(part).update(cctx, key, val, ver, expireTime, oldRow); } + /** {@inheritDoc} */ + @Override public void updateBatch( + BatchedCacheEntries batchEntries + ) throws IgniteCheckedException { + dataStore(batchEntries.part()).updateBatch(batchEntries); + } + + /** {@inheritDoc} */ + @Override public void updateBatch( + GridCacheContext cctx, + List keys, + GridDhtLocalPartition part, + Map items + ) throws IgniteCheckedException { + dataStore(part).updateBatch(cctx, keys, items); + } + /** {@inheritDoc} */ @Override public boolean mvccInitialValue( GridCacheMapEntry entry, @@ -680,6 +721,8 @@ private Iterator cacheData(boolean primary, boolean backup, Affi CacheDataRow row = dataStore != null ? dataStore.find(cctx, key) : null; +// log.info(">xxx> Key=" + key + " dataStore=" + dataStore + " row=" + row); + assert row == null || row.value() != null : row; return row; @@ -1616,6 +1659,8 @@ void decrementSize(int cacheId) { * @param dataRow New row. * @return {@code True} if it is possible to update old row data. * @throws IgniteCheckedException If failed. + * + * todo think about this meth */ private boolean canUpdateOldRow(GridCacheContext cctx, @Nullable CacheDataRow oldRow, DataRow dataRow) throws IgniteCheckedException { @@ -1626,7 +1671,7 @@ private boolean canUpdateOldRow(GridCacheContext cctx, @Nullable CacheDataRow ol return false; // Use grp.sharedGroup() flag since it is possible cacheId is not yet set here. - boolean sizeWithCacheId = grp.sharedGroup(); +// boolean sizeWithCacheId = grp.sharedGroup(); int oldLen = oldRow.size(); @@ -1646,6 +1691,256 @@ private boolean canUpdateOldRow(GridCacheContext cctx, @Nullable CacheDataRow ol invoke0(cctx, new SearchRow(cacheId, key), c); } + + /** {@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); + } + + /** {@inheritDoc} */ + @Override public void updateBatch(BatchedCacheEntries items) throws IgniteCheckedException { +// int size = items.size(); + + GridCacheContext cctx = items.context(); + + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + // todo bench perf linked vs not-linked + List updateRows = null; + + // todo can rid from it - measure performance with iterator. + Set insertKeys = null; + + // + if (items.preload() && !cctx.group().persistenceEnabled()) { +// cctx.kernalContext().diagnostic().beginTrack(PRELOAD_OFFHEAP_BATCH_FIND); +// List sortedKeys = new ArrayList<>(items.keys()); +// for (KeyCacheObject k : items.keys()) { +// } +// assert sortedKeys.size() > 1 : sortedKeys.size() + " cache="+cctx.name(); +// NavigableMap map = items.keys(); + Iterator itr = items.keys().iterator(); + KeyCacheObject firstKey = null; + KeyCacheObject lastKey = null; + + while (itr.hasNext()) { + lastKey = itr.next(); + + if (firstKey == null) + firstKey = lastKey; + } + + assert !items.preload() || lastKey.hashCode() >= firstKey.hashCode() : "Keys not sorted by hash: first=" + firstKey.hashCode() + ", last=" + lastKey.hashCode(); + + GridCursor cur = dataTree.find(new SearchRow(cacheId, firstKey), new SearchRow(cacheId, lastKey)); + + while (cur.next()) { + //todo optimize insertKeys creation + if (insertKeys == null) + insertKeys = new HashSet<>(items.keys()); +// assert false : "firstKey=" + firstKey.value(cctx.cacheObjectContext(), false) + ", lastKey=" + lastKey.value(cctx.cacheObjectContext(), false) + ", cur=" + cur.get().key().value(cctx.cacheObjectContext(), false); + + CacheDataRow row = cur.get(); + + try { + if (insertKeys.remove(row.key()) && items.needUpdate(row.key(), row)) { //, items.get(row.key()).version())) + if (updateRows == null) + updateRows = new ArrayList<>(8); + + updateRows.add(row); + } + } + catch (GridCacheEntryRemovedException e) { + items.onRemove(row.key()); + } + } + +// cctx.kernalContext().diagnostic().endTrack(PRELOAD_OFFHEAP_BATCH_FIND); + } + else { + insertKeys = new HashSet<>(); + + for (BatchedCacheEntries.BatchedCacheMapEntryInfo info : items.values()) { + try { + CacheDataRow row = find(cctx, info.key()); + + if (info.needUpdate(row)) { + if (row != null) + updateRows.add(row); + else + insertKeys.add(info.key()); + } + } + catch (GridCacheEntryRemovedException e) { + items.onRemove(info.key()); + } + } + } + + // Updates. + if (updateRows != null) + for (CacheDataRow row : updateRows) { + KeyCacheObject key = row.key(); + // todo why we don't need here to marshal cache object (call valueBytes) + + BatchedCacheEntries.BatchedCacheMapEntryInfo entry = items.get(key); + + update(cctx, key, entry.value(), entry.version(), entry.expireTime(), row); + } + + // New. + if (insertKeys == null) + insertKeys = items.keys(); + + List newRows = new ArrayList<>(insertKeys.size()); + + for (KeyCacheObject key : insertKeys) { + try { + if (!items.needUpdate(key, null)) + continue; + } + catch (GridCacheEntryRemovedException e) { + items.onRemove(key); + } + + BatchedCacheEntries.BatchedCacheMapEntryInfo entry = items.get(key); + + CacheObject val = entry.value(); + val.valueBytes(cctx.cacheObjectContext()); + key.valueBytes(cctx.cacheObjectContext()); + +// long expTime = entry.ttl() < 0 ? CU.toExpireTime(entry.ttl()) : entry.ttl(); + + DataRow row = makeDataRow(key, val, entry.version(), entry.expireTime(), cacheId); + + assert row.value() != null : key.hashCode(); + + newRows.add(row); + } + +// cctx.kernalContext().diagnostic().beginTrack(PRELOAD_OFFHEAP_BATCH_INSERT); + + rowStore.freeList().insertDataRows(newRows, grp.statisticsHolderData()); + +// cctx.kernalContext().diagnostic().endTrack(PRELOAD_OFFHEAP_BATCH_INSERT); + +// cctx.kernalContext().diagnostic().beginTrack(PRELOAD_OFFHEAP_BATCH_TREE_INSERT); + + for (DataRow row : newRows) { + dataTree.putx(row); + + finishUpdate(cctx, row, null); + } + +// cctx.kernalContext().diagnostic().endTrack(PRELOAD_OFFHEAP_BATCH_TREE_INSERT); + } + + @Override public void updateBatch( + GridCacheContext cctx, + List keys, + Map items +// OffheapInvokeClosure c + ) throws IgniteCheckedException { + // todo ensure sorted + int size = keys.size(); + + int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; + + KeyCacheObject first = keys.get(0); + KeyCacheObject last = keys.get(size - 1); + + assert last.hashCode() >= first.hashCode() : "Keys not sorted by hash: first=" + first.hashCode() + ", last=" + last.hashCode(); + + // todo check on which range we can loose performance (if there will be a lot of misses). + + GridCursor cur = dataTree.find(new SearchRow(cacheId, first), new SearchRow(cacheId, last)); + + // todo bench perf linked vs not-linked + Map updateKeys = new LinkedHashMap<>(); + // todo can rid from it - measure performance with iterator. + Set insertKeys = new HashSet<>(keys); + + while (cur.next()) { + CacheDataRow row = cur.get(); + + if (insertKeys.remove(row.key()) && needUpdate(cctx, row, items.get(row.key()).version())) + updateKeys.put(row.key(), row); + } + + // Updates. + for (Map.Entry e : updateKeys.entrySet()) { + KeyCacheObject key = e.getKey(); + + GridCacheEntryInfo entry = items.get(key); + + update(cctx, key, entry.value(), entry.version(), entry.expireTime(), e.getValue()); + } + + // New. + List dataRows = new ArrayList<>(insertKeys.size()); + + for (KeyCacheObject key : insertKeys) { + GridCacheEntryInfo entry = items.get(key); + + + CacheObject val = entry.value(); + val.valueBytes(cctx.cacheObjectContext()); + key.valueBytes(cctx.cacheObjectContext()); + + long expTime = entry.ttl() < 0 ? CU.toExpireTime(entry.ttl()) : entry.ttl(); + + DataRow row = makeDataRow(key, val, entry.version(), expTime, cacheId); + + assert row.value() != null : key.hashCode(); + + dataRows.add(row); + } + + rowStore.freeList().insertDataRows(dataRows, grp.statisticsHolderData()); + + for (DataRow row : dataRows) { + dataTree.putx(row); + + finishUpdate(cctx, row, null); + } + + + } + + // todo + private boolean needUpdate(GridCacheContext cctx, CacheDataRow row, GridCacheVersion ver) { + boolean update0; + + GridCacheVersion currVer = row != null ? row.version() : ver; + + boolean isStartVer = cctx.shared().versions().isStartVersion(currVer); + + if (cctx.group().persistenceEnabled()) { + if (!isStartVer) { + if (cctx.atomic()) + update0 = GridCacheMapEntry.ATOMIC_VER_COMPARATOR.compare(currVer, ver) < 0; + else + update0 = currVer.compareTo(ver) < 0; + } + else + update0 = true; + } + else + update0 = isStartVer; + + // todo update0 |= (!preload && deletedUnlocked()); + + return update0; + } + /** * @param cctx Cache context. * @param row Search row. @@ -1660,16 +1955,24 @@ private void invoke0(GridCacheContext cctx, CacheSearchRow row, OffheapInvokeClo try { assert cctx.shared().database().checkpointLockIsHeldByThread(); +// ctx.kernalContext().diagnostic().beginTrack(PRELOAD_TREE_INVOKE); + dataTree.invoke(row, CacheDataRowAdapter.RowData.NO_KEY, c); +// ctx.kernalContext().diagnostic().endTrack(PRELOAD_TREE_INVOKE); + switch (c.operationType()) { case PUT: { assert c.newRow() != null : c; CacheDataRow oldRow = c.oldRow(); +// ctx.kernalContext().diagnostic().beginTrack(PRELOAD_TREE_FINISH_UPDATE); + finishUpdate(cctx, c.newRow(), oldRow); +// ctx.kernalContext().diagnostic().endTrack(PRELOAD_TREE_FINISH_UPDATE); + break; } @@ -1693,6 +1996,72 @@ 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(); + +// ctx.kernalContext().diagnostic().beginTrack(PRELOAD_TREE_INVOKE); + + dataTree.invokeAll(rows, CacheDataRowAdapter.RowData.NO_KEY, c); + +// ctx.kernalContext().diagnostic().endTrack(PRELOAD_TREE_INVOKE); + + 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; + +// CacheDataRow oldRow = c.oldRow(); + +// ctx.kernalContext().diagnostic().beginTrack(PRELOAD_TREE_FINISH_UPDATE); + assert oldRow == null; + + finishUpdate(cctx, newRow, oldRow); + +// ctx.kernalContext().diagnostic().endTrack(PRELOAD_TREE_FINISH_UPDATE); + + break; + } + + case REMOVE: { + // todo oldRow - doesn't have a key (optimized) + // todo key should be get from arguments (from rows) + finishRemove(cctx, oldRow.key(), oldRow); + + break; + } + + case NOOP: + break; + + default: + assert false : opType; + } + } + + + } + finally { + busyLock.leaveBusy(); + } + } + /** {@inheritDoc} */ @Override public CacheDataRow createRow( GridCacheContext cctx, @@ -1705,8 +2074,11 @@ private void invoke0(GridCacheContext cctx, CacheSearchRow row, OffheapInvokeClo DataRow dataRow = makeDataRow(key, val, ver, expireTime, cacheId); - if (canUpdateOldRow(cctx, oldRow, dataRow) && rowStore.updateRow(oldRow.link(), dataRow, grp.statisticsHolderData())) + if (canUpdateOldRow(cctx, oldRow, dataRow) && rowStore.updateRow(oldRow.link(), dataRow, grp.statisticsHolderData())) { + assert false; + dataRow.link(oldRow.link()); + } else { CacheObjectContext coCtx = cctx.cacheObjectContext(); @@ -1718,8 +2090,11 @@ private void invoke0(GridCacheContext cctx, CacheSearchRow row, OffheapInvokeClo assert dataRow.link() != 0 : dataRow; - if (grp.sharedGroup() && dataRow.cacheId() == CU.UNDEFINED_CACHE_ID) + if (grp.sharedGroup() && dataRow.cacheId() == CU.UNDEFINED_CACHE_ID) { + assert false; + dataRow.cacheId(cctx.cacheId()); + } return dataRow; } @@ -1732,7 +2107,7 @@ private void invoke0(GridCacheContext cctx, CacheSearchRow row, OffheapInvokeClo * @param cacheId Cache id. * @return Made data row. */ - @NotNull private DataRow makeDataRow(KeyCacheObject key, CacheObject val, GridCacheVersion ver, long expireTime, + @NotNull public DataRow makeDataRow(KeyCacheObject key, CacheObject val, GridCacheVersion ver, long expireTime, int cacheId) { if (key.partition() == -1) key.partition(partId); @@ -2687,13 +3062,17 @@ private void updatePendingEntries(GridCacheContext cctx, CacheDataRow newRow, @N if (oldRow != null) { assert oldRow.link() != 0 : oldRow; - if (pendingTree() != null && oldRow.expireTime() != 0) + if (pendingTree() != null && oldRow.expireTime() != 0) { +// cctx.kernalContext().diagnostic().beginTrack(PRELOAD_PENDING_TREE_REMOVE); pendingTree().removex(new PendingRow(cacheId, oldRow.expireTime(), oldRow.link())); +// cctx.kernalContext().diagnostic().endTrack(PRELOAD_PENDING_TREE_REMOVE); + } } if (pendingTree() != null && expireTime != 0) { +// cctx.kernalContext().diagnostic().beginTrack(PRELOAD_PENDING_TREE_PUT); pendingTree().putx(new PendingRow(cacheId, expireTime, newRow.link())); - +// cctx.kernalContext().diagnostic().endTrack(PRELOAD_PENDING_TREE_PUT); hasPendingEntries = true; } } @@ -2964,6 +3343,7 @@ private void afterRowFound(@Nullable CacheDataRow row, KeyCacheObject key) throw dataTree.destroy(new IgniteInClosure() { @Override public void apply(CacheSearchRow row) { try { +// log.info("Remove row: " + row.key().hashCode() + " link " + row.link()); rowStore.removeRow(row.link(), grp.statisticsHolderData()); } catch (IgniteCheckedException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java index bae326424d0fb..f7bfc7e00ea7e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java @@ -70,6 +70,9 @@ public class GridDhtPartitionDemandMessage extends GridCacheGroupIdMessage { /** Topology version. */ private AffinityTopologyVersion topVer; + /** */ + private long timestamp; + /** * @param rebalanceId Rebalance id for this node. * @param topVer Topology version. @@ -132,6 +135,7 @@ public GridDhtPartitionDemandMessage withNewPartitionsMap(@NotNull IgniteDhtDema cp.workerId = workerId; cp.topVer = topVer; cp.parts = parts; + cp.timestamp = U.currentTimeMillis(); return cp; } @@ -163,6 +167,14 @@ long timeout() { return timeout; } + long timestamp() { + return timestamp; + } + + void timestamp(long timestamp) { + this.timestamp = timestamp; + } + /** * @param timeout Timeout. */ @@ -297,6 +309,12 @@ public GridCacheMessage convertIfNeeded(IgniteProductVersion target) { writer.incrementState(); + case 10: + if (!writer.writeLong("timestamp", timestamp)) + return false; + + writer.incrementState(); + } return true; @@ -360,7 +378,13 @@ public GridCacheMessage convertIfNeeded(IgniteProductVersion target) { return false; reader.incrementState(); + case 10: + timestamp = reader.readLong("timestamp"); + if (!reader.isLastRead()) + return false; + + reader.incrementState(); } return reader.afterMessageRead(GridDhtPartitionDemandMessage.class); 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 e970a639660bc..20f09d604219b 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,6 +30,7 @@ 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; @@ -40,6 +41,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.affinity.AffinityAssignment; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.BatchedCacheEntries; import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheMetricsImpl; @@ -47,8 +49,10 @@ 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.GridCachePartitionExchangeManager; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtInvalidPartitionException; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology; @@ -65,6 +69,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -78,6 +83,22 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STARTED; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STOPPED; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING; +import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH_LOCK; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH_PREPARE; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH_UNLOCK; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH_UPDATE; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_ENTRY; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.SEND_DEMAND; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.SEND_RECEIVE; +import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH_LOCK; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH_UNLOCK; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH_UPDATE; +import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_SINGLE; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.SUPPLY_MSG_SEND; +import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.TOTAL; import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE; import static org.apache.ignite.internal.processors.dr.GridDrType.DR_PRELOAD; @@ -85,6 +106,16 @@ * Thread pool for requesting partitions from other nodes and populating local cache. */ public class GridDhtPartitionDemander { + /** */ + 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, false); + /** */ private final GridCacheSharedContext ctx; @@ -372,6 +403,12 @@ Runnable addAssignments( log.debug(e.getMessage()); } }); +// else +// fut.listen(f -> { +// ctx.kernalContext().diagnostic().endTrack(TOTAL); +// +// ctx.kernalContext().diagnostic().printStats(); +// }); requestPartitions(fut, assignments); }; @@ -461,8 +498,8 @@ private void requestPartitions(final RebalanceFuture fut, GridDhtPreloaderAssign parts = fut.remaining.get(node.id()); U.log(log, "Prepared rebalancing [grp=" + grp.cacheOrGroupName() - + ", mode=" + cfg.getRebalanceMode() + ", supplier=" + node.id() + ", partitionsCount=" + parts.size() - + ", topVer=" + fut.topologyVersion() + ", parallelism=" + totalStripes + "]"); + + ", mode=" + cfg.getRebalanceMode() + ", supplier=" + node.id() + ", partitionsCount=" + parts.size() + + ", topVer=" + fut.topologyVersion() + ", parallelism=" + totalStripes + "]"); } int stripes = totalStripes; @@ -471,6 +508,8 @@ private void requestPartitions(final RebalanceFuture fut, GridDhtPreloaderAssign for (int i = 0; i < stripes; i++) stripePartitions.add(new IgniteDhtDemandedPartitionsMap()); + ctx.kernalContext().diagnostic().beginTrack(TOTAL); + // Reserve one stripe for historical partitions. if (parts.hasHistorical()) { stripePartitions.set(stripes - 1, new IgniteDhtDemandedPartitionsMap(parts.historicalMap(), null)); @@ -503,9 +542,15 @@ private void requestPartitions(final RebalanceFuture fut, GridDhtPreloaderAssign return; try { +// ctx.kernalContext().diagnostic().beginTrack(SEND_DEMAND); + + demandMsg.timestamp(U.currentTimeMillis()); + ctx.io().sendOrderedMessage(node, rebalanceTopics.get(topicId), demandMsg.convertIfNeeded(node.version()), grp.ioPolicy(), demandMsg.timeout()); +// ctx.kernalContext().diagnostic().beginTrack(SEND_RECEIVE); + // Cleanup required in case partitions demanded in parallel with cancellation. synchronized (fut) { if (fut.isDone()) @@ -533,6 +578,9 @@ private void requestPartitions(final RebalanceFuture fut, GridDhtPreloaderAssign fut.cancel(); } + +// ctx.kernalContext().diagnostic().endTrack(SEND_DEMAND); + }, true)); } } @@ -654,8 +702,12 @@ public void handleSupplyMessage( final UUID nodeId, final GridDhtPartitionSupplyMessage supplyMsg ) { + ctx.kernalContext().diagnostic().beginTrack(DEMANDER_PROCESS_MSG); + AffinityTopologyVersion topVer = supplyMsg.topologyVersion(); +// ctx.kernalContext().diagnostic().timeTrack(SUPPLY_MSG_SEND, (U.currentTimeMillis() - supplyMsg.timestamp())); + final RebalanceFuture fut = rebalanceFut; ClusterNode node = ctx.node(nodeId); @@ -764,33 +816,71 @@ public void handleSupplyMessage( part.lock(); +// log.info("process infos: " + e.getValue().infos().size()); + try { + int size = e.getValue().infos().size(); + + boolean batchEnabled = + batchPageWriteEnabled && size > BATCH_PRELOAD_THRESHOLD; + Iterator infos = e.getValue().infos().iterator(); + int nBatch = 0; + int total = size / CHECKPOINT_THRESHOLD; + + // todo improve code (iterations) +// int limit = CHECKPOINT_THRESHOLD; +// ctx.cache().persistentCaches().isEmpty() ? +// Math.max(e.getValue().infos().size(), CHECKPOINT_THRESHOLD) : CHECKPOINT_THRESHOLD; + + //assert limit >= CHECKPOINT_THRESHOLD : limit; + // Loop through all received entries and try to preload them. while (infos.hasNext()) { ctx.database().checkpointReadLock(); + boolean tail = (nBatch++ >= (total - 1)); + try { - for (int i = 0; i < 100; i++) { + List infosBatch = new ArrayList<>(CHECKPOINT_THRESHOLD); + + for (int i = 0; i < (tail ? CHECKPOINT_THRESHOLD + (size % CHECKPOINT_THRESHOLD) : CHECKPOINT_THRESHOLD); i++) { if (!infos.hasNext()) break; GridCacheEntryInfo entry = infos.next(); - if (!preloadEntry(node, p, entry, topVer)) { - if (log.isTraceEnabled()) - log.trace("Got entries for invalid partition during " + - "preloading (will skip) [p=" + p + ", entry=" + entry + ']'); + GridCacheContext cctx0 = grp.sharedGroup() ? + ctx.cacheContext(entry.cacheId()) : grp.singleCacheContext(); + +// if (cctx0.mvccEnabled() || !batchEnabled || entry.value() == null) { +// ctx.kernalContext().diagnostic().beginTrack(DEMANDER_PROCESS_MSG_SINGLE); +// +// try { +// preloadEntry(node, p, entry, topVer); +// +// for (GridCacheContext cctx : grp.caches()) { +// if (cctx.statisticsEnabled()) +// cctx.cache().metrics0().onRebalanceKeyReceived(); +// } +// } finally { +// ctx.kernalContext().diagnostic().endTrack(DEMANDER_PROCESS_MSG_SINGLE); +// } +// +// continue; +// } + + infosBatch.add(entry); + } - break; - } + if (batchEnabled && infosBatch.size() > BATCH_PRELOAD_THRESHOLD) { +// log.info("Preload batch: " + infosBatch.size()); - for (GridCacheContext cctx : grp.caches()) { - if (cctx.statisticsEnabled()) - cctx.cache().metrics0().onRebalanceKeyReceived(); - } + preloadEntries(node, p, infosBatch, topVer); } + else + preloadEntriesSingle(node, p, infosBatch, topVer); } finally { ctx.database().checkpointReadUnlock(); @@ -851,9 +941,14 @@ public void handleSupplyMessage( if (!topologyChanged(fut) && !fut.isDone()) { // Send demand message. try { +// ctx.kernalContext().diagnostic().beginTrack(SEND_DEMAND); + d.timestamp(U.currentTimeMillis()); +// ctx.io().sendOrderedMessage(node, rebalanceTopics.get(topicId), d.convertIfNeeded(node.version()), grp.ioPolicy(), grp.config().getRebalanceTimeout()); +// ctx.kernalContext().diagnostic().beginTrack(SEND_RECEIVE); +// if (log.isDebugEnabled()) log.debug("Send next demand message [" + demandRoutineInfo(topicId, nodeId, supplyMsg) + "]"); } @@ -862,6 +957,9 @@ public void handleSupplyMessage( log.debug("Supplier has left [" + demandRoutineInfo(topicId, nodeId, supplyMsg) + ", errMsg=" + e.getMessage() + ']'); } + +// ctx.kernalContext().diagnostic().endTrack(SEND_DEMAND); +// } else { if (log.isDebugEnabled()) @@ -872,6 +970,266 @@ public void handleSupplyMessage( catch (IgniteSpiException | IgniteCheckedException e) { LT.error(log, e, "Error during rebalancing [" + demandRoutineInfo(topicId, nodeId, supplyMsg) + ", err=" + e + ']'); + } finally { + ctx.kernalContext().diagnostic().endTrack(DEMANDER_PROCESS_MSG); + } + } + + /** + * todo should be removed (kept for benchamrking) + */ + public void preloadEntriesSingle(ClusterNode from, + int p, + Collection entries, + AffinityTopologyVersion topVer + ) throws IgniteCheckedException { + + ctx.kernalContext().diagnostic().beginTrack(DEMANDER_PROCESS_MSG_SINGLE); + try { + // Loop through all received entries and try to preload them. + for (GridCacheEntryInfo entry : entries) { + if (!preloadEntry(from, p, entry, topVer)) { + if (log.isTraceEnabled()) + log.trace("Got entries for invalid partition during " + + "preloading (will skip) [p=" + p + ", entry=" + entry + ']'); + + break; + } + + for (GridCacheContext cctx : grp.caches()) { + if (cctx.statisticsEnabled()) + cctx.cache().metrics0().onRebalanceKeyReceived(); + } + } + } finally { + ctx.kernalContext().diagnostic().endTrack(DEMANDER_PROCESS_MSG_SINGLE); + } + } + + /** + * todo + * @param from + * @param p + * @param entries + * @param topVer + * @throws IgniteCheckedException + */ + public void preloadEntries(ClusterNode from, + int p, + Collection entries, + AffinityTopologyVersion topVer + ) throws IgniteCheckedException { + ctx.kernalContext().diagnostic().beginTrack(DEMANDER_PROCESS_MSG_BATCH); + + try { + + if (entries.isEmpty()) + return; + + Map cctxMap = new HashMap<>(); + +// ctx.kernalContext().diagnostic().beginTrack(DEMANDER_PROCESS_MSG_BATCH_PREPARE); + + // Map by context. + for (GridCacheEntryInfo info : entries) { + try { + GridCacheContext cctx0 = grp.sharedGroup() ? ctx.cacheContext(info.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=" + info.key() + ", part=" + p + ", node=" + from.id() + ']'); + + BatchedCacheEntries batch = cctxMap.get(cctx.cacheId()); + + if (batch == null) { + cctx.continuousQueries().getListenerReadLock().lock(); + + cctxMap.put(cctx.cacheId(), batch = new BatchedCacheEntries(topVer, p, cctx, true)); + } + + batch.addEntry(info.key(), info.value(), info.expireTime(), info.ttl(), info.version(), DR_PRELOAD); + } + catch (GridDhtInvalidPartitionException ignored) { + if (log.isDebugEnabled()) + log.debug("Partition became invalid during rebalancing (will ignore): " + p); + ; + } + } + +// ctx.kernalContext().diagnostic().endTrack(DEMANDER_PROCESS_MSG_BATCH_PREPARE); + + for (BatchedCacheEntries batch : cctxMap.values()) { + assert batch.size() > BATCH_PRELOAD_THRESHOLD : batch.size(); + + GridCacheContext cctx = batch.context(); + +// ctx.kernalContext().diagnostic().beginTrack(DEMANDER_PROCESS_MSG_BATCH_LOCK); + batch.lock(); +// ctx.kernalContext().diagnostic().endTrack(DEMANDER_PROCESS_MSG_BATCH_LOCK); + try { + // todo ticket + assert !cctx.mvccEnabled() : "MVCC caches not supported"; + + // todo looks ugly (batch already have context) +// ctx.kernalContext().diagnostic().beginTrack(DEMANDER_PROCESS_MSG_BATCH_UPDATE); + cctx.offheap().invokeAll(cctx, batch.keys(), batch.part(), batch.new UpdateClosure()); +// cctx.offheap().updateBatch(batch); +// ctx.kernalContext().diagnostic().endTrack(DEMANDER_PROCESS_MSG_BATCH_UPDATE); + } + finally { +// ctx.kernalContext().diagnostic().beginTrack(DEMANDER_PROCESS_MSG_BATCH_UNLOCK); + batch.unlock(); +// ctx.kernalContext().diagnostic().endTrack(DEMANDER_PROCESS_MSG_BATCH_UNLOCK); + + cctx.continuousQueries().getListenerReadLock().unlock(); + + for (GridCacheContext cctx0 : grp.caches()) { + if (cctx0.statisticsEnabled()) + cctx0.cache().metrics0().onRebalanceKeysReceived(batch.size()); // todo size can be wrong + } + } + } + } finally { + ctx.kernalContext().diagnostic().endTrack(DEMANDER_PROCESS_MSG_BATCH); + } +// ctx.kernalContext().diagnostic().endTrack(DEMANDER_PROCESS_MSG_BATCH); + } + + /** + * todo should be removed (kept for benchmarks). + */ + public void preloadEntries2(ClusterNode from, + int p, + Collection entries, + AffinityTopologyVersion topVer + ) throws IgniteCheckedException { + + ctx.kernalContext().diagnostic().beginTrack(DEMANDER_PROCESS_MSG_BATCH); + + try { + + if (entries.isEmpty()) + return; + + GridDhtLocalPartition part = null; + + Map>> cctxMap = new HashMap<>(); + + // Map by context. + for (GridCacheEntryInfo entry : entries) { + GridCacheEntryEx cached = null; + + try { + GridCacheContext cctx0 = grp.sharedGroup() ? ctx.cacheContext(entry.cacheId()) : grp.singleCacheContext(); + + if (part == null) + part = cctx0.topology().localPartition(p); + + if (cctx0 == null) + return; + + if (cctx0.isNear()) + cctx0 = cctx0.dhtCache().context(); + + final GridCacheContext cctx = cctx0; + + cached = cctx.cache().entryEx(entry.key()); + // todo ensure free space + // todo check obsolete + + if (log.isTraceEnabled()) + log.trace("Rebalancing key [key=" + entry.key() + ", part=" + p + ", node=" + from.id() + ']'); + + List> entriesList = cctxMap.get(cctx.cacheId()); + + if (entriesList == null) { + cctx.continuousQueries().getListenerReadLock().lock(); + + cctxMap.put(cctx.cacheId(), entriesList = new ArrayList<>()); + } + + cached.lockEntry(); + + entriesList.add(new T2<>((GridCacheMapEntry)cached, entry)); + } + catch (GridDhtInvalidPartitionException ignored) { + if (log.isDebugEnabled()) + log.debug("Partition became invalid during rebalancing (will ignore): " + p); + + return; + } + } + + try { + for (Map.Entry>> mapEntries : cctxMap.entrySet()) { + GridCacheContext cctx = ctx.cacheContext(mapEntries.getKey()); + + // todo ticket + assert !cctx.mvccEnabled() : "MVCC caches not supported"; + + // todo think about sorting keys. + List keys = new ArrayList<>(mapEntries.getValue().size()); + + Map keyToEntry = new HashMap<>(U.capacity(mapEntries.getValue().size())); + + for (T2 pair : mapEntries.getValue()) { + KeyCacheObject key = pair.getValue().key(); + + keys.add(key); + + keyToEntry.put(key, pair.getValue()); + } + + cctx.offheap().updateBatch(cctx, keys, part, keyToEntry); + } + } + finally { + for (Map.Entry>> mapEntries : cctxMap.entrySet()) { + GridCacheContext cctx = ctx.cacheContext(mapEntries.getKey()); + + // todo ticket + assert !cctx.mvccEnabled() : "MVCC caches not supported"; + + assert cctx != null : mapEntries.getKey(); + + cctx.continuousQueries().getListenerReadLock().unlock(); + + for (T2 e : mapEntries.getValue()) { + try { + GridCacheEntryInfo info = e.get2(); + + long expTime = info.ttl() < 0 ? CU.toExpireTime(info.ttl()) : info.ttl(); + + // log.info("finish preload: " + info.key().hashCode()); + + e.get1().finishPreload(info.value(), expTime, info.ttl(), info.version(), topVer, + cctx.isDrEnabled() ? DR_PRELOAD : DR_NONE, null, true); + + } + finally { + e.get1().unlockEntry(); + + // todo record rebalance event + e.get1().touch(topVer); + } + } + + for (GridCacheContext cctx0 : grp.caches()) { + if (cctx0.statisticsEnabled()) + cctx0.cache().metrics0().onRebalanceKeysReceived(mapEntries.getValue().size()); + } + } + + } + } finally { + ctx.kernalContext().diagnostic().endTrack(DEMANDER_PROCESS_MSG_BATCH); } } @@ -893,6 +1251,8 @@ private boolean preloadEntry( ) throws IgniteCheckedException { assert ctx.database().checkpointLockIsHeldByThread(); +// ctx.kernalContext().diagnostic().beginTrack(PRELOAD_ENTRY); + try { GridCacheEntryEx cached = null; @@ -962,6 +1322,9 @@ else if (log.isTraceEnabled()) throw new IgniteCheckedException("Failed to cache rebalanced entry (will stop rebalancing) [local=" + ctx.localNode() + ", node=" + from.id() + ", key=" + entry.key() + ", part=" + p + ']', e); } +// finally { +// ctx.kernalContext().diagnostic().endTrack(PRELOAD_ENTRY); +// } return true; } @@ -1044,8 +1407,20 @@ public static class RebalanceFuture extends GridFutureAdapter { this.rebalanceId = rebalanceId; ctx = grp.shared(); + +// ctx.kernalContext().diagnostic().beginTrack(TOTAL); } +// @Override protected boolean onDone(@Nullable Boolean res, @Nullable Throwable err, boolean cancel) { +// if (ctx != null) { // can be dummy +// ctx.kernalContext().diagnostic().endTrack(TOTAL); +// +// ctx.kernalContext().diagnostic().printStats(); +// } +// +// return super.onDone(res, err, cancel); +// } + /** * Dummy future. Will be done by real one. */ @@ -1165,6 +1540,8 @@ private void cleanupRemoteContexts(UUID nodeId) { for (int idx = 0; idx < ctx.gridConfig().getRebalanceThreadPoolSize(); idx++) { d.topic(GridCachePartitionExchangeManager.rebalanceTopic(idx)); + d.timestamp(U.currentTimeMillis()); + ctx.io().sendOrderedMessage(node, GridCachePartitionExchangeManager.rebalanceTopic(idx), d.convertIfNeeded(node.version()), grp.ioPolicy(), grp.config().getRebalanceTimeout()); } @@ -1204,12 +1581,18 @@ 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 + "]")); + "rebalancing [grp=" + grp.cacheOrGroupName() + + ", supplier=" + nodeId + + ", topVer=" + topologyVersion() + + ", progress=" + (routines - remainingRoutines) + "/" + routines + "," + + ", batch=" + batchPageWriteEnabled + "]")); remaining.remove(nodeId); + + ctx.kernalContext().diagnostic().endTrack(DEMANDER_PROCESS_MSG); + + ctx.kernalContext().diagnostic().endTrack(TOTAL); + ctx.kernalContext().diagnostic().printStats(); } checkIsDone(); @@ -1311,3 +1694,4 @@ private void sendRebalanceFinishedEvent() { } } } + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java index d26d68ff20d0e..57853d09cf1f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java @@ -53,6 +53,9 @@ import org.apache.ignite.spi.IgniteSpiException; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMAND_MSG_SEND; +import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.SUPPLIER_PROCESS_MSG; +import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.TOTAL; /** * Class for supplying partitions to demanding nodes. @@ -184,6 +187,9 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand T3 contextId = new T3<>(nodeId, topicId, demandMsg.topologyVersion()); + //log.info("timestamp " + (U.currentTimeMillis() - demandMsg.timestamp())); +// grp.shared().kernalContext().diagnostic().timeTrack(DEMAND_MSG_SEND, U.currentTimeMillis() - demandMsg.timestamp()); + if (demandMsg.rebalanceId() < 0) { // Demand node requested context cleanup. synchronized (scMap) { SupplyContext sctx = scMap.get(contextId); @@ -253,6 +259,10 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand long maxBatchesCnt = grp.config().getRebalanceBatchesPrefetchCount(); if (sctx == null) { + grp.shared().kernalContext().diagnostic().beginTrack(TOTAL); + + grp.shared().kernalContext().diagnostic().beginTrack(SUPPLIER_PROCESS_MSG); + if (log.isDebugEnabled()) log.debug("Starting supplying rebalancing [" + supplyRoutineInfo(topicId, nodeId, demandMsg) + ", fullPartitions=" + S.compact(demandMsg.partitions().fullSet()) + @@ -444,6 +454,10 @@ else if (iter.isPartitionMissing(p)) { if (log.isInfoEnabled()) log.info("Finished supplying rebalancing [" + supplyRoutineInfo(topicId, nodeId, demandMsg) + "]"); + + grp.shared().kernalContext().diagnostic().endTrack(SUPPLIER_PROCESS_MSG); + grp.shared().kernalContext().diagnostic().endTrack(TOTAL); + grp.shared().kernalContext().diagnostic().printStats(); } catch (Throwable t) { if (grp.shared().kernalContext().isStopping()) @@ -516,6 +530,8 @@ private boolean reply( if (log.isDebugEnabled()) log.debug("Send next supply message [" + supplyRoutineInfo(topicId, demander.id(), demandMsg) + "]"); +// supplyMsg.timestamp(U.currentTimeMillis()); + grp.shared().io().sendOrderedMessage(demander, demandMsg.topic(), supplyMsg, grp.ioPolicy(), demandMsg.timeout()); // Throttle preloading. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java index 7e281e59a7e1e..67dbfb6031a26 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java @@ -85,6 +85,8 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple @GridDirectMap(keyType = int.class, valueType = long.class) private Map keysPerCache; + private long timestamp; + /** * @param rebalanceId Rebalance id. * @param grpId Cache group ID. @@ -342,6 +344,11 @@ public int size() { writer.incrementState(); + case 13: + if (!writer.writeLong("timestamp", timestamp)) + return false; + + writer.incrementState(); } return true; @@ -431,11 +438,28 @@ public int size() { reader.incrementState(); + case 13: + timestamp = reader.readLong("timestamp"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } return reader.afterMessageRead(GridDhtPartitionSupplyMessage.class); } + + public void timestamp(long timestamp) { + this.timestamp = timestamp; + } + + public long timestamp() { + return timestamp; + } + /** {@inheritDoc} */ @Override public short directType() { return 114; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java index b209cdba342ad..b7a7efe571c5e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java @@ -45,6 +45,8 @@ public class GridDhtPartitionSupplyMessageV2 extends GridDhtPartitionSupplyMessa /** Supplying process error bytes. */ private byte[] errBytes; + + /** * Default constructor. */ @@ -101,7 +103,7 @@ public GridDhtPartitionSupplyMessageV2( } switch (writer.state()) { - case 13: + case 14: if (!writer.writeByteArray("errBytes", errBytes)) return false; @@ -123,19 +125,19 @@ public GridDhtPartitionSupplyMessageV2( return false; switch (reader.state()) { - case 13: + case 14: errBytes = reader.readByteArray("errBytes"); if (!reader.isLastRead()) return false; reader.incrementState(); - } return reader.afterMessageRead(GridDhtPartitionSupplyMessageV2.class); } + /** {@inheritDoc} */ @Nullable @Override public Throwable error() { return err; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java index e92a240fafb3b..65cb77c8dc3a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java @@ -58,6 +58,7 @@ import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.RENTING; +import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.SUPPLIER_PROCESS_MSG; /** * DHT cache preloader. @@ -387,11 +388,15 @@ private List remoteOwners(int p, AffinityTopologyVersion topVer) { if (!enterBusy()) return; + grp.shared().kernalContext().diagnostic().beginTrack(SUPPLIER_PROCESS_MSG); + try { supplier.handleDemandMessage(idx, id, d); } finally { leaveBusy(); + + grp.shared().kernalContext().diagnostic().endTrack(SUPPLIER_PROCESS_MSG); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java index 85be9d259df3d..30ec730b96245 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java @@ -334,8 +334,13 @@ protected void readFullRow( byte[] bytes = PageUtils.getBytes(addr, off, len); off += len; + try { + key = coctx.kernalContext().cacheObjects().toKeyCacheObject(coctx, type, bytes); + } catch (RuntimeException e) { + System.out.println("(critical fail) len=" + len); - key = coctx.kernalContext().cacheObjects().toKeyCacheObject(coctx, type, bytes); + throw e; + } if (rowData == RowData.KEY_ONLY) return; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java index 35dd3c46ee431..ab9bf86913bdd 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.Collection; import java.util.Random; import java.util.concurrent.ThreadLocalRandom; import org.apache.ignite.IgniteCheckedException; @@ -307,6 +308,26 @@ 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, + Collection arg, + R lockFailed, + IoStatisticsHolder statHolder) throws IgniteCheckedException { + return PageHandler.writePageBatch(pageMem, grpId, pageId, this, h, init, wal, null, arg, 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 f78428d055e52..2e6815a26921d 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; @@ -29,6 +30,7 @@ import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; import javax.cache.processor.EntryProcessor; +import javax.naming.OperationNotSupportedException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.failure.FailureContext; @@ -51,11 +53,13 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdatePartitionDataRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionDestroyRecord; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.BatchedCacheEntries; import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; +import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; import org.apache.ignite.internal.processors.cache.GridCacheTtlManager; import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl; import org.apache.ignite.internal.processors.cache.KeyCacheObject; @@ -83,6 +87,7 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; import org.apache.ignite.internal.processors.cache.tree.CacheDataRowStore; import org.apache.ignite.internal.processors.cache.tree.CacheDataTree; +import org.apache.ignite.internal.processors.cache.tree.DataRow; import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree; import org.apache.ignite.internal.processors.cache.tree.PendingRow; import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateResult; @@ -97,6 +102,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING; @@ -1486,7 +1492,8 @@ private CacheDataStore init0(boolean checkExists) throws IgniteCheckedException null, ctx.wal(), reuseRoot.pageId().pageId(), - reuseRoot.isAllocated()) { + reuseRoot.isAllocated(), + ctx.kernalContext()) { /** {@inheritDoc} */ @Override protected long allocatePageNoReuse() throws IgniteCheckedException { assert grp.shared().database().checkpointLockIsHeldByThread(); @@ -1734,6 +1741,13 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { } } + @Override + public @NotNull DataRow makeDataRow(KeyCacheObject key, CacheObject val, GridCacheVersion ver, long expireTime, + int cacheId) { + //todo + throw new RuntimeException("fuck off"); + } + /** {@inheritDoc} */ @Override public int partId() { return partId; @@ -1945,6 +1959,39 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { delegate.update(cctx, key, val, ver, expireTime, oldRow); } + /** {@inheritDoc} */ + @Override public void updateBatch( + BatchedCacheEntries batch + ) throws IgniteCheckedException { + assert ctx.database().checkpointLockIsHeldByThread(); + + CacheDataStore delegate = init0(false); + + delegate.updateBatch(batch); + } + +// /** {@inheritDoc} */ +// @Override public void insertDataRows(Collection rows) throws IgniteCheckedException { +// assert ctx.database().checkpointLockIsHeldByThread(); +// +// CacheDataStore delegate = init0(false); +// +// delegate.insertDataRows(rows); +// } + + /** {@inheritDoc} */ + @Override public void updateBatch( + GridCacheContext cctx, + List keys, + Map items + ) throws IgniteCheckedException { + assert ctx.database().checkpointLockIsHeldByThread(); + + CacheDataStore delegate = init0(false); + + delegate.updateBatch(cctx, keys, items); + } + /** {@inheritDoc} */ @Override public boolean mvccInitialValue( GridCacheContext cctx, @@ -2121,6 +2168,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 7fc70d0b8923d..b4a6a4a219f94 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 @@ -248,13 +248,14 @@ protected void initPageMemoryDataStructures(DataStorageConfiguration dbCfg) thro boolean persistenceEnabled = memPlcCfg.isPersistenceEnabled(); CacheFreeListImpl freeList = new CacheFreeListImpl(0, - cctx.igniteInstanceName(), + memPlc.config().getName(), memMetrics, memPlc, null, persistenceEnabled ? cctx.wal() : null, 0L, - true); + true, + cctx.kernalContext()); freeListMap.put(memPlcCfg.getName(), freeList); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java index 91fd2070cc048..d37c12460b4c4 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; @@ -27,6 +28,10 @@ import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.stat.IoStatisticsHolder; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_OFFHEAP_INVOKE_INSERT_FREELIST; + +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_FREELIST_REMOVE; + /** * Data store for H2 rows. */ @@ -80,11 +85,12 @@ public void removeRow(long link, IoStatisticsHolder statHolder) throws IgniteChe freeList.removeDataRowByLink(link, statHolder); else { ctx.database().checkpointReadLock(); - +// ctx.kernalContext().diagnostic().beginTrack(PRELOAD_FREELIST_REMOVE); try { freeList.removeDataRowByLink(link, statHolder); } finally { +// ctx.kernalContext().diagnostic().endTrack(PRELOAD_FREELIST_REMOVE); ctx.database().checkpointReadUnlock(); } } @@ -111,6 +117,27 @@ public void addRow(CacheDataRow row, IoStatisticsHolder statHolder) throws Ignit } } + /** + * @param rows Rows. + * @throws IgniteCheckedException If failed. + */ + public void addRows(Collection rows, IoStatisticsHolder statHolder) throws IgniteCheckedException { + if (!persistenceEnabled) + freeList.insertDataRows(rows, statHolder); + else { + ctx.database().checkpointReadLock(); + + try { + freeList.insertDataRows(rows, statHolder); + } + finally { + ctx.database().checkpointReadUnlock(); + } + } + } + + + /** * @param link Row link. * @param row New row data. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java index 60aefb927ce6f..254c38abbaf04 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,9 +17,17 @@ package org.apache.ignite.internal.processors.cache.persistence.freelist; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicReferenceArray; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.pagemem.PageIdAllocator; import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.pagemem.PageUtils; @@ -30,6 +38,7 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageUpdateRecord; import org.apache.ignite.internal.processors.cache.persistence.DataRegion; import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl; +import org.apache.ignite.internal.processors.cache.persistence.IndexStorageImpl; import org.apache.ignite.internal.processors.cache.persistence.Storable; import org.apache.ignite.internal.processors.cache.persistence.evict.PageEvictionTracker; import org.apache.ignite.internal.processors.cache.persistence.tree.io.AbstractDataPageIO; @@ -42,8 +51,16 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler; import org.apache.ignite.internal.stat.IoStatisticsHolder; import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp; +import org.apache.ignite.internal.util.lang.GridTuple3; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.T3; import org.apache.ignite.internal.util.typedef.internal.U; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH_ALLOC_PAGE; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH_BIN_INSERT; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH_BIN_PACK; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.DEMANDER_PROCESS_MSG_BATCH_BIN_SEARCH; + /** */ public abstract class AbstractFreeList extends PagesList implements FreeList, ReuseList { @@ -86,6 +103,9 @@ public abstract class AbstractFreeList extends PagesList imp /** */ private final PageEvictionTracker evictionTracker; + /** */ + private final GridKernalContext ctx; + /** * */ @@ -133,12 +153,15 @@ private final class UpdateRowHandler extends PageHandler { /** */ private final PageHandler writeRow = new WriteRowHandler(); + /** */ + private final PageHandler writeRows = new WriteRowHandlerBatch(); + /** * */ - 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 +169,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; @@ -156,18 +206,11 @@ private final class WriteRowHandler extends PageHandler { assert oldFreeSpace > 0 : oldFreeSpace; // If the full row does not fit into this page write only a fragment. +// System.out.println(">xxx> free=" + oldFreeSpace + ", rowSize=" + rowSize + " hash=" + row.hashCode()); + 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 +228,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 +268,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 +297,83 @@ private int addRowFragment( return written + payloadSize; } + + /** + * Put page to 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 WriteRowHandlerBatch extends WriteRowHandler { + /** {@inheritDoc} */ + @Override public Integer runBatch( + int cacheId, + long pageId, + long page, + long pageAddr, + PageIO io, + Boolean walPlc, + Collection args, + IoStatisticsHolder statHolder + ) throws IgniteCheckedException { + int maxPayloadSize = pageSize() - AbstractDataPageIO.MIN_DATA_PAGE_OVERHEAD; + + AbstractDataPageIO iox = (AbstractDataPageIO)io; + + // todo !! DO NOT FORGET WAL DELTA !! + if (iox.getFreeSpace(pageAddr) == maxPayloadSize) { + // todo save links for WAL + + iox.addRows(pageMem, pageId, pageAddr, args, pageSize()); + + // todo update wal + } + else { + for (T row : args) { + assert iox.getFreeSpace(pageAddr) > 0 : iox.getFreeSpace(pageAddr); + + int size = row.size(); + + int written = size > maxPayloadSize ? + addRowFragment(pageId, page, pageAddr, iox, row, size - (size % maxPayloadSize), 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(); + + evictionTracker.touchPage(pageId); + } + } + + // return page to freelist if needed + putPage((AbstractDataPageIO)io, pageId, page, pageAddr, statHolder); + + return COMPLETE; + } } /** */ @@ -345,7 +465,8 @@ public AbstractFreeList( ReuseList reuseList, IgniteWriteAheadLogManager wal, long metaPageId, - boolean initNew) throws IgniteCheckedException { + boolean initNew, + GridKernalContext ctx) throws IgniteCheckedException { super(cacheId, name, memPlc.pageMemory(), BUCKETS, wal, metaPageId); rmvRow = new RemoveRowHandler(cacheId == 0); @@ -374,6 +495,8 @@ public AbstractFreeList( this.memMetrics = memMetrics; init(metaPageId, initNew); + + this.ctx = ctx; } /** @@ -435,7 +558,14 @@ public long freeSpace() { log.info("FreeList [name=" + name + ", buckets=" + BUCKETS + ", dataPages=" + dataPages + - ", reusePages=" + bucketsSize[REUSE_BUCKET].longValue() + "]"); + ", reusePages=" + bucketsSize[REUSE_BUCKET].longValue() + "" + + ", bucket[0]=" + bucketsSize[0] + + ", bucket[1]=" + bucketsSize[1] + + ", bucket[2]=" + bucketsSize[2] + + ", bucket[3]=" + bucketsSize[3] + + ", bucket[4]=" + bucketsSize[4] + + ", bucket[5]=" + bucketsSize[5] + + "]"); } } @@ -509,6 +639,260 @@ else if (PageIdUtils.tag(pageId) != PageIdAllocator.FLAG_DATA) while (written != COMPLETE); } + /** {@inheritDoc} */ + @Override public void insertDataRows(Collection rows, IoStatisticsHolder statHolder) throws IgniteCheckedException { + // 1. split into 3 bags + // A. Large objects. + // B1. Tails of large objects + // B2. small objects + + // Max bytes per data page. + int maxPayloadSize = pageSize() - AbstractDataPageIO.MIN_DATA_PAGE_OVERHEAD; + + int maxRowsPerPage = IndexStorageImpl.MAX_IDX_NAME_LEN; + + // Data rows <-> count of pages needed + List largeRows = new ArrayList<>(16); + + // other objects + List regularRows = new ArrayList<>(16); + + for (T dataRow : rows) { + if (dataRow.size() < maxPayloadSize) + regularRows.add(dataRow); + else { + largeRows.add(dataRow); + + int tailSize = dataRow.size() % maxPayloadSize; + + if (tailSize > 0) + regularRows.add(dataRow); + } + } + + // Writing large objects. + for (T row : largeRows) { + int rowSize = row.size(); + + int written = 0; + + do { + if (written != 0) + memMetrics.incrementLargeEntriesPages(); + + int remaining = rowSize - written; + + long pageId; + + if (remaining >= MIN_SIZE_FOR_DATA_PAGE) + pageId = takeEmptyPage(REUSE_BUCKET, ioVersions(), statHolder); + else + break; + + 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. + } + while (written != COMPLETE); + } + + List dataRows = new ArrayList<>(maxRowsPerPage); + + int remainPageSpace = 0; + + long pageId = 0; + + AbstractDataPageIO initIo = null; + + for (int i = 0; i < regularRows.size(); i++) { + T row = regularRows.get(i); + + boolean tail = i == (regularRows.size() - 1); + + boolean fragment = row.size() > maxPayloadSize; + + int payloadSize = fragment ? (row.size() % maxPayloadSize) + 12 : row.size() + 4; + + // There is no space left on this page. + if (((remainPageSpace - payloadSize) < 0 || dataRows.size() == maxRowsPerPage) && pageId != 0) { + int written = write(pageId, writeRows, initIo, dataRows, FAIL_I, statHolder); + + assert written == COMPLETE : written; + + initIo = null; + remainPageSpace = 0; + pageId = 0; + dataRows.clear(); + } + + dataRows.add(row); + + if (pageId == 0) { + int minBucket = bucket(payloadSize, false) + 1; + + if (payloadSize != MIN_SIZE_FOR_DATA_PAGE) { + for (int b = REUSE_BUCKET - 1; b >= minBucket; b--) { + pageId = takeEmptyPage(b, ioVersions(), statHolder); + + if (pageId != 0L) { + remainPageSpace = (b << shift) + 4; // todo explain "+4"? + + 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()); + + if (remainPageSpace == 0) + remainPageSpace = maxPayloadSize; + } + + remainPageSpace -= payloadSize; + + if (tail) { + int written; + + if (dataRows.size() == 1) { + written = fragment ? row.size() - (rows.size() % maxPayloadSize) : 0; + + written = write(pageId, writeRows, initIo, row, written, FAIL_I, statHolder); + } else + written = write(pageId, writeRows, initIo, dataRows, FAIL_I, statHolder); + + assert written == COMPLETE : written; + } + + } + +// for (T2, Integer> bin : bins) { +// long pageId = 0; +// +// int remaining = bin.get2(); +// +//// ctx.diagnostic().beginTrack(DEMANDER_PROCESS_MSG_BATCH_BIN_SEARCH); +// +// int buck = bucket(remaining, false) + 1; +// +// for (int b = remaining < MIN_SIZE_FOR_DATA_PAGE ? buck : REUSE_BUCKET; b < BUCKETS; b++) { +// pageId = takeEmptyPage(b, ioVersions(), statHolder); +// +// if (pageId != 0L) +// break; +// } +// +//// ctx.diagnostic().endTrack(DEMANDER_PROCESS_MSG_BATCH_BIN_SEARCH); +// +//// ctx.diagnostic().beginTrack(DEMANDER_PROCESS_MSG_BATCH_ALLOC_PAGE); +// +// T row = bin.get1().get(0); +// +// AbstractDataPageIO initIo = null; +// +// 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()); +// +//// ctx.diagnostic().endTrack(DEMANDER_PROCESS_MSG_BATCH_ALLOC_PAGE); +//// +//// ctx.diagnostic().beginTrack(DEMANDER_PROCESS_MSG_BATCH_BIN_INSERT); +// +// int written = write(pageId, writeRows, initIo, bin.get1(), FAIL_I, statHolder); +// +//// ctx.diagnostic().endTrack(DEMANDER_PROCESS_MSG_BATCH_BIN_INSERT); +// +// assert written == COMPLETE : written; +// } + } + + // todo move out + // todo experiment with "bestfit" approach + private List, Integer>> binPack(List> rows, int cap) { + // Initialize result (Count of bins) + int cnt = 0; + + // Result. + List, Integer>> bins = new ArrayList<>(rows.size()); + + // Create an array to store remaining space in bins + // there can be at most n bins + int[] remains = new int[rows.size()]; + + // Place items one by one + for (int i = (rows.size() - 1); i >= 0; i--) { + // Find the first bin that can accommodate weight[i] + int j; + + T3 t3 = rows.get(i); + + int size = t3.get1() + (t3.get3() ? 12 : 4); // + inner pointer + pageId (for head of large row) + + for (j = 0; j < cnt; j++) { + if (remains[j] >= size) { + remains[j] -= size; + + T row = rows.get(i).get2(); + + bins.get(j).get1().add(row); + bins.get(j).set2(bins.get(j).get2() + size); + +// binMap.put(row, j); + + break; + } + } + + // If no bin could accommodate sizes[i]. + if (j == cnt) { + remains[cnt] = cap - size; + + // todo remove magic number + List list = new ArrayList<>(16); + + bins.add(new T2<>(list, size)); + + T row = rows.get(i).get2(); + + list.add(row); + +// binMap.put(row, j); + + cnt++; + } + } + + return bins; + } + /** * @param reusedPageId Reused page id. * @param partId Partition id. @@ -579,7 +963,7 @@ private long initReusedPage(long reusedPageId, int partId, long nextLink = write(pageId, rmvRow, bag, itemId, FAIL_L, statHolder); - assert nextLink != FAIL_L; // Can't fail here. + assert nextLink != FAIL_L : pageId; // Can't fail here. while (nextLink != 0L) { memMetrics.decrementLargeEntriesPages(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/CacheFreeListImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/CacheFreeListImpl.java index 625c0b15d9d56..beab554d978dc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/CacheFreeListImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/CacheFreeListImpl.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.cache.persistence.freelist; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; @@ -46,8 +47,8 @@ public class CacheFreeListImpl extends AbstractFreeList { */ public CacheFreeListImpl(int cacheId, String name, DataRegionMetricsImpl regionMetrics, DataRegion dataRegion, ReuseList reuseList, - IgniteWriteAheadLogManager wal, long metaPageId, boolean initNew) throws IgniteCheckedException { - super(cacheId, name, regionMetrics, dataRegion, reuseList, wal, metaPageId, initNew); + IgniteWriteAheadLogManager wal, long metaPageId, boolean initNew, GridKernalContext ctx) throws IgniteCheckedException { + super(cacheId, name, regionMetrics, dataRegion, reuseList, wal, metaPageId, initNew, ctx); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java index e28d421bdf063..f49addab848aa 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,11 +17,15 @@ 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; import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler; +import org.apache.ignite.internal.processors.cache.tree.DataRow; import org.apache.ignite.internal.stat.IoStatisticsHolder; +import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.lang.IgniteInClosure; /** */ @@ -32,6 +36,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/metastorage/MetaStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetaStorage.java index 7ff82577fd6b3..eb72a77b5837d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetaStorage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetaStorage.java @@ -626,7 +626,7 @@ public class FreeListImpl extends AbstractFreeList { FreeListImpl(int cacheId, String name, DataRegionMetricsImpl regionMetrics, DataRegion dataRegion, ReuseList reuseList, IgniteWriteAheadLogManager wal, long metaPageId, boolean initNew) throws IgniteCheckedException { - super(cacheId, name, regionMetrics, dataRegion, reuseList, wal, metaPageId, initNew); + super(cacheId, name, regionMetrics, dataRegion, reuseList, wal, metaPageId, initNew, cctx.kernalContext()); } /** {@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 e9b72482fe872..0a3addeeeefc4 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 @@ -19,11 +19,14 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.Iterator; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import javax.naming.OperationNotSupportedException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; @@ -46,6 +49,8 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.RemoveRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.ReplaceRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.SplitExistingPageRecord; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; import org.apache.ignite.internal.processors.cache.persistence.DataStructure; import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO; @@ -58,6 +63,7 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler; import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandlerWrapper; +import org.apache.ignite.internal.processors.cache.tree.DataRow; import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.stat.IoStatisticsHolder; import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp; @@ -72,6 +78,7 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteInClosure; import org.jetbrains.annotations.Nullable; +import sun.reflect.generics.reflectiveObjects.NotImplementedException; import static org.apache.ignite.IgniteSystemProperties.IGNITE_BPLUS_TREE_LOCK_RETRIES; import static org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.Bool.DONE; @@ -978,7 +985,7 @@ private GridCursor findLowerUnbounded(L upper, TreeRowClosure c, Object /** * Check if the tree is getting destroyed. */ - private void checkDestroyed() { + protected void checkDestroyed() { if (destroyed.get()) throw new IllegalStateException("Tree is being concurrently destroyed: " + getName()); } @@ -1025,6 +1032,41 @@ public final GridCursor find(L lower, L upper, TreeRowClosure c, Object } } + +// /** +// * @param lower Lower bound inclusive or {@code null} if unbounded. +// * @param upper Upper bound inclusive or {@code null} if unbounded. +// * @param c Filter closure. +// * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row. +// * @return Cursor. +// * @throws IgniteCheckedException If failed. +// */ +// public final GridCursor findAll(List filteredRows, TreeRowClosure c, Object x) throws IgniteCheckedException { +// checkDestroyed(); +// +// lower, L upper +// +// try { +// if (lower == null) +// return findLowerUnbounded(upper, c, x); +// +// ForwardAllCursor cursor = new ForwardAllCursor(lower, upper, c, x); +// +// cursor.find(); +// +// return cursor; +// } +// catch (IgniteCheckedException e) { +// throw new IgniteCheckedException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e); +// } +// catch (RuntimeException | AssertionError e) { +// throw new CorruptedTreeException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e); +// } +// finally { +// checkDestroyed(); +// } +// } + /** * @param lower Lower bound inclusive. * @param upper Upper bound inclusive. @@ -1822,6 +1864,10 @@ public final boolean removex(L row) throws IgniteCheckedException { } } + @Override public void invokeAll(List rows, Object z, InvokeAllClosure c) throws IgniteCheckedException { + throw new UnsupportedOperationException(); + } + /** * @param x Invoke operation. * @param pageId Page ID. @@ -3646,6 +3692,10 @@ public Result tryReplace(long pageId, long page, long fwdId, int lvl) throws Ign } } +// public final class InvokeBatch extends Invoke { +// +// } + /** * Invoke operation. */ @@ -5486,6 +5536,149 @@ private void iterate() throws IgniteCheckedException { } } + /** + * Forward cursor. + */ + private final class ForwardAllCursor extends AbstractForwardCursor implements GridCursor { + /** */ + final Object x; + + /** */ + private T[] rows = (T[])EMPTY; + + /** */ + private int row = -1; + + /** */ + private final TreeRowClosure c; + + /** + * @param lowerBound Lower bound. + * @param upperBound Upper bound. + * @param c Filter closure. + * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row. + */ + ForwardAllCursor(L lowerBound, L upperBound, TreeRowClosure c, Object x) { + super(lowerBound, upperBound); + + this.c = c; + this.x = x; + } + + /** {@inheritDoc} */ + @Override boolean fillFromBuffer0(long pageAddr, BPlusIO io, int startIdx, int cnt) throws IgniteCheckedException { + if (startIdx == -1) { + if (lowerBound != null) + startIdx = findLowerBound(pageAddr, io, cnt); + else + startIdx = 0; + } + + if (upperBound != null && cnt != startIdx) + cnt = findUpperBound(pageAddr, io, startIdx, cnt); + + int cnt0 = cnt - startIdx; + + if (cnt0 == 0) + return false; + + if (rows == EMPTY) + rows = (T[])new Object[cnt0]; + + int resCnt = 0; + + for (int idx = startIdx; idx < cnt; idx++) { + //todo here we can filter values + if (c == null || c.apply(BPlusTree.this, io, pageAddr, idx)) { + T locRow = getRow(io, pageAddr, idx, x); + + + + rows = GridArrays.set(rows, resCnt++, locRow); + } + } + + if (resCnt == 0) { + rows = (T[])EMPTY; + + return false; + } + + GridArrays.clearTail(rows, resCnt); + + return true; + } + + /** {@inheritDoc} */ + @Override boolean reinitialize0() throws IgniteCheckedException { + return next(); + } + + /** {@inheritDoc} */ + @Override void onNotFound(boolean readDone) { + if (readDone) + rows = null; + else { + if (rows != EMPTY) { + assert rows.length > 0; // Otherwise it makes no sense to create an array. + + // Fake clear. + rows[0] = null; + } + } + } + + /** {@inheritDoc} */ + @Override void init0() { + row = -1; + } + + /** {@inheritDoc} */ + @Override public boolean next() throws IgniteCheckedException { + if (rows == null) + return false; + + if (++row < rows.length && rows[row] != null) { + clearLastRow(); // Allow to GC the last returned row. + + return true; + } + + T lastRow = clearLastRow(); + + row = 0; + + return nextPage(lastRow); + } + + /** + * @return Cleared last row. + */ + private T clearLastRow() { + if (row == 0) + return null; + + int last = row - 1; + + T r = rows[last]; + + assert r != null; + + rows[last] = null; + + return r; + } + + /** {@inheritDoc} */ + @Override public T get() { + T r = rows[row]; + + assert r != null; + + return r; + } + } + /** * Forward cursor. */ 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 6176eeb0bd281..abb1363d5d427 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 @@ -31,6 +31,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; @@ -802,19 +803,26 @@ 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); int directCnt = getDirectCount(pageAddr); + +// System.out.println(">xxx> pageAddr="+pageAddr+", but dirCnt="+directCnt); + int indirectCnt = getIndirectCount(pageAddr); +// System.out.println(">xxx> pageAddr=" + pageAddr + ", but dirCnt=" + directCnt + " indirectCnt=" + indirectCnt); + int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize); writeRowData(pageAddr, dataOff, rowSize, row, true); int itemId = addItem(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize); +// System.out.println(">xxx> link pageId="+pageId + ", itemId="+itemId); + setLinkByPageId(row, pageId, itemId); } @@ -893,6 +901,8 @@ private int addItem(final long pageAddr, int itemId = insertItem(pageAddr, dataOff, directCnt, indirectCnt, pageSize); +// System.out.println(">xxx> pageAddr=" + pageAddr + "itemId=" + itemId + ", off=" + dataOff + ", cnt=" + directCnt + ", indcnt=" + indirectCnt); + assert checkIndex(itemId) : itemId; assert getIndirectCount(pageAddr) <= getDirectCount(pageAddr); @@ -969,6 +979,75 @@ 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 { + // todo code duplication (3 times!) + int maxPayloadSIze = pageSize - MIN_DATA_PAGE_OVERHEAD; + int dataOff = pageSize; + int cnt = 0; + int written = 0; + + for (T row : rows) { + boolean fragment = row.size() > maxPayloadSIze; + + int payloadSize = row.size() % maxPayloadSIze; + + assert payloadSize <= getFreeSpace(pageAddr) : "can't call addRow if not enough space for the whole row"; + + int sizeSetup = fragment ? SHOW_PAYLOAD_LEN | SHOW_LINK | SHOW_ITEM : SHOW_PAYLOAD_LEN | SHOW_ITEM; + + int fullEntrySize = getPageEntrySize(payloadSize, sizeSetup); + + 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()); + + // todo is it 0? + 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; + } + + 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. * @@ -1104,6 +1183,9 @@ private int insertItem(long pageAddr, int dataOff, int directCnt, int indirectCn setItem(pageAddr, directCnt, directItemFromOffset(dataOff)); setDirectCount(pageAddr, directCnt + 1); + +// System.out.println("pageAddr " + pageAddr + " directCnt="+getDirectCount(pageAddr) + " free="+getFreeSpace(pageAddr)); + assert getDirectCount(pageAddr) == directCnt + 1; return directCnt; // Previous directCnt will be our itemId. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java index 5ab1bf38dbc18..72302bf36ecad 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.Collection; 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 runBatch( + int cacheId, + long pageId, + long page, + long pageAddr, + PageIO io, + Boolean walPlc, + Collection args, + IoStatisticsHolder statHolder + ) throws IgniteCheckedException { + // todo + throw new UnsupportedOperationException(); + } + /** * @param cacheId Cache ID. * @param pageId Page ID. @@ -308,6 +335,74 @@ 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, + Collection args, + 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.runBatch(grpId, pageId, page, pageAddr, init, walPlc, args, 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/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java index dbd2f14a3040a..7fa0be95450d4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java @@ -1781,8 +1781,8 @@ private SegmentArchiveResult archiveSegment(long absIdx) throws StorageException File dstFile = new File(walArchiveDir, name); - if (log.isInfoEnabled()) - log.info("Starting to copy WAL segment [absIdx=" + absIdx + ", segIdx=" + segIdx + + if (log.isDebugEnabled()) + log.debug("Starting to copy WAL segment [absIdx=" + absIdx + ", segIdx=" + segIdx + ", origFile=" + origFile.getAbsolutePath() + ", dstFile=" + dstFile.getAbsolutePath() + ']'); try { @@ -1804,8 +1804,8 @@ private SegmentArchiveResult archiveSegment(long absIdx) throws StorageException ", dstFile=" + dstTmpFile.getAbsolutePath() + ']', e); } - if (log.isInfoEnabled()) - log.info("Copied file [src=" + origFile.getAbsolutePath() + + if (log.isDebugEnabled()) + log.debug("Copied file [src=" + origFile.getAbsolutePath() + ", dst=" + dstFile.getAbsolutePath() + ']'); return new SegmentArchiveResult(absIdx, origFile, dstFile); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java index e56b05baefaa2..3b3f7178fb946 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java @@ -61,6 +61,7 @@ import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor; import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor; import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor; +import org.apache.ignite.internal.processors.diag.DiagnosticProcessor; import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.processors.hadoop.HadoopHelper; import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter; @@ -495,6 +496,11 @@ protected IgniteConfiguration prepareIgniteConfiguration() { return null; } + /** {@inheritDoc} */ + @Override public DiagnosticProcessor diagnostic() { + return null; + } + /** {@inheritDoc} */ @Override public void printMemoryStats() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java index 8949a2f5dd1ad..21bbe0902da6f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java @@ -137,6 +137,8 @@ import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL; import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS; import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_INDEXING_REMOVE; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.PRELOAD_INDEXING_STORE; /** * Query and index manager. @@ -388,6 +390,8 @@ public void store(CacheDataRow newRow, @Nullable CacheDataRow prevRow, if (!enterBusy()) throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); +// cctx.kernalContext().diagnostic().beginTrack(PRELOAD_INDEXING_STORE); + try { if (isIndexingSpiEnabled()) { CacheObjectContext coctx = cctx.cacheObjectContext(); @@ -403,6 +407,8 @@ public void store(CacheDataRow newRow, @Nullable CacheDataRow prevRow, qryProc.store(cctx, newRow, prevRow, prevRowAvailable); } finally { +// cctx.kernalContext().diagnostic().endTrack(PRELOAD_INDEXING_STORE); + invalidateResultCache(); leaveBusy(); @@ -422,6 +428,8 @@ public void remove(KeyCacheObject key, @Nullable CacheDataRow prevRow) if (!enterBusy()) return; // Ignore index update when node is stopping. +// cctx.kernalContext().diagnostic().beginTrack(PRELOAD_INDEXING_REMOVE); + try { if (isIndexingSpiEnabled()) { Object key0 = unwrapIfNeeded(key, cctx.cacheObjectContext()); @@ -434,6 +442,8 @@ public void remove(KeyCacheObject key, @Nullable CacheDataRow prevRow) qryProc.remove(cctx, prevRow); } finally { +// cctx.kernalContext().diagnostic().endTrack(PRELOAD_INDEXING_REMOVE); + invalidateResultCache(); leaveBusy(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java index f0e5c90e375cf..a17940575a18b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java @@ -44,7 +44,7 @@ public AbstractDataLeafIO(int type, int ver, int itemSize) { /** {@inheritDoc} */ @Override public void storeByOffset(long pageAddr, int off, CacheSearchRow row) { - assert row.link() != 0; + assert row.link() != 0 : row.hashCode(); PageUtils.putLong(pageAddr, off, row.link()); off += 8; 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 eba0e7fabccc7..bcfa4a9c862a0 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.List; +import java.util.ListIterator; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.PageUtils; import org.apache.ignite.internal.processors.cache.CacheGroupContext; @@ -38,6 +41,9 @@ import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow; 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.T2; +import org.apache.ignite.internal.util.typedef.T3; import org.apache.ignite.internal.util.typedef.internal.CU; import static org.apache.ignite.internal.pagemem.PageIdUtils.itemId; @@ -121,6 +127,155 @@ public CacheDataRowStore rowStore() { return rowStore; } + /** {@inheritDoc} */ + @Override public void invokeAll(List rows, Object z1, InvokeAllClosure c) throws IgniteCheckedException { + checkDestroyed(); + + // todo No algorithm this is draft implementation only for check that closure is working properly + CacheSearchRow min = rows.iterator().next(); + + CacheSearchRow max = rows.listIterator(rows.size()).previous(); + + List> batch = new ArrayList<>(); + +// GridCursor cur = find(min, max, new TreeRowClosure() { +// +// private final ListIterator rowItr = rows.listIterator(); +// +// private KeyCacheObject lastKey; +// +// private CacheSearchRow lastSearchRow; +// +// @Override +// public boolean apply(BPlusTree tree, BPlusIO io, long pageAddr, int idx) throws IgniteCheckedException { +// CacheDataRow row = getRow(io, pageAddr, idx, null); +// +// KeyCacheObject key = row.key(); +// +// while (rowItr.hasNext() && (lastKey == null || lastKey.hashCode() < key.hashCode())) { +// //tuple.set(OperationType.PUT, null, lastRow); +// batch.add(new T2<>(row, lastSearchRow)); +// +// lastSearchRow = rowItr.next(); +// +// lastKey = lastSearchRow.key(); +// } +// +// ListIterator eqItr = rows.listIterator(rowItr.nextIndex() - 1); +// +// while (lastKey != null && lastKey.hashCode() == key.hashCode()) { +// if (lastKey.equals(key)) { +// batch.add(new T2<>(row, lastSearchRow)); +// +// return true; +// } +// +// lastKey = eqItr.next().key(); +// } +// +// return false; +// } +// }, null); + + final ListIterator rowItr = rows.listIterator(); + + GridCursor cur = find(min, max, null, null); + + while (cur.next()) { + CacheDataRow row = cur.get(); + KeyCacheObject key = row.key(); + + CacheSearchRow lastSearchRow = null; + KeyCacheObject lastKey = null; + + while (rowItr.hasNext() && (lastKey == null || lastKey.hashCode() < key.hashCode())) { + //tuple.set(OperationType.PUT, null, lastRow); + batch.add(new T2<>(row, lastSearchRow)); + + lastSearchRow = rowItr.next(); + + lastKey = lastSearchRow.key(); + } + + ListIterator eqItr = rows.listIterator(rowItr.nextIndex() - 1); + + while (lastKey != null && lastKey.hashCode() == key.hashCode()) { + if (lastKey.equals(key)) + batch.add(new T2<>(row, lastSearchRow)); + + lastKey = eqItr.next().key(); + } + } + + while (rowItr.hasNext()) + batch.add(new T2<>(null, rowItr.next())); + + + // todo call on insertion point + c.call(batch); + + // todo + for (T3 t3 : c.result()) { + OperationType oper = t3.get1(); + CacheDataRow newRow = t3.get3(); +// 1482869858 + if (oper == OperationType.PUT) + put(newRow); + } + +// while (cur.next()) { +// T t = cur.get(); +// +// +// } + +// InvokeAll x = new InvokeAll(row, z, c); + +// try { +// for (;;) { +// x.init(); +// +// Result res = invokeDown(x, x.rootId, 0L, 0L, x.rootLvl); +// +// switch (res) { +// case RETRY: +// case RETRY_ROOT: +// checkInterrupted(); +// +// continue; +// +// default: +// if (!x.isFinished()) { +// res = x.tryFinish(); +// +// if (res == RETRY || res == RETRY_ROOT) { +// checkInterrupted(); +// +// continue; +// } +// +// assert x.isFinished(): res; +// } +// +// return; +// } +// } +// } +// catch (UnregisteredClassException | UnregisteredBinaryTypeException e) { +// throw e; +// } +// catch (IgniteCheckedException e) { +// throw new IgniteCheckedException("Runtime failure on search row: " + row, e); +// } +// catch (RuntimeException | AssertionError e) { +// throw new CorruptedTreeException("Runtime failure on search row: " + row, e); +// } +// finally { +// x.releaseAll(); +// checkDestroyed(); +// } + } + /** {@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/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index 76cb89901b421..a80ceedd8d3ea 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -53,6 +53,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteInterruptedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.cluster.ClusterTopologyException; @@ -73,6 +74,7 @@ import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor; +import org.apache.ignite.internal.processors.cache.BatchedCacheEntries; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.CacheObjectContext; import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; @@ -129,6 +131,10 @@ public class DataStreamerImpl implements IgniteDataStreamer, Delayed /** Per thread buffer size. */ private int bufLdrSzPerThread = DFLT_PER_THREAD_BUFFER_SIZE; + /** */ + private static final boolean batchPageWriteEnabled = + IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DATA_STORAGE_BATCH_PAGE_WRITE, false); + /** * Thread buffer map: on each thread there are future and list of entries which will be streamed after filling * thread batch. @@ -136,7 +142,7 @@ public class DataStreamerImpl implements IgniteDataStreamer, Delayed private final Map threadBufMap = new ConcurrentHashMap<>(); /** Isolated receiver. */ - private static final StreamReceiver ISOLATED_UPDATER = new IsolatedUpdater(); + private static final StreamReceiver ISOLATED_UPDATER = new IsolatedUpdater();//batchPageWriteEnabled ? new OptimizedIsolatedUpdater() : new IsolatedUpdater(); /** Amount of permissions should be available to continue new data processing. */ private static final int REMAP_SEMAPHORE_PERMISSIONS_COUNT = Integer.MAX_VALUE; @@ -2332,6 +2338,205 @@ else if (ttl == CU.TTL_NOT_CHANGED) } } + /** + * Isolated batch receiver which only loads entry initial value. + * + * todo + */ + protected static class OptimizedIsolatedUpdater extends IsolatedUpdater { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public void receive( + IgniteCache cache, + Collection> entries + ) { + IgniteCacheProxy proxy = (IgniteCacheProxy)cache; + + GridCacheAdapter internalCache = proxy.context().cache(); + + if (internalCache.context().mvccEnabled() || internalCache.isNear() || internalCache.context().isLocal() || entries.size() < 10) { // todo threshold + super.receive(cache, entries); + + return; + } + +// if (internalCache.isNear()) +// internalCache = internalCache.context().near().dht(); + + GridCacheContext cctx = internalCache.context(); + + GridDhtTopologyFuture topFut = cctx.shared().exchange().lastFinishedFuture(); + + AffinityTopologyVersion topVer = topFut.topologyVersion(); + + GridCacheVersion ver = cctx.versions().isolatedStreamerVersion(); + + long ttl = CU.TTL_ETERNAL; + long expiryTime = CU.EXPIRE_TIME_ETERNAL; + + ExpiryPolicy plc = cctx.expiry(); + + Collection reservedParts = new HashSet<>(); + Collection ignoredParts = new HashSet<>(); + + Map batchMap = new HashMap<>(); + + try { +// log.info("Received " + entries.size()); + + for (Entry e : entries) { +// cctx.shared().database().checkpointReadLock(); + + try { + e.getKey().finishUnmarshal(cctx.cacheObjectContext(), cctx.deploy().globalLoader()); + + BatchedCacheEntries batch = null; + + if (plc != null) { + ttl = CU.toTtl(plc.getExpiryForCreation()); + + if (ttl == CU.TTL_ZERO) + continue; + else if (ttl == CU.TTL_NOT_CHANGED) + ttl = 0; + + expiryTime = CU.toExpireTime(ttl); + } + + // todo kill duplication + int p = cctx.affinity().partition(e.getKey()); + + if (ignoredParts.contains(p)) + continue; + + if (!reservedParts.contains(p)) { + GridDhtLocalPartition part = cctx.topology().localPartition(p, topVer, true); + + if (!part.reserve()) { + ignoredParts.add(p); + + continue; + } + else { + // We must not allow to read from RENTING partitions. + if (part.state() == GridDhtPartitionState.RENTING) { + part.release(); + + ignoredParts.add(p); + + continue; + } + + reservedParts.add(p); + } + } + + /// + batch = batchMap.computeIfAbsent(p, v -> new BatchedCacheEntries(topVer, p, cctx, false)); + + boolean primary = cctx.affinity().primaryByKey(cctx.localNode(), e.getKey(), topVer); + + batch.addEntry(e.getKey(), e.getValue(), expiryTime, ttl, ver, primary ? GridDrType.DR_LOAD : GridDrType.DR_PRELOAD); + + +// if (topFut != null) { +// Throwable err = topFut.validateCache(cctx, false, false, entry.key(), null); +// +// if (err != null) +// throw new IgniteCheckedException(err); +// } + +// boolean primary = cctx.affinity().primaryByKey(cctx.localNode(), entry.key(), topVer); +// +// entry.initialValue(e.getValue(), +// ver, +// ttl, +// expiryTime, +// false, +// topVer, +// primary ? GridDrType.DR_LOAD : GridDrType.DR_PRELOAD, +// false); +// +// entry.touch(topVer); +// +// CU.unwindEvicts(cctx); +// +// entry.onUnlock(); +// } + } + catch (GridDhtInvalidPartitionException ignored) { + ignoredParts.add(cctx.affinity().partition(e.getKey())); + } +// catch (GridCacheEntryRemovedException ignored) { +// // No-op. +// } + catch (IgniteCheckedException ex) { + IgniteLogger log = cache.unwrap(Ignite.class).log(); + + U.error(log, "Failed to set initial value for cache entry: " + e, ex); + + throw new IgniteException("Failed to set initial value for cache entry.", ex); + } +// finally { +//// cctx.shared().database().checkpointReadUnlock(); +// } + } + + cctx.shared().database().checkpointReadLock(); + + try { + for (BatchedCacheEntries b : batchMap.values()) { + b.lock(); + try { + // todo topFut.validateCache + + cctx.offheap().invokeAll(b.context(), b.keys(), b.part(), b.new UpdateClosure()); + //cctx.offheap().updateBatch(batch); + + + } finally { + b.unlock(); + } + } + } + catch (IgniteCheckedException e) { + // todo handle exceptions properly + IgniteLogger log = cache.unwrap(Ignite.class).log(); + + U.error(log, "Failed to set initial value for cache entry.", e); + + throw new IgniteException("Failed to set initial value for cache entry.", e); + } + finally { + cctx.shared().database().checkpointReadUnlock(); + } + + } + finally { + for (Integer part : reservedParts) { + GridDhtLocalPartition locPart = cctx.topology().localPartition(part, topVer, false); + + assert locPart != null : "Evicted reserved partition: " + locPart; + + locPart.release(); + } + + try { + if (!cctx.isNear() && cctx.shared().wal() != null) + cctx.shared().wal().flush(null, false); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to write preloaded entries into write-ahead log.", e); + + throw new IgniteException("Failed to write preloaded entries into write-ahead log.", e); + } + } + } + } + + /** * Key object wrapper. Using identity equals prevents slow down in case of hash code collision. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/diag/DiagnosticProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/diag/DiagnosticProcessor.java new file mode 100644 index 0000000000000..c3f9ce5c7e1aa --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/diag/DiagnosticProcessor.java @@ -0,0 +1,166 @@ +/* + * 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.diag; + +import java.util.Comparator; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.LongAdder; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.GridProcessorAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.TOTAL; + +/** + * General rebalance diagnostic processing API + */ +public class DiagnosticProcessor extends GridProcessorAdapter { + /** */ + private final ConcurrentMap timings = new ConcurrentHashMap<>(); + + /** */ + private final ConcurrentMap counts = new ConcurrentHashMap<>(); + + /** */ + private final ConcurrentMap tracks = new ConcurrentHashMap<>(); + + /** */ + private volatile boolean enabled; + + /** + * @param ctx Context. + */ + public DiagnosticProcessor(GridKernalContext ctx) { + super(ctx); + } + + /** {@inheritDoc} */ + @Override public void start() throws IgniteCheckedException { + for (DiagnosticTopics topics : DiagnosticTopics.values()) { + timings.put(topics.name(), new LongAdder()); + + counts.put(topics.name(), new LongAdder()); + } + + U.quietAndInfo(log, "DiagnosticProcessor started"); + } + + /** {@inheritDoc} */ + @Override public void stop(boolean cancel) throws IgniteCheckedException { + super.stop(cancel); + + resetCounts(); + } + + /** */ + public void beginTrack(DiagnosticTopics topic) { + if (TOTAL == topic) + enabled = true; + + if (!enabled) + return; + + beginTrack(topic.name()); + } + + /** */ + private void beginTrack(String topic) { + tracks.putIfAbsent(topic, U.currentTimeMillis()); + } + + /** */ + public void endTrack(DiagnosticTopics topic) { + if (!enabled) + return; + + if (TOTAL == topic) + enabled = false; + + endTrack(topic.name()); + } + + /** */ + public void timeTrack(DiagnosticTopics topic, long time) { + if (!enabled) + return; + + if (TOTAL == topic) + enabled = false; + + timings.get(topic.name()).add(time); + } + + /** */ + private void endTrack(String topic) { + Long value = tracks.remove(topic); + + if (value == null) + return; + + timings.get(topic).add(U.currentTimeMillis() - value); + counts.get(topic).increment(); + } + + /** */ + public synchronized void printStats() { + long total = timings.get(TOTAL.name()).longValue(); + + StringBuilder buf = new StringBuilder(); + + String out = timings.entrySet() + .stream() + .filter(e -> e.getValue().longValue() != 0) + .sorted(Comparator.comparingInt(o -> DiagnosticTopics.valueOf(o.getKey()).ordinal())) + .map(e -> String.format("# %s : %s ms : %.2f : %s", + DiagnosticTopics.valueOf(e.getKey()).desc(), + e.getValue().longValue(), + ( ((double)e.getValue().longValue()) / total * 100), + counts.get(e.getKey()).longValue())) + .collect(Collectors.joining("\n")); + + buf.append("\n# Diagnostic processor info: \n" + out); + + resetCounts(); + + if (!tracks.isEmpty()) { + String str = tracks.entrySet() + .stream() + .map(e -> "# " + DiagnosticTopics.valueOf(e.getKey()).desc() + " : " + (e.getValue() - U.currentTimeMillis())) + .collect(Collectors.joining("\n")); + + buf.append("\n# Unfinished tracks: \n" + str); + } + + log.info(buf.toString()); + + tracks.clear(); + } + + /** */ + public synchronized void resetCounts() { + for (Map.Entry e : timings.entrySet()) + e.getValue().reset(); + + for (Map.Entry c : counts.entrySet()) + c.getValue().reset(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/diag/DiagnosticTopics.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/diag/DiagnosticTopics.java new file mode 100644 index 0000000000000..5588ee707541d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/diag/DiagnosticTopics.java @@ -0,0 +1,77 @@ +package org.apache.ignite.internal.processors.diag; + +import java.util.HashMap; +import java.util.Map; + +/** + * + */ +public enum DiagnosticTopics { + /** Root. */ + +// /** GridDhtPartitionDemander#preloadEntry(..) */ +// PRELOAD_ENTRY("# # preload on demander"), +// /** GridCacheMapEntry#storeValue(..) */ +// PRELOAD_OFFHEAP_INVOKE("# # # offheap().invoke(..)"), +// +// PRELOAD_OFFHEAP_INVOKE_INSERT_FREELIST("# # # freeList.insertDataRow"), + + +// /** CacheDataStoreImpl#invoke0(..) */ +// PRELOAD_TREE_INVOKE("# # # # dataTree.invoke(..)"), +// /** rowStore.addRow(..) */ +// PRELOAD_TREE_ADD_ROW("# # # # # FreeList.insertDataRow(..)"), +// /** */ +// PRELOAD_TREE_FINISH_UPDATE("# # # # CacheDataStoreImpl.finishUpdate(..)"), +// /** CacheDataStoreImpl.finishUpdate(..) */ +// PRELOAD_INDEXING_STORE("# # # # # indexing().store(..)"), +// /** CacheDataStoreImpl.finishUpdate(..) */ +// PRELOAD_PENDING_TREE_REMOVE("# # # # # pendingTree().removex(..)"), +// /** CacheDataStoreImpl.finishUpdate(..) */ +// PRELOAD_PENDING_TREE_PUT("# # # # # pendingTree().putx(..)"), +// /** CacheDataStoreImpl#finishRemove(..) */ +// PRELOAD_INDEXING_REMOVE("# # # # finishRemove -> indexing().remove(..)"), +// /** CacheDataStoreImpl#finishRemove(..) */ +// PRELOAD_FREELIST_REMOVE("# # # # finishRemove -> freeList.removeDataRowByLink(..)"), +// /** */ +// PRELOAD_UPDATED("# # # ttl().addTrackedEntry(..)"), +// /** */ +// PRELOAD_ON_WAL_LOG("# # # wal.log(..)"), +// /** */ +// PRELOAD_ON_ENTRY_UPDATED("# # # continuousQueries().onEntryUpdated(..)"), +// +// SEND_DEMAND("# message serialization"), +// SEND_RECEIVE("# network delay between nodes"), +// DEMAND_MSG_SEND("# # demand message send"), +// SUPPLY_MSG_SEND("# # supply message send"), + SUPPLIER_PROCESS_MSG("# make batch on supplier handleDemandMessage(..)"), + + DEMANDER_PROCESS_MSG_SINGLE("# # demander process single"), +// DEMANDER_PROCESS_MSG_BATCH_BIN_SEARCH("# # # # # demander search freelist"), +// DEMANDER_PROCESS_MSG_BATCH_BIN_PACK("# # # # # demander process binPack"), +// DEMANDER_PROCESS_MSG_BATCH_BIN_INSERT("# # # # # demander process insert"), +// DEMANDER_PROCESS_MSG_BATCH_ALLOC_PAGE("# # # # # demander alloc page"), +// PRELOAD_OFFHEAP_BATCH_FIND("# # # # # demander find"), +// PRELOAD_OFFHEAP_BATCH_INSERT("# # # # demander rowStore.freeList().insertBatch"), +// PRELOAD_OFFHEAP_BATCH_TREE_INSERT("# # # # demander dataTree.putx"), +// DEMANDER_PROCESS_MSG_BATCH_LOCK("# # # batch lock"), +// DEMANDER_PROCESS_MSG_BATCH_UNLOCK("# # # batch unlock"), +// DEMANDER_PROCESS_MSG_BATCH_UPDATE("# # # demander batch update"), + DEMANDER_PROCESS_MSG_BATCH("# # demander process batch"), + + DEMANDER_PROCESS_MSG("# demander handleSupplyMessage(..)"), + TOTAL("# cache rebalance total"); + + /** */ + private String desc; + + /** */ + DiagnosticTopics(String desc) { + this.desc = desc; + } + + /** */ + public String desc() { + return desc; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java index 9e854d28f6cb0..12d1a6d3918dc 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,12 @@ 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.T2; +import org.apache.ignite.internal.util.typedef.T3; import org.jetbrains.annotations.Nullable; /** @@ -42,6 +46,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 +142,25 @@ interface InvokeClosure { OperationType operationType(); } + /** + * T found row + * L search row + */ + interface InvokeAllClosure { + /** + * + * @param rows Old row -> new row + * @throws IgniteCheckedException If failed. + */ + void call(@Nullable Collection> rows) throws IgniteCheckedException; + + /** + * + * @return operation, old row, new row + */ + Collection> result(); + } + /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java index 7e3221ebfdbf9..ac2839e8cbb7f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java @@ -55,6 +55,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.future.GridCompoundFuture; @@ -559,6 +560,15 @@ GridNioFuture send(GridNioSession ses, GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses; +// if (msg instanceof GridIoMessage) { +// GridIoMessage msg0 = (GridIoMessage)msg; +// +// Message msg1 = msg0.message(); +// +// if (msg1 instanceof GridDhtPartitionSupplyMessage) +// ((GridDhtPartitionSupplyMessage)msg1).timestamp(U.currentTimeMillis()); +// } + if (createFut) { NioOperationFuture fut = new NioOperationFuture(impl, NioOperation.REQUIRE_WRITE, msg, skipRecoveryPred.apply(msg), ackC); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 044ffef92c19c..84a26744cc167 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -71,9 +71,11 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.managers.communication.GridIoMessage; import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.managers.eventstorage.HighPriorityListener; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; import org.apache.ignite.internal.util.GridConcurrentFactory; import org.apache.ignite.internal.util.GridSpinReadWriteLock; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -158,6 +160,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR; import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; +//import static org.apache.ignite.internal.processors.diag.DiagnosticTopics.SUPPLY_MSG_SEND; import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META; import static org.apache.ignite.plugin.extensions.communication.Message.DIRECT_TYPE_SIZE; import static org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationConnectionCheckFuture.SES_FUT_META; @@ -822,6 +825,18 @@ else if (connKey.dummy()) { else c = NOOP; +// if (msg instanceof GridIoMessage) { +// GridIoMessage msg0 = (GridIoMessage)msg; +// +// Message msg1 = msg0.message(); +// +// if (msg1 instanceof GridDhtPartitionSupplyMessage) { +// +//// ((IgniteEx)ignite).context().diagnostic().timeTrack(SUPPLY_MSG_SEND, (U.currentTimeMillis() - ((GridDhtPartitionSupplyMessage)msg1).timestamp())); +// +// } +// } + notifyListener(connKey.nodeId(), msg, c); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index 358dfc391ae01..ad6f20fff783a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -706,6 +706,12 @@ void recheckLock() { return false; } + /** @inheritDoc */ + @Override public void finishPreload(@Nullable CacheObject val, long expTime, long ttl, GridCacheVersion ver, + AffinityTopologyVersion topVer, GridDrType drType, MvccVersion mvccVer, boolean preload) { + assert false; + } + /** @inheritDoc */ @Override public GridCacheVersionedEntryEx versionedEntry(final boolean keepBinary) throws IgniteCheckedException { return null; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/PutAllTxCheck.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/PutAllTxCheck.java new file mode 100644 index 0000000000000..48662b794c932 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/PutAllTxCheck.java @@ -0,0 +1,42 @@ +package org.apache.ignite.internal.processors.cache.distributed.replicated; + +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +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.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Test; + +public class PutAllTxCheck extends GridCommonAbstractTest { + + + + @Test + public void check() throws Exception { + Ignite node = startGrids(2); + + IgniteCache cache = node.createCache(ccfg(DEFAULT_CACHE_NAME)); + +// Map data = new HashMap<>(); + + for (int i = 0; i < 3; i++) + cache.put(i, i); + +// cache.putAll(data); + } + + private CacheConfiguration ccfg(String name) { + CacheConfiguration ccfg = new CacheConfiguration(name); + + ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); +// ccfg.setAffinity(new RendezvousAffinityFunction(false, 1)); +// ccfg.setCacheMode(CacheMode.REPLICATED); + ccfg.setBackups(1); + + return ccfg; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java index 4c0d898427422..eb06f9aee28a9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java @@ -363,7 +363,7 @@ protected FreeList createFreeList(int pageSize) throws Exception { DataRegion dataRegion = new DataRegion(pageMem, plcCfg, regionMetrics, new NoOpPageEvictionTracker()); - return new CacheFreeListImpl(1, "freelist", regionMetrics, dataRegion, null, null, metaPageId, true); + return new CacheFreeListImpl(1, "freelist", regionMetrics, dataRegion, null, null, metaPageId, true, null); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListBatchBench.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListBatchBench.java new file mode 100644 index 0000000000000..73404199114db --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListBatchBench.java @@ -0,0 +1,330 @@ +/* + * 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.text.DecimalFormat; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.ignite.DataRegionMetrics; +import org.apache.ignite.IgniteCheckedException; +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.internal.processors.cache.persistence.DataRegion; +import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * + */ +@RunWith(JUnit4.class) +public class FreeListBatchBench extends GridCommonAbstractTest { + /** */ + private static final DecimalFormat DECIMAL_FORMAT = new DecimalFormat("#0.0"); + + /** */ + private static final long DEF_REG_SIZE = 4 * 1024 * 1024 * 1024L; + + /** */ + 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 boolean MEM_STAT = true; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + DataRegionConfiguration reg1 = new DataRegionConfiguration(); + reg1.setInitialSize(DEF_REG_SIZE); + reg1.setMaxSize(DEF_REG_SIZE); + reg1.setMetricsEnabled(MEM_STAT); + reg1.setName(REG_BATCH); + + DataRegionConfiguration reg2 = new DataRegionConfiguration(); + reg2.setInitialSize(DEF_REG_SIZE); + reg2.setMaxSize(DEF_REG_SIZE); + reg2.setMetricsEnabled(MEM_STAT); + reg2.setName(REG_SINGLE); + + DataStorageConfiguration storeCfg = new DataStorageConfiguration(); + + storeCfg.setDataRegionConfigurations(reg1, reg2); + + cfg.setDataStorageConfiguration(storeCfg); + + return cfg; + } + /** + * + */ + @Test + public void testBatch() throws Exception { + startGrid(0); + + int batchSize = 505; + + bench(batchSize, 50, 0, 4); + bench(batchSize, 50, 0, 16); + bench(batchSize, 50, 0, 256); + bench(batchSize, 50, 0, 512); + bench(batchSize, 50, 0, 1024); + bench(batchSize, 20, 0, 8192); + bench(batchSize, 10, 4096, 16384); +// bench(batchSize, 40, 700, 999); +// bench(batchSize / 10, 100, 4096, 16384); +// bench(batchSize / 50, 500, 4096, 16384); +// bench(batchSize / 100, 1000, 4096, 16384); + + //bench(batchSize / 10, 50, 4096, 16384); + } + + /** */ + private void bench(int batchSize, int iterations, int minObjSIze, int maxObjSize) throws Exception { + int sizes[] = new int[batchSize]; + int minSize = maxObjSize; + int maxSize = minObjSIze; + long sum = 0; + + 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; + + sum += size; + } + + long avgSize = sum / sizes.length; + + int subIters = 100; + + long batchTotalTime = 0; + long singleTotalTime = 0; + long[] batchTimes = new long[subIters / 2]; + long[] singleTimes = new long[subIters / 2]; + + IgniteEx node = grid(0); + + node.createCache(ccfg(true)); + node.createCache(ccfg(false)); + + try { + log.info(">>> Warm up " + subIters / 10 + " iterations."); + + int subOff = subIters / 10; + + GridCacheContext cctxBatch = node.cachex(CACHE_BATCH).context(); + GridCacheContext cctxSingle = node.cachex(CACHE_SINGLE).context(); + + for (int i = 0; i < subOff ; i++) { + boolean batch = i % 2 == 0; + + doBatchUpdate(batch ? cctxBatch : cctxSingle, batch, batchSize, iterations, sizes, i * iterations); + } + + log.info(">>> Starting " + subIters + " iterations, batch=" + batchSize); + + for (int i = 0; i < subIters; i++) { + long batch, single; + if (i % 2 == 0) { + batch = doBatchUpdate(cctxBatch, true, batchSize, iterations, sizes, i * iterations + (subOff * iterations)); + + batchTimes[i / 2] = batch; + + batchTotalTime += batch; + } + else { + single = doBatchUpdate(cctxSingle, false, batchSize, iterations, sizes, i * iterations + (subOff * iterations)); + + singleTimes[i / 2] = single; + + singleTotalTime += single; + } + } + + // Check mean err. + long batchAvg = batchTotalTime / (subIters / 2); + long singleAvg = singleTotalTime / (subIters / 2); + + double batchMean = meanError(batchTimes, batchAvg); + double singleMean = meanError(singleTimes, singleAvg); + + String str = String.format("\n####################################################################################\n" + + "\t>>> cnt=%d\n" + + "\t>>> objects size: min=%d, max=%d, avg=%d\n" + + "\t>>> time: batch=%d, single=%d ---> %s%%\n" + + "######[MEANS]#######################################################################\n" + + "\t>>> Batch: %.4f (avg=%d) %s%%\n" + + "\t>>> Single: %.4f (avg=%d) %s%%" + + "\n####################################################################################", + batchSize, minSize, maxSize, avgSize, batchAvg, singleAvg, percent(batchAvg, singleAvg), + batchMean, batchAvg, DECIMAL_FORMAT.format((batchMean / (double)batchAvg) * 100), + singleMean, singleAvg, DECIMAL_FORMAT.format((singleMean / (double)singleAvg) * 100)); + + log.info(str); + + if (MEM_STAT) { + IgniteCacheDatabaseSharedManager dbMgr = grid(0).context().cache().context().database(); + + dbMgr.dumpStatistics(log()); + + printMemMetrics(dbMgr, REG_BATCH); + printMemMetrics(dbMgr, REG_SINGLE); + } + } + finally { + grid(0).destroyCache(CACHE_BATCH); + grid(0).destroyCache(CACHE_SINGLE); + } + } + + /** */ + private long doBatchUpdate( + GridCacheContext cctx, + boolean batch, + int batchSize, + int iterations, + int[] objSizes, + int off + ) throws Exception { + GridDhtPreloader preloader = (GridDhtPreloader)cctx.group().preloader(); + + GridDhtPartitionDemander demander = GridTestUtils.getFieldValue(preloader, "demander"); + + long nanos = 0; + + for (int iter = off; iter < off + iterations; iter++) { + List infos = prepareBatch(cctx, iter * batchSize, batchSize, objSizes); + + long start = System.nanoTime(); + + if (batch) + demander.preloadEntries(null, 0, infos, cctx.topology().readyTopologyVersion()); + else + demander.preloadEntriesSingle(null, 0, infos, cctx.topology().readyTopologyVersion()); + + nanos += (System.nanoTime() - start); + } + + return nanos / iterations; + } + + /** + * @return Mean squared error. + */ + public double meanError(long[] times, long avg) { + double sum = 0.0; + + for (int i = 0; i < times.length; i++) { + double x = (double)(times[i] - avg); + + sum += x * x; + } + + return Math.sqrt(sum / (times.length - 1)); + } + + /** + * 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; + } + + /** + * Format percentage. + */ + private String percent(long time, long time1) { + return DECIMAL_FORMAT.format((100 - ((double)time) / ((double)time1) * 100) * -1); + } + + + /** */ + private void printMemMetrics(IgniteCacheDatabaseSharedManager dbMgr, String regName) throws IgniteCheckedException { + DataRegion reg = dbMgr.dataRegion(regName); + + DataRegionMetrics metrics = reg.memoryMetrics(); + + log.info(regName + ": pages=" + metrics.getTotalAllocatedPages() + + ", fill=" + new DecimalFormat("#0.0000").format(metrics.getPagesFillFactor())); + } + + /** + * @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); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListBatchUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListBatchUpdateTest.java new file mode 100644 index 0000000000000..f27acc381d55e --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListBatchUpdateTest.java @@ -0,0 +1,510 @@ +/* + * 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.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.commons.io.FileUtils; +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.common.GridCommonAbstractTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +//import ru.sbrf.gg.load.LoadTable; +//import ru.sbrf.gg.load.ProcessTableFile; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; +import static org.junit.Assert.assertArrayEquals; + +/** + * + */ +//@RunWith(JUnit4.class) +@RunWith(Parameterized.class) +public class FreeListBatchUpdateTest extends GridCommonAbstractTest { + /** */ + private static final int HDR_SIZE = 8 + 32; + + /** */ + private static final long DEF_REG_SIZE = 6 * 1024 * 1024 * 1024L; + + /** */ + private static final String DEF_CACHE_NAME = "DepoHist_DPL_union-module"; + + /** */ + @Parameterized.Parameters(name = "with atomicity={0} and persistence={1}") + public static Iterable setup() { + return Arrays.asList(new Object[][]{ + {CacheAtomicityMode.ATOMIC, false}, +// {CacheAtomicityMode.ATOMIC, true}, +// {CacheAtomicityMode.TRANSACTIONAL, false}, +// {CacheAtomicityMode.TRANSACTIONAL, true}, +// {CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT, false}, +// {CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT, true} + }); + } + + @Parameterized.Parameter(0) + public CacheAtomicityMode cacheAtomicityMode; + + @Parameterized.Parameter(1) + public boolean persistence; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + DataRegionConfiguration def = new DataRegionConfiguration(); + def.setInitialSize(3400 * 1024 * 1024L); + def.setMaxSize(DEF_REG_SIZE); + def.setPersistenceEnabled(persistence); + + DataStorageConfiguration storeCfg = new DataStorageConfiguration(); + + storeCfg.setDefaultDataRegionConfiguration(def); + + 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(); + + System.clearProperty(IGNITE_PDS_WAL_REBALANCE_THRESHOLD); + } + + @Test + public void checkStreamer() throws Exception { + Ignite node = startGrids(4); + + node.cluster().active(true); + + IgniteCache cache = node.createCache(ccfg(8, CacheMode.REPLICATED)); + + awaitPartitionMapExchange(); + + int cnt = 1024; + + //IgniteCache cache = ; + + try (IgniteDataStreamer streamer = node.dataStreamer(DEF_CACHE_NAME)) { + + for (int i = 0; i < cnt; i++) + streamer.addData(String.valueOf(i), new byte[128]); + } + + log.info("Sleep"); + + U.sleep(5_000); + + assert GridTestUtils.waitForCondition(() -> { + return cache.size() == cnt; + }, 10_000); + + for (int i = 0; i < cnt; i++) + assertTrue(cache.get(String.valueOf(i)).length == 128); + } + + /** + * + */ + @Test + public void testBatchPartialRebalance() throws Exception { + if (!persistence) + return; + + // TODO https://issues.apache.org/jira/browse/IGNITE-7384 + // http://apache-ignite-developers.2346864.n4.nabble.com/Historical-rebalance-td38380.html + if (cacheAtomicityMode == CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT) + return; + + System.setProperty(IGNITE_PDS_WAL_REBALANCE_THRESHOLD, "100"); + + 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(1024)]; + + srcMap.put(String.valueOf(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 < 1000; i++) { + String key = String.valueOf(i); + + if (i % 33 == 0) { + cache.remove(key); + + srcMap.remove(key); + } + else { + byte[] bytes = cache.get(key); + + Arrays.fill(bytes, (byte)1); + + srcMap.put(key, bytes); + cache.put(key, 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); + } + + + /** + * + */ + @Test + public void testBatchPutAll() throws Exception { + Ignite node = startGrid(0); + + node.cluster().active(true); + + node.createCache(ccfg()); + + int cnt = 1_000_000; + int minSize = 0; + int maxSize = 2048; + int start = 0; + + log.info("Loading " + cnt + " random entries per " + minSize + " - " + maxSize + " bytes."); + + Map srcMap = new HashMap<>(); + + for (int i = start; i < start + cnt; i++) { + int size = 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); + } + + srcMap.put(1, new byte[65536]); + + node.cache(DEF_CACHE_NAME).put(String.valueOf(1), new byte[65536]); + + log.info("Done"); + + IgniteCache cache = node.cache(DEF_CACHE_NAME); + + 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); + + U.sleep(2_000); + + node.close(); + + log.info("Verification on node2"); + + 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 +// public void testBatchPutAllLoader() throws Exception { +// Ignite node = startGrid(0); +// +// node.cluster().active(true); +// +// node.createCache(ccfg()); +// +// ExecutorService execSvc = Executors.newFixedThreadPool(4); +// +// ProcessTableFile load = new LoadTable("EIP_DBAOSB_DEPOHISTPARAM", "/home/xtern/src/data/cod_data_mini.zip", execSvc, node, 1); +// +// load.process(); +// +// execSvc.shutdown(); +// +// log.info("Done"); +// +// IgniteCache cache = node.cache(DEF_CACHE_NAME); +// +// 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); +// +// U.sleep(2_000); +// +// node.close(); +// +// log.info("Verification on node2"); +// } + + /** + * @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(1000); + + assertTrue(ok); + } + + /** + * @param cache Cache. + * @param map Map. + */ + @SuppressWarnings("unchecked") + private void validateCacheEntries(IgniteCache cache, Map map) { + if (true) + return; + + log.info("Cache validation: " + map.size()); + + assertEquals(map.size(), cache.size()); + + for (Map.Entry e : map.entrySet()) { + String idx = "idx=" + e.getKey(); + + byte[] bytes = (byte[])cache.get(e.getKey()); + + assertNotNull(idx, bytes); + + assertEquals(idx + ": length not equal", e.getValue().length, bytes.length); + + assertArrayEquals(idx, e.getValue(), bytes); + } + } + + + /** + * @return Cache configuration. + */ + private CacheConfiguration ccfg() { + return ccfg(1024, 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); + } + + @Test + public void testFUCK() { + ///home/xtern/tools/sdk/jdk1.8.0_152/bin/java -ea -Xmx2G -classpath + //com.intellij.rt.execution.junit.JUnitStarter -ideVersion5 -junit4 org.apache.ignite.internal.processors.database.FreeListBatchBench,testBatch + String cp = "/home/xtern/tools/ide/idea-IU-182.4505.22/lib/idea_rt.jar:/home/xtern/tools/ide/idea-IU-182.4505.22/plugins/junit/lib/junit-rt.jar:/home/xtern/tools/ide/idea-IU-182.4505.22/plugins/junit/lib/junit5-rt.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/charsets.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/deploy.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/ext/cldrdata.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/ext/dnsns.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/ext/jaccess.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/ext/jfxrt.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/ext/localedata.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/ext/nashorn.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/ext/sunec.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/ext/sunjce_provider.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/ext/sunpkcs11.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/ext/zipfs.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/javaws.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/jce.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/jfr.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/jfxswt.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/jsse.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/management-agent.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/plugin.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/resources.jar:/home/xtern/tools/sdk/jdk1.8.0_152/jre/lib/rt.jar:/home/xtern/src/ignite/modules/core/target/test-classes:/home/xtern/src/ignite/modules/core/target/classes:/home/xtern/.m2/repository/javax/cache/cache-api/1.0.0/cache-api-1.0.0.jar:/home/xtern/.m2/repository/org/jetbrains/annotations/16.0.3/annotations-16.0.3.jar:/home/xtern/.m2/repository/mx4j/mx4j-tools/3.0.1/mx4j-tools-3.0.1.jar:/home/xtern/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/home/xtern/.m2/repository/commons-dbcp/commons-dbcp/1.4/commons-dbcp-1.4.jar:/home/xtern/.m2/repository/commons-pool/commons-pool/1.5.4/commons-pool-1.5.4.jar:/home/xtern/.m2/repository/com/thoughtworks/xstream/xstream/1.4.8/xstream-1.4.8.jar:/home/xtern/.m2/repository/xmlpull/xmlpull/1.1.3.1/xmlpull-1.1.3.1.jar:/home/xtern/.m2/repository/xpp3/xpp3_min/1.1.4c/xpp3_min-1.1.4c.jar:/home/xtern/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/home/xtern/.m2/repository/org/hsqldb/hsqldb/1.8.0.10/hsqldb-1.8.0.10.jar:/home/xtern/.m2/repository/com/h2database/h2/1.4.197/h2-1.4.197.jar:/home/xtern/.m2/repository/org/mockito/mockito-all/1.9.5/mockito-all-1.9.5.jar:/home/xtern/.m2/repository/org/eclipse/jetty/jetty-servlets/9.4.11.v20180605/jetty-servlets-9.4.11.v20180605.jar:/home/xtern/.m2/repository/org/eclipse/jetty/jetty-continuation/9.4.11.v20180605/jetty-continuation-9.4.11.v20180605.jar:/home/xtern/.m2/repository/org/eclipse/jetty/jetty-http/9.4.11.v20180605/jetty-http-9.4.11.v20180605.jar:/home/xtern/.m2/repository/org/eclipse/jetty/jetty-util/9.4.11.v20180605/jetty-util-9.4.11.v20180605.jar:/home/xtern/.m2/repository/org/eclipse/jetty/jetty-io/9.4.11.v20180605/jetty-io-9.4.11.v20180605.jar:/home/xtern/.m2/repository/org/eclipse/jetty/jetty-webapp/9.4.11.v20180605/jetty-webapp-9.4.11.v20180605.jar:/home/xtern/.m2/repository/org/eclipse/jetty/jetty-xml/9.4.11.v20180605/jetty-xml-9.4.11.v20180605.jar:/home/xtern/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.11.v20180605/jetty-servlet-9.4.11.v20180605.jar:/home/xtern/.m2/repository/org/eclipse/jetty/jetty-security/9.4.11.v20180605/jetty-security-9.4.11.v20180605.jar:/home/xtern/.m2/repository/org/eclipse/jetty/jetty-server/9.4.11.v20180605/jetty-server-9.4.11.v20180605.jar:/home/xtern/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/home/xtern/.m2/repository/com/esotericsoftware/kryo/kryo/2.20/kryo-2.20.jar:/home/xtern/.m2/repository/com/esotericsoftware/reflectasm/reflectasm/1.07/reflectasm-1.07-shaded.jar:/home/xtern/.m2/repository/org/ow2/asm/asm/4.0/asm-4.0.jar:/home/xtern/.m2/repository/com/esotericsoftware/minlog/minlog/1.2/minlog-1.2.jar:/home/xtern/.m2/repository/org/objenesis/objenesis/1.2/objenesis-1.2.jar:/home/xtern/.m2/repository/c3p0/c3p0/0.9.1/c3p0-0.9.1.jar:/home/xtern/.m2/repository/org/gridgain/ignite-shmem/1.0.0/ignite-shmem-1.0.0.jar:/home/xtern/.m2/repository/log4j/log4j/1.2.17/log4j-1.2.17.jar:/home/xtern/.m2/repository/org/springframework/spring-beans/4.3.18.RELEASE/spring-beans-4.3.18.RELEASE.jar:/home/xtern/.m2/repository/org/springframework/spring-core/4.3.18.RELEASE/spring-core-4.3.18.RELEASE.jar:/home/xtern/.m2/repository/commons-logging/commons-logging/1.2/commons-logging-1.2.jar:/home/xtern/.m2/repository/org/springframework/spring-context/4.3.18.RELEASE/spring-context-4.3.18.RELEASE.jar:/home/xtern/.m2/repository/org/springframework/spring-aop/4.3.18.RELEASE/spring-aop-4.3.18.RELEASE.jar:/home/xtern/.m2/repository/org/springframework/spring-expression/4.3.18.RELEASE/spring-expression-4.3.18.RELEASE.jar:/home/xtern/.m2/repository/commons-io/commons-io/2.4/commons-io-2.4.jar:/home/xtern/.m2/repository/org/apache/ignite/binary/test1/1.1/test1-1.1.jar:/home/xtern/.m2/repository/org/apache/ignite/binary/test2/1.1/test2-1.1.jar:/home/xtern/.m2/repository/com/google/guava/guava/25.1-jre/guava-25.1-jre.jar:/home/xtern/.m2/repository/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2.jar:/home/xtern/.m2/repository/org/checkerframework/checker-qual/2.0.0/checker-qual-2.0.0.jar:/home/xtern/.m2/repository/com/google/errorprone/error_prone_annotations/2.1.3/error_prone_annotations-2.1.3.jar:/home/xtern/.m2/repository/com/google/j2objc/j2objc-annotations/1.1/j2objc-annotations-1.1.jar:/home/xtern/.m2/repository/org/codehaus/mojo/animal-sniffer-annotations/1.14/animal-sniffer-annotations-1.14.jar:/home/xtern/.m2/repository/org/javassist/javassist/3.20.0-GA/javassist-3.20.0-GA.jar:/home/xtern/.m2/repository/junit/junit/4.11/junit-4.11.jar:/home/xtern/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/home/xtern/.m2/repository/ignite-cod-data-loader/ignite-cod-data-loader/0.0.1-SNAPSHOT/ignite-cod-data-loader-0.0.1-SNAPSHOT.jar:/home/xtern/.m2/repository/org/scala-lang/scala-library/2.11.8/scala-library-2.11.8.jar:/home/xtern/.m2/repository/org/scala-lang/modules/scala-xml_2.11/1.0.6/scala-xml_2.11-1.0.6.jar:/home/xtern/.m2/repository/com/github/scopt/scopt_2.11/3.7.0/scopt_2.11-3.7.0.jar:/home/xtern/src/ignite/modules/spring/target/classes:/home/xtern/.m2/repository/org/springframework/spring-tx/4.3.18.RELEASE/spring-tx-4.3.18.RELEASE.jar:/home/xtern/.m2/repository/org/springframework/spring-jdbc/4.3.18.RELEASE/spring-jdbc-4.3.18.RELEASE.jar:/home/xtern/src/ignite/modules/log4j/target/classes:/home/xtern/src/ignite/modules/indexing/target/classes:/home/xtern/.m2/repository/commons-codec/commons-codec/1.11/commons-codec-1.11.jar:/home/xtern/.m2/repository/org/apache/lucene/lucene-core/7.4.0/lucene-core-7.4.0.jar:/home/xtern/.m2/repository/org/apache/lucene/lucene-analyzers-common/7.4.0/lucene-analyzers-common-7.4.0.jar:/home/xtern/.m2/repository/org/apache/lucene/lucene-queryparser/7.4.0/lucene-queryparser-7.4.0.jar:/home/xtern/.m2/repository/org/apache/lucene/lucene-queries/7.4.0/lucene-queries-7.4.0.jar:/home/xtern/.m2/repository/org/apache/lucene/lucene-sandbox/7.4.0/lucene-sandbox-7.4.0.jar:/home/xtern/src/ignite/modules/spring-data/target/classes:/home/xtern/.m2/repository/org/springframework/data/spring-data-commons/1.13.14.RELEASE/spring-data-commons-1.13.14.RELEASE.jar:/home/xtern/.m2/repository/org/slf4j/slf4j-api/1.7.25/slf4j-api-1.7.25.jar:/home/xtern/.m2/repository/net/logstash/log4j/jsonevent-layout/1.7/jsonevent-layout-1.7.jar:/home/xtern/.m2/repository/net/minidev/json-smart/1.1.1/json-smart-1.1.1.jar:/home/xtern/.m2/repository/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/home/xtern/tools/sdk/jdk1.8.0_152/lib/tools.jar"; + Arrays.stream(cp.split(":")).forEach(v -> cp(v, "/home/xtern/out")); + + + } + + private void cp(String path, String target) { + + File source = new File(path); + File target0 = new File(target + "/" + source.getName()); + + System.out.println("> cp " + source + " -> " + target0); + + try { + if (source.isDirectory()) + FileUtils.copyDirectory(source, target0); + else + FileUtils.copyFile(source, target0); + } catch (IOException e) { + throw new RuntimeException(e.getMessage()); + } + + + //Files. + + } + +// private static final String delim = ";"; +// private static final String encoding = "ISO-8859-1"; +//// class CSVReader { +//// +//// final int maxLines; +//// +//// final InputStream file; +//// +//// int cnt = 0; +//// +//// CSVReader(InputStream file) { +//// maxLines = Integer.valueOf(System.getProperty("MAX_LINES", "-1")); +//// +//// if (maxLines != -1) +//// log.warning(maxLines + " would be readed from stream"); +//// } +//// } +//// +//// +//// +//// +//// override def iterator: Iterator[Array[String]] = new Iterator[Array[String]] { +//// override def hasNext: Boolean = { +//// cnt += 1 +//// (maxLines == -1 || cnt < maxLines) && lines.hasNext +//// } +//// +//// override def next(): Array[String] = { +//// val line = lines.next() +//// line.split(delim, -1) +//// } +//// } +//// } +} diff --git a/parent/pom.xml b/parent/pom.xml index ce3cff6a13a2d..1477f7235b0ee 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -253,6 +253,12 @@ 4.11 test + + ignite-cod-data-loader + ignite-cod-data-loader + 0.0.1-SNAPSHOT + test +