From 32ea9a9e2219965a1e718838d71c3424bbd4435c Mon Sep 17 00:00:00 2001 From: Nikolay Date: Tue, 23 Mar 2021 17:38:57 +0300 Subject: [PATCH 01/28] IGNITE-13582 WAL force rollover timeout introduced (#8902) --- .../DataStorageConfiguration.java | 26 ++++++ .../wal/FileWriteAheadLogManager.java | 90 +++++++++++++------ .../utils/PlatformConfigurationUtils.java | 4 +- .../node/VisorDataStorageConfiguration.java | 15 +++- .../db/wal/reader/IgniteWalReaderTest.java | 44 +++++++++ .../Config/full-config.xml | 1 + .../IgniteConfigurationSerializerTest.cs | 2 + .../IgniteConfigurationTest.cs | 3 + .../Configuration/DataStorageConfiguration.cs | 9 ++ .../IgniteConfigurationSection.xsd | 5 ++ 10 files changed, 170 insertions(+), 29 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java index 6f5b3380f479f..4574febe54be3 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java @@ -27,6 +27,7 @@ import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteExperimental; import org.apache.ignite.mxbean.MetricsMxBean; import org.jetbrains.annotations.Nullable; @@ -288,6 +289,10 @@ public class DataStorageConfiguration implements Serializable { */ private long walAutoArchiveAfterInactivity = -1; + /** Time interval (in milliseconds) for force archiving of incompletely WAL segment. */ + @IgniteExperimental + private long walForceArchiveTimeout = -1; + /** * If true, threads that generate dirty pages too fast during ongoing checkpoint will be throttled. */ @@ -1015,6 +1020,27 @@ public long getWalAutoArchiveAfterInactivity() { return walAutoArchiveAfterInactivity; } + /** + * @param walForceArchiveTimeout time in millis to run auto archiving segment (even if incomplete) after last + * record logging.
Positive value enables incomplete segment archiving after timeout (inactivity).
Zero or + * negative value disables auto archiving. + * @return current configuration instance for chaining + */ + @IgniteExperimental + public DataStorageConfiguration setWalForceArchiveTimeout(long walForceArchiveTimeout) { + this.walForceArchiveTimeout = walForceArchiveTimeout; + + return this; + } + + /** + * @return time in millis to run auto archiving WAL segment (even if incomplete) after last record log + */ + @IgniteExperimental + public long getWalForceArchiveTimeout() { + return walForceArchiveTimeout; + } + /** * This property defines order of writing pages to disk storage during checkpoint. * 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 9407404cc26bd..3982cb2d4734e 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 @@ -326,6 +326,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl */ private final long walAutoArchiveAfterInactivity; + /** Positive (non-0) value indicates WAL must be archived even if not complete. */ + private final long walForceArchiveTimeout; + /** * Container with last WAL record logged timestamp.
Zero value means there was no records logged to current * segment, skip possible archiving for this case
Value is filled only for case {@link @@ -333,6 +336,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl */ private final AtomicLong lastRecordLoggedMs = new AtomicLong(); + /** Last rollover time. */ + private AtomicLong lastRolloverMs; + /** * Cancellable task for {@link WALMode#BACKGROUND}, should be cancelled at shutdown. * Null for non background modes. @@ -401,8 +407,12 @@ public FileWriteAheadLogManager(final GridKernalContext ctx) { ioFactory = mode == WALMode.FSYNC ? dsCfg.getFileIOFactory() : new RandomAccessFileIOFactory(); segmentFileInputFactory = new SimpleSegmentFileInputFactory(); walAutoArchiveAfterInactivity = dsCfg.getWalAutoArchiveAfterInactivity(); + walForceArchiveTimeout = dsCfg.getWalForceArchiveTimeout(); - timeoutRolloverMux = walAutoArchiveAfterInactivity > 0 ? new Object() : null; + timeoutRolloverMux = (walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0) ? new Object() : null; + + if (walForceArchiveTimeout > 0) + lastRolloverMs = new AtomicLong(); double thresholdWalArchiveSizePercentage = IgniteSystemProperties.getDouble( IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE, DFLT_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE); @@ -748,21 +758,30 @@ private void checkWalConfiguration() throws IgniteCheckedException { if (mode == WALMode.BACKGROUND) backgroundFlushSchedule = cctx.time().schedule(this::doFlush, flushFreq, flushFreq); - if (walAutoArchiveAfterInactivity > 0) - scheduleNextInactivityPeriodElapsedCheck(); + if (walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0) + scheduleNextRolloverCheck(); } /** - * Schedules next check of inactivity period expired. Based on current record update timestamp. At timeout method - * does check of inactivity period and schedules new launch. + * Schedules next rollover check. + * If {@link DataStorageConfiguration#getWalForceArchiveTimeout()} configured rollover happens forcefully. + * Else check based on current record update timestamp and at timeout method does check of inactivity period and schedules new launch. */ - private void scheduleNextInactivityPeriodElapsedCheck() { - assert walAutoArchiveAfterInactivity > 0; + private void scheduleNextRolloverCheck() { + assert walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0; assert timeoutRolloverMux != null; synchronized (timeoutRolloverMux) { - long lastRecMs = lastRecordLoggedMs.get(); - long nextEndTime = lastRecMs <= 0 ? U.currentTimeMillis() : lastRecMs + walAutoArchiveAfterInactivity; + long nextEndTime; + + if (walForceArchiveTimeout > 0) { + long lastRollover = lastRolloverMs.get(); + nextEndTime = lastRollover == 0 ? U.currentTimeMillis() : lastRollover + walForceArchiveTimeout; + } + else { + long lastRecMs = lastRecordLoggedMs.get(); + nextEndTime = lastRecMs <= 0 ? U.currentTimeMillis() : lastRecMs + walAutoArchiveAfterInactivity; + } cctx.time().addTimeoutObject(timeoutRollover = new TimeoutRollover(nextEndTime)); } @@ -774,25 +793,38 @@ private void scheduleNextInactivityPeriodElapsedCheck() { } /** - * Checks if there was elapsed significant period of inactivity. If WAL auto-archive is enabled using - * {@link #walAutoArchiveAfterInactivity} > 0 this method will activate roll over by timeout.
+ * Checks if there was elapsed significant period of inactivity or force archive timeout. + * If WAL auto-archive is enabled using {@link #walAutoArchiveAfterInactivity} > 0 or {@link #walForceArchiveTimeout} + * this method will activate roll over by timeout. */ - private void checkWalRolloverRequiredDuringInactivityPeriod() { - if (walAutoArchiveAfterInactivity <= 0) - return; // feature not configured, nothing to do + private void checkWalRolloverRequired() { + if (walAutoArchiveAfterInactivity <= 0 && walForceArchiveTimeout <= 0) + return; // feature not configured, nothing to do. final long lastRecMs = lastRecordLoggedMs.get(); if (lastRecMs == 0) - return; //no records were logged to current segment, does not consider inactivity + return; //no records were logged to current segment, does not consider inactivity. - final long elapsedMs = U.currentTimeMillis() - lastRecMs; + if (walForceArchiveTimeout > 0) { + final long lastRollover = lastRolloverMs.get(); + final long elapsedMs = U.currentTimeMillis() - lastRollover; - if (elapsedMs <= walAutoArchiveAfterInactivity) - return; // not enough time elapsed since last write + if (elapsedMs < walForceArchiveTimeout) + return; // not enough time elapsed since last rollover. - if (!lastRecordLoggedMs.compareAndSet(lastRecMs, 0)) - return; // record write occurred concurrently + if (!lastRolloverMs.compareAndSet(lastRollover, 0)) + return; // record write occurred concurrently. + } + else { + final long elapsedMs = U.currentTimeMillis() - lastRecMs; + + if (elapsedMs <= walAutoArchiveAfterInactivity) + return; // not enough time elapsed since last write. + + if (!lastRecordLoggedMs.compareAndSet(lastRecMs, 0)) + return; // record write occurred concurrently. + } final FileWriteHandle handle = currentHandle(); @@ -881,7 +913,7 @@ else if (rolloverType == RolloverType.CURRENT_SEGMENT) { if (ptr != null) { metrics.onWalRecordLogged(rec.size()); - if (walAutoArchiveAfterInactivity > 0) + if (walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0) lastRecordLoggedMs.set(U.currentTimeMillis()); return ptr; @@ -1305,9 +1337,13 @@ private FileWriteHandle rollOver(FileWriteHandle cur, @Nullable WALRecord rec) t assert updated : "Concurrent updates on rollover are not allowed"; - if (walAutoArchiveAfterInactivity > 0) + if (walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0) { lastRecordLoggedMs.set(0); + if (walForceArchiveTimeout > 0) + lastRolloverMs.set(U.currentTimeMillis()); + } + // Let other threads to proceed with new segment. hnd.signalNextAvailable(); } @@ -3443,7 +3479,7 @@ private TimeoutRollover(long endTime) { /** {@inheritDoc} */ @Override public void onTimeout() { - assert walAutoArchiveAfterInactivity > 0; + assert walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0; assert timeoutRolloverMux != null; synchronized (timeoutRolloverMux) { @@ -3453,9 +3489,9 @@ private TimeoutRollover(long endTime) { new Time(U.currentTimeMillis()).toString() + ")"); } - checkWalRolloverRequiredDuringInactivityPeriod(); + checkWalRolloverRequired(); - scheduleNextInactivityPeriodElapsedCheck(); + scheduleNextRolloverCheck(); } } } @@ -3464,7 +3500,7 @@ private TimeoutRollover(long endTime) { * Cancel auto rollover. */ public void cancel() { - assert walAutoArchiveAfterInactivity > 0; + assert walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0; assert timeoutRolloverMux != null; synchronized (timeoutRolloverMux) { @@ -3480,7 +3516,7 @@ public void cancel() { * Stop auto rollover. */ private void stopAutoRollover() { - if (walAutoArchiveAfterInactivity > 0) { + if (walAutoArchiveAfterInactivity > 0 || walForceArchiveTimeout > 0) { assert timeoutRolloverMux != null; synchronized (timeoutRolloverMux) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java index fbe3218014e31..eec16076007b8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java @@ -2055,7 +2055,8 @@ private static DataStorageConfiguration readDataStorageConfiguration(BinaryRawRe .setSystemRegionMaxSize(in.readLong()) .setPageSize(in.readInt()) .setConcurrencyLevel(in.readInt()) - .setWalAutoArchiveAfterInactivity(in.readLong()); + .setWalAutoArchiveAfterInactivity(in.readLong()) + .setWalForceArchiveTimeout(in.readLong()); if (in.readBoolean()) res.setCheckpointReadLockTimeout(in.readLong()); @@ -2191,6 +2192,7 @@ private static void writeDataStorageConfiguration(BinaryRawWriter w, DataStorage w.writeInt(cfg.getPageSize()); w.writeInt(cfg.getConcurrencyLevel()); w.writeLong(cfg.getWalAutoArchiveAfterInactivity()); + w.writeLong(cfg.getWalForceArchiveTimeout()); if (cfg.getCheckpointReadLockTimeout() != null) { w.writeBoolean(true); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java index d0e0f915312ba..243127c183e99 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java @@ -116,9 +116,12 @@ public class VisorDataStorageConfiguration extends VisorDataTransferObject { /** Time interval (in milliseconds) for rate-based metrics. */ private long metricsRateTimeInterval; - /** Time interval (in milliseconds) for running auto archiving for incompletely WAL segment */ + /** Time interval (in milliseconds) for running auto archiving for incompletely WAL segment. */ private long walAutoArchiveAfterInactivity; + /** Time interval (in milliseconds) for running auto archiving for incompletely WAL segment. */ + private long walForceArchiveTimeout; + /** If true, threads that generate dirty pages too fast during ongoing checkpoint will be throttled. */ private boolean writeThrottlingEnabled; @@ -177,6 +180,7 @@ public VisorDataStorageConfiguration(DataStorageConfiguration cfg) { metricsSubIntervalCount = cfg.getMetricsSubIntervalCount(); metricsRateTimeInterval = cfg.getMetricsRateTimeInterval(); walAutoArchiveAfterInactivity = cfg.getWalAutoArchiveAfterInactivity(); + walForceArchiveTimeout = cfg.getWalForceArchiveTimeout(); writeThrottlingEnabled = cfg.isWriteThrottlingEnabled(); walCompactionEnabled = cfg.isWalCompactionEnabled(); } @@ -370,6 +374,13 @@ public long getWalAutoArchiveAfterInactivity() { return walAutoArchiveAfterInactivity; } + /** + * @return Time in millis. + */ + public long getWalForceArchiveTimeout() { + return walForceArchiveTimeout; + } + /** * @return Flag indicating whether write throttling is enabled. */ @@ -425,6 +436,7 @@ public boolean isWalCompactionEnabled() { out.writeInt(metricsSubIntervalCount); out.writeLong(metricsRateTimeInterval); out.writeLong(walAutoArchiveAfterInactivity); + out.writeLong(walForceArchiveTimeout); out.writeBoolean(writeThrottlingEnabled); out.writeInt(walBufSize); out.writeBoolean(walCompactionEnabled); @@ -460,6 +472,7 @@ public boolean isWalCompactionEnabled() { metricsSubIntervalCount = in.readInt(); metricsRateTimeInterval = in.readLong(); walAutoArchiveAfterInactivity = in.readLong(); + walForceArchiveTimeout = in.readLong(); writeThrottlingEnabled = in.readBoolean(); if (protoVer > V1) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java index 76188f56330c4..005d491bf147d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java @@ -88,6 +88,7 @@ import org.junit.Test; import static java.util.Arrays.fill; +import static org.apache.ignite.cluster.ClusterState.ACTIVE; import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_ARCHIVED; import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_COMPACTED; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD; @@ -119,6 +120,9 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest { */ private int archiveIncompleteSegmentAfterInactivityMs; + /** Force archive timeout in milliseconds. */ + private int forceArchiveSegmentMs; + /** Custom wal mode. */ private WALMode customWalMode; @@ -156,6 +160,9 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest { if (archiveIncompleteSegmentAfterInactivityMs > 0) dsCfg.setWalAutoArchiveAfterInactivity(archiveIncompleteSegmentAfterInactivityMs); + if (forceArchiveSegmentMs > 0) + dsCfg.setWalForceArchiveTimeout(forceArchiveSegmentMs); + String workDir = U.defaultWorkDirectory(); File db = U.resolveWorkDirectory(workDir, DFLT_STORE_DIR, false); File wal = new File(db, "wal"); @@ -342,6 +349,43 @@ private boolean checkWhetherWALRelatedEventFired(int evtType) throws Exception { return evtRecorded.get(); } + /** + * Tests force time out based WAL segment archiving. + * + * @throws Exception if failure occurs. + */ + @Test + public void testForceArchiveSegment() throws Exception { + AtomicBoolean waitingForEvt = new AtomicBoolean(); + + CountDownLatch forceArchiveSegment = new CountDownLatch(1); + + forceArchiveSegmentMs = 1000; + + Ignite ignite = startGrid(); + + ignite.cluster().state(ACTIVE); + + IgniteEvents evts = ignite.events(); + + evts.localListen(e -> { + if (waitingForEvt.get()) + forceArchiveSegment.countDown(); + + return true; + }, EVT_WAL_SEGMENT_ARCHIVED); + + putDummyRecords(ignite, 100); + + waitingForEvt.set(true); // Flag for skipping regular log() and rollOver(). + + boolean recordedAfterSleep = forceArchiveSegment.await(forceArchiveSegmentMs + 1001, TimeUnit.MILLISECONDS); + + stopGrid(); + + assertTrue(recordedAfterSleep); + } + /** * Tests time out based WAL segment archiving. * diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml index bcc8347536866..d4fcad6619396 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml @@ -168,6 +168,7 @@ walArchivePath="abc" walFlushFrequency="00:00:12" walFsyncDelayNanos="13" walHistorySize="14" walMode="Background" walRecordIteratorBufferSize="15" walSegments="16" walSegmentSize="17" walPath="wal-store" writeThrottlingEnabled="true" walAutoArchiveAfterInactivity="00:00:18" + walForceArchiveTimeout="00:00:19" walPageCompression="Zstd"> @@ -887,6 +889,7 @@ private static IgniteConfiguration GetCustomConfig() ConcurrencyLevel = 1, PageSize = 8 * 1024, WalAutoArchiveAfterInactivity = TimeSpan.FromMinutes(5), + WalForceArchiveTimeout = TimeSpan.FromMinutes(6), CheckpointReadLockTimeout = TimeSpan.FromSeconds(9.5), DefaultDataRegionConfiguration = new DataRegionConfiguration { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs index 2e2fc72d249bb..671fd29113557 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs @@ -200,6 +200,7 @@ public DataStorageConfiguration() SystemRegionMaxSize = DefaultSystemRegionMaxSize; PageSize = DefaultPageSize; WalAutoArchiveAfterInactivity = DefaultWalAutoArchiveAfterInactivity; + WalForceArchiveTimeout = DefaultWalAutoArchiveAfterInactivity; MaxWalArchiveSize = DefaultMaxWalArchiveSize; WalPageCompression = DefaultWalPageCompression; ConcurrencyLevel = DefaultConcurrencyLevel; @@ -241,6 +242,7 @@ internal DataStorageConfiguration(IBinaryRawReader reader) PageSize = reader.ReadInt(); ConcurrencyLevel = reader.ReadInt(); WalAutoArchiveAfterInactivity = reader.ReadLongAsTimespan(); + WalForceArchiveTimeout = reader.ReadLongAsTimespan(); CheckpointReadLockTimeout = reader.ReadTimeSpanNullable(); WalPageCompression = (DiskPageCompression)reader.ReadInt(); WalPageCompressionLevel = reader.ReadIntNullable(); @@ -296,6 +298,7 @@ internal void Write(IBinaryRawWriter writer) writer.WriteInt(PageSize); writer.WriteInt(ConcurrencyLevel); writer.WriteTimeSpanAsLong(WalAutoArchiveAfterInactivity); + writer.WriteTimeSpanAsLong(WalForceArchiveTimeout); writer.WriteTimeSpanAsLongNullable(CheckpointReadLockTimeout); writer.WriteInt((int)WalPageCompression); writer.WriteIntNullable(WalPageCompressionLevel); @@ -500,6 +503,12 @@ internal void Write(IBinaryRawWriter writer) [DefaultValue(typeof(TimeSpan), "-00:00:00.001")] public TimeSpan WalAutoArchiveAfterInactivity { get; set; } + /// + /// Gets or sets the time for running auto archiving for incompletely WAL segment. + /// + [DefaultValue(typeof(TimeSpan), "-00:00:00.001")] + public TimeSpan WalForceArchiveTimeout { get; set; } + /// /// Gets or sets the timeout for checkpoint read lock acquisition. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd index 0b35ca410b29c..7d80e4e24e55c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd @@ -2053,6 +2053,11 @@ Inactivity time after which to run WAL segment auto archiving. + + + Time interval (in milliseconds) for force archiving of incompletely WAL segment. + + From db8bb814c5c5a4876b21de7a22126ed74d6ffddf Mon Sep 17 00:00:00 2001 From: Nikolay Date: Thu, 25 Mar 2021 18:34:03 +0300 Subject: [PATCH 02/28] IGNITE-13596 Flag to distinguish DataRecord on primary and backup added (#8904) --- .../GridCommandHandlerClusterByClassTest.java | 3 +- .../ignite/util/GridCommandHandlerTest.java | 3 +- .../pagemem/wal/record/DataEntry.java | 16 ++- .../pagemem/wal/record/DataRecord.java | 2 +- .../pagemem/wal/record/LazyDataEntry.java | 6 +- .../pagemem/wal/record/MvccDataEntry.java | 2 +- .../pagemem/wal/record/UnwrapDataEntry.java | 6 +- .../pagemem/wal/record/WALRecord.java | 23 +++- .../processors/cache/GridCacheMapEntry.java | 42 +++++--- .../GridDistributedTxRemoteAdapter.java | 3 +- .../colocated/GridDhtDetachedCacheEntry.java | 8 +- .../distributed/near/GridNearCacheEntry.java | 8 +- .../GridCacheDatabaseSharedManager.java | 3 + .../reader/StandaloneWalRecordsIterator.java | 7 +- .../serializer/RecordDataV1Serializer.java | 47 +++++--- .../serializer/RecordDataV2Serializer.java | 10 +- .../transactions/IgniteTxLocalAdapter.java | 3 +- .../org/apache/ignite/TestStorageUtils.java | 3 +- ...itePdsSporadicDataRecordsOnBackupTest.java | 2 +- ...pointSimulationWithRealCpDisabledTest.java | 2 +- .../db/wal/IgniteWalRebalanceTest.java | 3 +- .../db/wal/reader/IgniteWalReaderTest.java | 100 +++++++++++++++++- .../testframework/wal/record/RecordUtils.java | 9 ++ .../development/utils/DataEntryWrapper.java | 3 +- .../ignite/development/utils/WalStat.java | 2 +- .../IgniteWalConverterArgumentsTest.java | 4 +- .../IgniteWalConverterSensitiveDataTest.java | 3 +- .../utils/IgniteWalConverterTest.java | 4 +- 28 files changed, 257 insertions(+), 70 deletions(-) diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java index 4014d4e853852..aa55b91c69703 100644 --- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java +++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java @@ -1523,7 +1523,8 @@ private void corruptDataEntry( new GridCacheVersion(), 0L, partId, - updateCntr + updateCntr, + false ); GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ctx.shared().database(); diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index 80c33ee1aec3a..0b525ee8fdd2a 100644 --- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -2630,7 +2630,8 @@ private void corruptDataEntry( new GridCacheVersion(), 0L, partId, - updateCntr + updateCntr, + false ); GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ctx.shared().database(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java index dd05726e78a36..2244897df942b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java @@ -61,6 +61,10 @@ public class DataEntry { @GridToStringInclude protected long partCnt; + /** If {@code true} then change made on primary node. */ + @GridToStringInclude + protected boolean primary; + /** Constructor. */ private DataEntry() { // No-op, used from factory methods. @@ -76,6 +80,7 @@ private DataEntry() { * @param expireTime Expire time. * @param partId Partition ID. * @param partCnt Partition counter. + * @param primary {@code True} if node is primary for partition in the moment of logging. */ public DataEntry( int cacheId, @@ -86,7 +91,8 @@ public DataEntry( GridCacheVersion writeVer, long expireTime, int partId, - long partCnt + long partCnt, + boolean primary ) { this.cacheId = cacheId; this.key = key; @@ -97,6 +103,7 @@ public DataEntry( this.expireTime = expireTime; this.partId = partId; this.partCnt = partCnt; + this.primary = primary; // Only READ, CREATE, UPDATE and DELETE operations should be stored in WAL. assert op == GridCacheOperation.READ || op == GridCacheOperation.CREATE || op == GridCacheOperation.UPDATE || op == GridCacheOperation.DELETE : op; @@ -177,6 +184,13 @@ public long expireTime() { return expireTime; } + /** + * @return {@code True} if node is primary for partition in the moment of logging. + */ + public boolean primary() { + return primary; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(DataEntry.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java index ef6c3bafbc307..2507fd455e006 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java @@ -35,7 +35,7 @@ public class DataRecord extends TimeStampRecord { /** {@inheritDoc} */ @Override public RecordType type() { - return RecordType.DATA_RECORD; + return RecordType.DATA_RECORD_V2; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java index ba2fabc9f0eeb..e771bdfac95be 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java @@ -60,6 +60,7 @@ public class LazyDataEntry extends DataEntry implements MarshalledDataEntry { * @param expireTime Expire time. * @param partId Partition ID. * @param partCnt Partition counter. + * @param primary {@code True} if node is primary for partition in the moment of logging. */ public LazyDataEntry( GridCacheSharedContext cctx, @@ -73,9 +74,10 @@ public LazyDataEntry( GridCacheVersion writeVer, long expireTime, int partId, - long partCnt + long partCnt, + boolean primary ) { - super(cacheId, null, null, op, nearXidVer, writeVer, expireTime, partId, partCnt); + super(cacheId, null, null, op, nearXidVer, writeVer, expireTime, partId, partCnt, primary); this.cctx = cctx; this.keyType = keyType; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MvccDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MvccDataEntry.java index c86593f813679..cecd867c9382a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MvccDataEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MvccDataEntry.java @@ -56,7 +56,7 @@ public MvccDataEntry( long partCnt, MvccVersion mvccVer ) { - super(cacheId, key, val, op, nearXidVer, writeVer, expireTime, partId, partCnt); + super(cacheId, key, val, op, nearXidVer, writeVer, expireTime, partId, partCnt, false); this.mvccVer = mvccVer; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java index ad5e1d0c80835..93951427de2c5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java @@ -48,6 +48,7 @@ public class UnwrapDataEntry extends DataEntry implements UnwrappedDataEntry { * @param partCnt Partition counter. * @param cacheObjValCtx cache object value context for unwrapping objects. * @param keepBinary disable unwrapping for non primitive objects, Binary Objects would be returned instead. + * @param primary {@code True} if node is primary for partition in the moment of logging. */ public UnwrapDataEntry( final int cacheId, @@ -60,8 +61,9 @@ public UnwrapDataEntry( final int partId, final long partCnt, final CacheObjectValueContext cacheObjValCtx, - final boolean keepBinary) { - super(cacheId, key, val, op, nearXidVer, writeVer, expireTime, partId, partCnt); + final boolean keepBinary, + final boolean primary) { + super(cacheId, key, val, op, nearXidVer, writeVer, expireTime, partId, partCnt, primary); this.cacheObjValCtx = cacheObjValCtx; this.keepBinary = keepBinary; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java index f07b71a82fc89..2f95c1dae617c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java @@ -45,7 +45,8 @@ public enum RecordType { /** */ PAGE_RECORD(1, PHYSICAL), - /** */ + /** @deprecated Use {@link #DATA_RECORD_V2} instead. */ + @Deprecated DATA_RECORD(2, LOGICAL), /** Checkpoint (begin) record */ @@ -206,7 +207,11 @@ public enum RecordType { /** Encrypted WAL-record. */ ENCRYPTED_RECORD(52, PHYSICAL), - /** Ecnrypted data record. */ + /** + * Ecnrypted data record. + * @deprecated Use {@link #ENCRYPTED_DATA_RECORD_V3} instead. + */ + @Deprecated ENCRYPTED_DATA_RECORD(53, LOGICAL), /** Mvcc data record. */ @@ -239,7 +244,11 @@ public enum RecordType { /** Encrypted WAL-record. */ ENCRYPTED_RECORD_V2(63, PHYSICAL), - /** Ecnrypted data record. */ + /** + * Ecnrypted data record. + * @deprecated Use {@link #ENCRYPTED_DATA_RECORD_V3} instead. + */ + @Deprecated ENCRYPTED_DATA_RECORD_V2(64, LOGICAL), /** Master key change record containing multiple keys for single cache group. */ @@ -252,7 +261,13 @@ public enum RecordType { PARTITION_META_PAGE_DELTA_RECORD_V3(67, PHYSICAL), /** Index meta page delta record includes encryption status data. */ - INDEX_META_PAGE_DELTA_RECORD(68, PHYSICAL); + INDEX_META_PAGE_DELTA_RECORD(68, PHYSICAL), + + /** Data record V2. */ + DATA_RECORD_V2(69, LOGICAL), + + /** Ecnrypted data record. */ + ENCRYPTED_DATA_RECORD_V3(70, LOGICAL); /** Index for serialization. Should be consistent throughout all versions. */ private final int idx; 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 b0221a78b66af..3807404a23399 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 @@ -1569,7 +1569,7 @@ else if (interceptorVal != val0) updateCntr0 = nextPartitionCounter(tx, updateCntr); if (tx != null && cctx.group().persistenceEnabled() && cctx.group().walEnabled()) - logPtr = logTxUpdate(tx, val, expireTime, updateCntr0); + logPtr = logTxUpdate(tx, val, expireTime, updateCntr0, topVer); update(val, expireTime, ttl, newVer, true); @@ -1791,7 +1791,7 @@ protected Object keyValue(boolean cpy) { updateCntr0 = nextPartitionCounter(tx, updateCntr); if (tx != null && cctx.group().persistenceEnabled() && cctx.group().walEnabled()) - logPtr = logTxUpdate(tx, null, 0, updateCntr0); + logPtr = logTxUpdate(tx, null, 0, updateCntr0, topVer); drReplicate(drType, null, newVer, topVer); @@ -2148,7 +2148,7 @@ else if (ttl != CU.TTL_ZERO) update(updated, expireTime, ttl, ver, true); - logUpdate(op, updated, ver, expireTime, 0); + logUpdate(op, updated, ver, expireTime, 0, cctx.affinity().affinityTopologyVersion()); if (evt) { CacheObject evtOld = null; @@ -2180,7 +2180,7 @@ else if (ttl != CU.TTL_ZERO) update(null, CU.TTL_ETERNAL, CU.EXPIRE_TIME_ETERNAL, ver, true); - logUpdate(op, null, ver, CU.EXPIRE_TIME_ETERNAL, 0); + logUpdate(op, null, ver, CU.EXPIRE_TIME_ETERNAL, 0, cctx.affinity().affinityTopologyVersion()); if (evt) { CacheObject evtOld = null; @@ -3495,7 +3495,8 @@ else if (deletedUnlocked()) ver, expireTime, partition(), - updateCntr + updateCntr, + cctx.affinity().primaryByPartition(cctx.localNode(), partition(), topVer) ))); } } @@ -4323,9 +4324,16 @@ protected boolean storeValue( * @param writeVer Write version. * @param expireTime Expire time. * @param updCntr Update counter. + * @param topVer Topology version. */ - protected void logUpdate(GridCacheOperation op, CacheObject val, GridCacheVersion writeVer, long expireTime, long updCntr) - throws IgniteCheckedException { + protected void logUpdate( + GridCacheOperation op, + CacheObject val, + GridCacheVersion writeVer, + long expireTime, + long updCntr, + AffinityTopologyVersion topVer + ) throws IgniteCheckedException { // We log individual updates only in ATOMIC cache. assert cctx.atomic(); @@ -4340,7 +4348,8 @@ protected void logUpdate(GridCacheOperation op, CacheObject val, GridCacheVersio writeVer, expireTime, partition(), - updCntr))); + updCntr, + cctx.affinity().primaryByPartition(cctx.localNode(), partition(), topVer)))); } catch (StorageException e) { throw new IgniteCheckedException("Failed to log ATOMIC cache update [key=" + key + ", op=" + op + @@ -4353,10 +4362,16 @@ protected void logUpdate(GridCacheOperation op, CacheObject val, GridCacheVersio * @param val Value. * @param expireTime Expire time (or 0 if not applicable). * @param updCntr Update counter. + * @param topVer Topology version. * @throws IgniteCheckedException In case of log failure. */ - protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr) - throws IgniteCheckedException { + protected WALPointer logTxUpdate( + IgniteInternalTx tx, + CacheObject val, + long expireTime, + long updCntr, + AffinityTopologyVersion topVer + ) throws IgniteCheckedException { assert cctx.transactional() && !cctx.transactionalSnapshot(); if (tx.local()) { // For remote tx we log all updates in batch: GridDistributedTxRemoteAdapter.commitIfLocked() @@ -4375,7 +4390,8 @@ protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expi tx.writeVersion(), expireTime, key.partition(), - updCntr))); + updCntr, + cctx.affinity().primaryByPartition(cctx.localNode(), partition(), topVer)))); } else return null; @@ -6484,7 +6500,7 @@ else if (interceptorVal != updated0) { long updateCntr0 = entry.nextPartitionCounter(topVer, primary, false, updateCntr); - entry.logUpdate(op, updated, newVer, newExpireTime, updateCntr0); + entry.logUpdate(op, updated, newVer, newExpireTime, updateCntr0, topVer); if (!entry.isNear()) { newRow = entry.localPartition().dataStore().createRow( @@ -6571,7 +6587,7 @@ private void remove(@Nullable GridCacheVersionConflictContext conflictCtx, long updateCntr0 = entry.nextPartitionCounter(topVer, primary, false, updateCntr); - entry.logUpdate(op, null, newVer, 0, updateCntr0); + entry.logUpdate(op, null, newVer, 0, updateCntr0, topVer); if (oldVal != null) { assert !entry.deletedUnlocked(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index 173888bd0025e..7cfe869407eec 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -631,7 +631,8 @@ else if (conflictCtx.isMerge()) { writeVersion(), 0, txEntry.key().partition(), - txEntry.updateCounter() + txEntry.updateCounter(), + cacheCtx.affinity().primaryByPartition(cctx.localNode(), txEntry.key().partition(), topVer) ), txEntry ) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java index 48b8e8ece2eb8..8e46629acb2cc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.colocated; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheOperation; @@ -73,13 +74,14 @@ public void resetFromPrimary(CacheObject val, GridCacheVersion ver) { } /** {@inheritDoc} */ - @Override protected void logUpdate(GridCacheOperation op, CacheObject val, GridCacheVersion writeVer, long expireTime, long updCntr) throws IgniteCheckedException { + @Override protected void logUpdate(GridCacheOperation op, CacheObject val, GridCacheVersion writeVer, + long expireTime, long updCntr, AffinityTopologyVersion topVer) { // No-op for detached entries, index is updated on primary or backup nodes. } /** {@inheritDoc} */ - @Override protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr) - throws IgniteCheckedException { + @Override protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr, + AffinityTopologyVersion topVer) { return null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java index d8b161553a3e4..fc70aa3d6c420 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java @@ -471,14 +471,14 @@ public boolean loadedValue(@Nullable IgniteInternalTx tx, } /** {@inheritDoc} */ - @Override protected void logUpdate(GridCacheOperation op, CacheObject val, GridCacheVersion ver, long expireTime, long updCntr) - throws IgniteCheckedException { + @Override protected void logUpdate(GridCacheOperation op, CacheObject val, GridCacheVersion ver, long expireTime, + long updCntr, AffinityTopologyVersion topVer) { // No-op: queries are disabled for near cache. } /** {@inheritDoc} */ - @Override protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr) - throws IgniteCheckedException { + @Override protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr, + AffinityTopologyVersion topVer) { return null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index 8fa76c80903d1..8c0228eb4d22e 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -2496,6 +2496,7 @@ public void applyUpdatesOnRecovery( switch (rec.type()) { case MVCC_DATA_RECORD: case DATA_RECORD: + case DATA_RECORD_V2: checkpointReadLock(); try { @@ -2635,8 +2636,10 @@ private RestoreLogicalState applyLogicalUpdates( case MVCC_DATA_RECORD: case DATA_RECORD: + case DATA_RECORD_V2: case ENCRYPTED_DATA_RECORD: case ENCRYPTED_DATA_RECORD_V2: + case ENCRYPTED_DATA_RECORD_V3: DataRecord dataRec = (DataRecord)rec; for (DataEntry dataEntry : dataRec.writeEntries()) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java index 472afba7192fb..cfaa67031349a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java @@ -354,7 +354,9 @@ private boolean checkBounds(long idx) { GridKernalContext kernalCtx = sharedCtx.kernalContext(); IgniteCacheObjectProcessor processor = kernalCtx.cacheObjects(); - if (processor != null && (rec.type() == RecordType.DATA_RECORD || rec.type() == RecordType.MVCC_DATA_RECORD)) { + if (processor != null && (rec.type() == RecordType.DATA_RECORD + || rec.type() == RecordType.DATA_RECORD_V2 + || rec.type() == RecordType.MVCC_DATA_RECORD)) { try { return postProcessDataRecord((DataRecord)rec, kernalCtx, processor); } @@ -498,7 +500,8 @@ private DataEntry unwrapDataEntry(CacheObjectContext coCtx, DataEntry dataEntry, dataEntry.partitionId(), dataEntry.partitionCounter(), coCtx, - keepBinary); + keepBinary, + dataEntry.primary()); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java index bc5f9e3b2b342..6ad08ddd2044c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java @@ -43,6 +43,7 @@ import org.apache.ignite.internal.pagemem.wal.record.MasterKeyChangeRecordV2; import org.apache.ignite.internal.pagemem.wal.record.MemoryRecoveryRecord; import org.apache.ignite.internal.pagemem.wal.record.MetastoreDataRecord; +import org.apache.ignite.internal.pagemem.wal.record.MvccDataEntry; import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot; import org.apache.ignite.internal.pagemem.wal.record.ReencryptionStartRecord; import org.apache.ignite.internal.pagemem.wal.record.TxRecord; @@ -121,7 +122,9 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD_V2; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_DATA_RECORD_V2; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_DATA_RECORD_V3; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_RECORD; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_RECORD_V2; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.MASTER_KEY_CHANGE_RECORD_V2; @@ -406,7 +409,7 @@ assert record instanceof PageSnapshot; case PARTITION_DESTROY: return /*cacheId*/4 + /*partId*/4; - case DATA_RECORD: + case DATA_RECORD_V2: DataRecord dataRec = (DataRecord)record; return 4 + dataSize(dataRec); @@ -667,12 +670,13 @@ WALRecord readPlainRecord(RecordType type, ByteBufferBackedDataInput in, break; case DATA_RECORD: + case DATA_RECORD_V2: int entryCnt = in.readInt(); List entries = new ArrayList<>(entryCnt); for (int i = 0; i < entryCnt; i++) - entries.add(readPlainDataEntry(in)); + entries.add(readPlainDataEntry(in, type)); res = new DataRecord(entries, 0L); @@ -680,12 +684,13 @@ WALRecord readPlainRecord(RecordType type, ByteBufferBackedDataInput in, case ENCRYPTED_DATA_RECORD: case ENCRYPTED_DATA_RECORD_V2: + case ENCRYPTED_DATA_RECORD_V3: entryCnt = in.readInt(); entries = new ArrayList<>(entryCnt); for (int i = 0; i < entryCnt; i++) - entries.add(readEncryptedDataEntry(in, type == ENCRYPTED_DATA_RECORD_V2)); + entries.add(readEncryptedDataEntry(in, type)); res = new DataRecord(entries, 0L); @@ -1352,6 +1357,7 @@ void writePlainRecord(WALRecord rec, ByteBuffer buf) throws IgniteCheckedExcepti break; case DATA_RECORD: + case DATA_RECORD_V2: DataRecord dataRec = (DataRecord)rec; buf.putInt(dataRec.writeEntries().size()); @@ -1956,6 +1962,9 @@ else if (!entry.value().putValue(buf)) buf.putInt(entry.partitionId()); buf.putLong(entry.partitionCounter()); buf.putLong(entry.expireTime()); + + if (!(entry instanceof MvccDataEntry)) + buf.put(entry.primary() ? (byte)1 : 0); } /** @@ -2002,14 +2011,16 @@ private static void putRow(ByteBuffer buf, byte[] rowBytes) { /** * @param in Input to read from. - * @param readKeyId If {@code true} encryption key identifier will be read from {@code in}. + * @param recType Record type. * @return Read entry. * @throws IOException If failed. * @throws IgniteCheckedException If failed. */ - DataEntry readEncryptedDataEntry(ByteBufferBackedDataInput in, boolean readKeyId) throws IOException, IgniteCheckedException { + DataEntry readEncryptedDataEntry(ByteBufferBackedDataInput in, RecordType recType) throws IOException, IgniteCheckedException { boolean needDecryption = in.readByte() == ENCRYPTED; + RecordType dataRecordType = recType == ENCRYPTED_DATA_RECORD_V3 ? DATA_RECORD_V2 : DATA_RECORD; + if (needDecryption) { if (encSpi == null) { skipEncryptedRecord(in, false); @@ -2017,22 +2028,23 @@ DataEntry readEncryptedDataEntry(ByteBufferBackedDataInput in, boolean readKeyId return new EncryptedDataEntry(); } - T3 clData = readEncryptedData(in, false, readKeyId); + T3 clData = readEncryptedData(in, false, + recType == ENCRYPTED_DATA_RECORD_V2 || recType == ENCRYPTED_DATA_RECORD_V3); if (clData.get1() == null) return null; - return readPlainDataEntry(clData.get1()); + return readPlainDataEntry(clData.get1(), dataRecordType); } - return readPlainDataEntry(in); + return readPlainDataEntry(in, dataRecordType); } /** * @param in Input to read from. * @return Read entry. */ - DataEntry readPlainDataEntry(ByteBufferBackedDataInput in) throws IOException, IgniteCheckedException { + DataEntry readPlainDataEntry(ByteBufferBackedDataInput in, RecordType type) throws IOException, IgniteCheckedException { int cacheId = in.readInt(); int keySize = in.readInt(); @@ -2061,6 +2073,7 @@ DataEntry readPlainDataEntry(ByteBufferBackedDataInput in) throws IOException, I int partId = in.readInt(); long partCntr = in.readLong(); long expireTime = in.readLong(); + boolean primary = type == DATA_RECORD_V2 && in.readByte() == (byte)1; GridCacheContext cacheCtx = cctx.cacheContext(cacheId); @@ -2083,7 +2096,8 @@ DataEntry readPlainDataEntry(ByteBufferBackedDataInput in) throws IOException, I writeVer, expireTime, partId, - partCntr + partCntr, + primary ); } else @@ -2099,7 +2113,9 @@ DataEntry readPlainDataEntry(ByteBufferBackedDataInput in) throws IOException, I writeVer, expireTime, partId, - partCntr); + partCntr, + primary + ); } /** @@ -2113,10 +2129,10 @@ RecordType recordType(WALRecord rec) { if (needEncryption(rec)) return ENCRYPTED_RECORD_V2; - if (rec.type() != DATA_RECORD) + if (rec.type() != DATA_RECORD && rec.type() != DATA_RECORD_V2) return rec.type(); - return isDataRecordEncrypted((DataRecord)rec) ? ENCRYPTED_DATA_RECORD_V2 : DATA_RECORD; + return isDataRecordEncrypted((DataRecord)rec) ? ENCRYPTED_DATA_RECORD_V3 : rec.type(); } /** @@ -2235,7 +2251,8 @@ protected int entrySize(DataEntry entry) throws IgniteCheckedException { /*write ver*/CacheVersionIO.size(entry.writeVersion(), false) + /*part ID*/4 + /*expire Time*/8 + - /*part cnt*/8; + /*part cnt*/8 + + /*primary*/(entry instanceof MvccDataEntry ? 0 : 1); } /** @@ -2268,7 +2285,7 @@ private int cacheStatesSize(Map states) { public static class EncryptedDataEntry extends DataEntry { /** Constructor. */ EncryptedDataEntry() { - super(0, null, null, READ, null, null, 0, 0, 0); + super(0, null, null, READ, null, null, 0, 0, 0, false); } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java index 8622629497b19..08f132acd648d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java @@ -56,8 +56,6 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; -import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_DATA_RECORD_V2; - /** * Record data V2 serializer. */ @@ -96,6 +94,7 @@ public RecordDataV2Serializer(GridCacheSharedContext cctx) { return 4/*entry count*/ + 8/*timestamp*/ + dataSize((DataRecord)rec); case DATA_RECORD: + case DATA_RECORD_V2: return super.plainSize(rec) + 8/*timestamp*/; case SNAPSHOT: @@ -160,13 +159,14 @@ public RecordDataV2Serializer(GridCacheSharedContext cctx) { return cpRec; case DATA_RECORD: + case DATA_RECORD_V2: int entryCnt = in.readInt(); long timeStamp = in.readLong(); List entries = new ArrayList<>(entryCnt); for (int i = 0; i < entryCnt; i++) - entries.add(readPlainDataEntry(in)); + entries.add(readPlainDataEntry(in, type)); return new DataRecord(entries, timeStamp); @@ -183,13 +183,14 @@ public RecordDataV2Serializer(GridCacheSharedContext cctx) { case ENCRYPTED_DATA_RECORD: case ENCRYPTED_DATA_RECORD_V2: + case ENCRYPTED_DATA_RECORD_V3: entryCnt = in.readInt(); timeStamp = in.readLong(); entries = new ArrayList<>(entryCnt); for (int i = 0; i < entryCnt; i++) - entries.add(readEncryptedDataEntry(in, type == ENCRYPTED_DATA_RECORD_V2)); + entries.add(readEncryptedDataEntry(in, type)); return new DataRecord(entries, timeStamp); @@ -261,6 +262,7 @@ public RecordDataV2Serializer(GridCacheSharedContext cctx) { case MVCC_DATA_RECORD: case DATA_RECORD: + case DATA_RECORD_V2: DataRecord dataRec = (DataRecord)rec; buf.putInt(dataRec.writeEntries().size()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index 7b6cd7f10dc98..208a9a060df36 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -861,7 +861,8 @@ else if (op == READ) { writeVersion(), 0, txEntry.key().partition(), - txEntry.updateCounter()))); + txEntry.updateCounter(), + cacheCtx.affinity().primaryByPartition(cctx.localNode(), txEntry.key().partition(), topVer)))); } ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); diff --git a/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java b/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java index 17ff24197d0c8..e0f415d44e8ed 100644 --- a/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java +++ b/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java @@ -82,7 +82,8 @@ public static void corruptDataEntry( ver, 0L, partId, - updateCntr + updateCntr, + false ); IgniteCacheDatabaseSharedManager db = ctx.shared().database(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSporadicDataRecordsOnBackupTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSporadicDataRecordsOnBackupTest.java index d82d26327c20e..55b6b616434ce 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSporadicDataRecordsOnBackupTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsSporadicDataRecordsOnBackupTest.java @@ -158,7 +158,7 @@ private long findSporadicDataRecords(String nodeFolderName) throws IgniteChecked params.bufferSize(1024 * 1024); params.filesOrDirs(walDir, walArchiveDir); - params.filter((type, pointer) -> type == WALRecord.RecordType.DATA_RECORD); + params.filter((type, pointer) -> type == WALRecord.RecordType.DATA_RECORD_V2); int cacheId = CU.cacheId(TX_CACHE_NAME); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java index f829b90d69c8d..a8dd46c141132 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java @@ -358,7 +358,7 @@ private void checkDataWalEntries(boolean mvcc) throws Exception { cctx.affinity().partition(i), i, new MvccVersionImpl(1000L, 10L, i + 1 /* Non-zero */)) : new DataEntry(cctx.cacheId(), key, val, op, null, cctx.cache().nextVersion(), 0L, - cctx.affinity().partition(i), i)); + cctx.affinity().partition(i), i, false)); } UUID cpId = UUID.randomUUID(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java index 7567705057640..422cc23de5fc2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java @@ -763,7 +763,8 @@ public void testSwitchHistoricalRebalanceToFullWhileIteratingOverWAL() throws Ex new GridCacheVersion(0, 1, 1, 0), 0, 0, - 0 + 0, + false ))); File walDir = U.field(walMgr, "walWorkDir"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java index 005d491bf147d..5c6c7c00526a7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java @@ -91,7 +91,7 @@ import static org.apache.ignite.cluster.ClusterState.ACTIVE; import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_ARCHIVED; import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_COMPACTED; -import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD_V2; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.MVCC_DATA_RECORD; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.CREATE; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE; @@ -132,6 +132,12 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest { /** Whether to enable WAL archive compaction. */ private boolean enableWalCompaction; + /** Backup count. */ + private int backupCnt; + + /** DataEntry from primary flag. */ + private boolean primary = true; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); @@ -142,6 +148,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest { ccfg.setRebalanceMode(CacheRebalanceMode.SYNC); ccfg.setAffinity(new RendezvousAffinityFunction(false, 32)); ccfg.setIndexedTypes(Integer.class, IndexedObject.class); + ccfg.setBackups(backupCnt); cfg.setCacheConfiguration(ccfg); @@ -266,13 +273,16 @@ private int iterateAndCount(WALIterator walIter) throws IgniteCheckedException { WALRecord walRecord = tup.get2(); - if (walRecord.type() == DATA_RECORD || walRecord.type() == MVCC_DATA_RECORD) { + if (walRecord.type() == DATA_RECORD_V2 || walRecord.type() == MVCC_DATA_RECORD) { DataRecord record = (DataRecord)walRecord; for (DataEntry entry : record.writeEntries()) { KeyCacheObject key = entry.key(); CacheObject val = entry.value(); + if (walRecord.type() == DATA_RECORD_V2) + assertEquals(primary, entry.primary()); + if (DUMP_RECORDS) log.info("Op: " + entry.op() + ", Key: " + key + ", Value: " + val); } @@ -1040,6 +1050,87 @@ private void runRemoveOperationTest(CacheAtomicityMode mode) throws Exception { deletesFound != null && deletesFound > 0); } + /** + * Tests transaction generation and WAL for putAll cache operation. + * + * @throws Exception if failed. + */ + @Test + public void testPrimaryFlagOnTwoNodes() throws Exception { + backupCnt = 1; + + IgniteEx ignite = startGrid("node0"); + Ignite ignite1 = startGrid(1); + + ignite.cluster().state(ACTIVE); + + IgniteCache cache = ignite.cache(CACHE_NAME); + + backupCnt = 0; + + int cntEntries = 100; + + List keys = findKeys(ignite.localNode(), cache, cntEntries, 0, 0); + + Map map = new TreeMap<>(); + + for (Integer key : keys) + map.putIfAbsent(key, new IndexedObject(key)); + + cache.putAll(map); + + ignite.cluster().active(false); + + String subfolderName1 = genDbSubfolderName(ignite, 0); + String subfolderName2 = genDbSubfolderName(ignite1, 1); + + stopAllGrids(); + + String workDir = U.defaultWorkDirectory(); + + IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log); + + Map operationsFound = new EnumMap<>(GridCacheOperation.class); + + IgniteInClosure drHnd = dataRecord -> { + List entries = dataRecord.writeEntries(); + + for (DataEntry entry : entries) { + GridCacheOperation op = entry.op(); + Integer cnt = operationsFound.get(op); + + operationsFound.put(op, cnt == null ? 1 : (cnt + 1)); + } + }; + + scanIterateAndCount( + factory, + createIteratorParametersBuilder(workDir, subfolderName1) + .filesOrDirs( + workDir + "/db/wal/" + subfolderName1, + workDir + "/db/wal/archive/" + subfolderName1 + ), + 1, + 1, + null, drHnd + ); + + primary = false; + + scanIterateAndCount( + factory, + createIteratorParametersBuilder(workDir, subfolderName2) + .filesOrDirs( + workDir + "/db/wal/" + subfolderName2, + workDir + "/db/wal/archive/" + subfolderName2 + ), + 1, + 1, + null, + drHnd + ); + } + /** * Tests transaction generation and WAL for putAll cache operation. * @@ -1397,7 +1488,7 @@ private Map iterateAndCountDataRecord( //noinspection EnumSwitchStatementWhichMissesCases switch (type) { - case DATA_RECORD: + case DATA_RECORD_V2: // Fallthrough. case MVCC_DATA_RECORD: { assert walRecord instanceof DataRecord; @@ -1410,6 +1501,9 @@ private Map iterateAndCountDataRecord( List entries = dataRecord.writeEntries(); for (DataEntry entry : entries) { + if (walRecord.type() == DATA_RECORD_V2) + assertEquals(primary, entry.primary()); + GridCacheVersion globalTxId = entry.nearXidVersion(); Object unwrappedKeyObj; diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/wal/record/RecordUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/wal/record/RecordUtils.java index 27aa08a7a5973..41bf1ef555ffa 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/wal/record/RecordUtils.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/wal/record/RecordUtils.java @@ -112,8 +112,10 @@ import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_PAGE_SET_FREE_LIST_PAGE; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_PAGE_UPDATE_RECORD; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD_V2; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_DATA_RECORD; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_DATA_RECORD_V2; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_DATA_RECORD_V3; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_RECORD; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.ENCRYPTED_RECORD_V2; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.EXCHANGE; @@ -170,6 +172,7 @@ public class RecordUtils { put(TX_RECORD, RecordUtils::buildTxRecord); put(PAGE_RECORD, RecordUtils::buildPageSnapshot); put(DATA_RECORD, RecordUtils::buildDataRecord); + put(DATA_RECORD_V2, RecordUtils::buildDataRecord); put(CHECKPOINT_RECORD, RecordUtils::buildCheckpointRecord); put(HEADER_RECORD, RecordUtils::buildHeaderRecord); put(INIT_NEW_PAGE_RECORD, RecordUtils::buildInitNewPageRecord); @@ -231,6 +234,7 @@ public class RecordUtils { put(ENCRYPTED_DATA_RECORD, RecordUtils::buildEncryptedDataRecord); put(ENCRYPTED_RECORD_V2, RecordUtils::buildEncryptedRecordV2); put(ENCRYPTED_DATA_RECORD_V2, RecordUtils::buildEncryptedDataRecordV2); + put(ENCRYPTED_DATA_RECORD_V3, RecordUtils::buildEncryptedDataRecordV3); put(MVCC_DATA_RECORD, RecordUtils::buildMvccDataRecord); put(MVCC_TX_RECORD, RecordUtils::buildMvccTxRecord); put(CONSISTENT_CUT, RecordUtils::buildConsistentCutRecord); @@ -593,6 +597,11 @@ public static UnsupportedWalRecord buildEncryptedDataRecordV2() { return new UnsupportedWalRecord(ENCRYPTED_DATA_RECORD_V2); } + /** **/ + public static UnsupportedWalRecord buildEncryptedDataRecordV3() { + return new UnsupportedWalRecord(ENCRYPTED_DATA_RECORD_V3); + } + /** **/ public static MvccDataRecord buildMvccDataRecord() { return new MvccDataRecord(Collections.emptyList(), 1); diff --git a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java index 49edc6ca0dbf7..25e404f0abbf8 100644 --- a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java +++ b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java @@ -65,7 +65,8 @@ public DataEntryWrapper( dataEntry.writeVersion(), dataEntry.expireTime(), dataEntry.partitionId(), - dataEntry.partitionCounter() + dataEntry.partitionCounter(), + dataEntry.primary() ); this.source = dataEntry; diff --git a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/WalStat.java b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/WalStat.java index 993ae8803d2fd..cbed361d8026d 100644 --- a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/WalStat.java +++ b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/WalStat.java @@ -127,7 +127,7 @@ void registerRecord(WALRecord record, WALPointer walPointer, boolean workDir) { if (type == WALRecord.RecordType.PAGE_RECORD) registerPageSnapshot((PageSnapshot)record); - else if (type == WALRecord.RecordType.DATA_RECORD || type == WALRecord.RecordType.MVCC_DATA_RECORD) + else if (type == WALRecord.RecordType.DATA_RECORD || type == WALRecord.RecordType.DATA_RECORD_V2 || type == WALRecord.RecordType.MVCC_DATA_RECORD) registerDataRecord((DataRecord)record); else if (type == WALRecord.RecordType.TX_RECORD || type == WALRecord.RecordType.MVCC_TX_RECORD) registerTxRecord((TxRecord)record); diff --git a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterArgumentsTest.java b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterArgumentsTest.java index b5c697613e6a0..928706f7cae42 100644 --- a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterArgumentsTest.java +++ b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterArgumentsTest.java @@ -391,7 +391,7 @@ public void testParse() throws IOException { "binaryMetadataFileStoreDir=" + binaryMetadataDir.getAbsolutePath(), "marshallerMappingFileStoreDir=" + marshallerDir.getAbsolutePath(), "keepBinary=false", - "recordTypes=DATA_RECORD,TX_RECORD", + "recordTypes=DATA_RECORD_V2,TX_RECORD", "walTimeFromMillis=1575158400000", "walTimeToMillis=1577836740999", "recordContainsText=search string", @@ -406,7 +406,7 @@ public void testParse() throws IOException { Assert.assertEquals(binaryMetadataDir, parseArgs.getBinaryMetadataFileStoreDir()); Assert.assertEquals(marshallerDir, parseArgs.getMarshallerMappingFileStoreDir()); Assert.assertFalse(parseArgs.isKeepBinary()); - Assert.assertTrue(parseArgs.getRecordTypes().contains(WALRecord.RecordType.DATA_RECORD)); + Assert.assertTrue(parseArgs.getRecordTypes().contains(WALRecord.RecordType.DATA_RECORD_V2)); Assert.assertTrue(parseArgs.getRecordTypes().contains(WALRecord.RecordType.TX_RECORD)); Assert.assertEquals(1575158400000L, (long)parseArgs.getFromTime()); Assert.assertEquals(1577836740999L, (long)parseArgs.getToTime()); diff --git a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java index bb4cab959a750..487f791b5479d 100644 --- a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java +++ b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java @@ -298,7 +298,8 @@ private Collection withSensitiveData() { new GridCacheVersion(), 0, 0, - 0 + 0, + false ); byte[] sensitiveDataBytes = SENSITIVE_DATA_VALUE_PREFIX.getBytes(StandardCharsets.UTF_8); diff --git a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterTest.java b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterTest.java index 896dc6742e942..063e08f1ff22d 100644 --- a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterTest.java +++ b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterTest.java @@ -316,7 +316,7 @@ public void testIgniteWalConverterWithBrokenWal() throws Exception { final int len = Integer.reverseBytes(raf.readInt()); - if (recordTypeIndex == WALRecord.RecordType.DATA_RECORD.index()) { + if (recordTypeIndex == WALRecord.RecordType.DATA_RECORD_V2.index()) { int i = 0; int b; @@ -426,7 +426,7 @@ public void testIgniteWalConverterWithUnreadableWal() throws Exception { if (recordTypeIndex > 0) { recordTypeIndex--; - if (recordTypeIndex == WALRecord.RecordType.DATA_RECORD.index()) { + if (recordTypeIndex == WALRecord.RecordType.DATA_RECORD_V2.index()) { find++; if (find == 2) { From e579b1e557c597d159945c252ea1af793046b1ce Mon Sep 17 00:00:00 2001 From: Nikolay Date: Fri, 26 Mar 2021 10:51:20 +0300 Subject: [PATCH 03/28] IGNITE-14360 Refactor FileLockHolder for reusage (#8905) --- .../cache/persistence/FileLockHolder.java | 202 ++++++++++++++++++ .../GridCacheDatabaseSharedManager.java | 169 ++------------- .../filename/PdsConsistentIdProcessor.java | 22 +- .../filename/PdsFolderSettings.java | 8 +- 4 files changed, 238 insertions(+), 163 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FileLockHolder.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FileLockHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FileLockHolder.java new file mode 100644 index 0000000000000..b7b3d5cd7f440 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FileLockHolder.java @@ -0,0 +1,202 @@ +/* + * 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.persistence; + +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.channels.OverlappingFileLockException; +import java.nio.file.Paths; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * Abstract file lock holder. + * Implementations should provide {@link #lockId()} that will appear in error message for concurrent processes + * that will try to lock the same file and {@link #warningMessage(String)} to print on each lock try. + * + * @see GridCacheDatabaseSharedManager.NodeFileLockHolder + */ +public abstract class FileLockHolder implements AutoCloseable { + /** Lock file name. */ + private static final String lockFileName = "lock"; + + /** File. */ + private final File file; + + /** Channel. */ + private final RandomAccessFile lockFile; + + /** Lock. */ + private volatile FileLock lock; + + /** Logger. */ + private final IgniteLogger log; + + /** + * @param rootDir Root directory for lock file. + * @param log Log. + */ + protected FileLockHolder(String rootDir, IgniteLogger log) { + try { + file = Paths.get(rootDir, lockFileName).toFile(); + + lockFile = new RandomAccessFile(file, "rw"); + + this.log = log; + } + catch (IOException e) { + throw new IgniteException(e); + } + } + + /** + * This id will appear in error message of concurrent processes that will try to lock on the same file. + * + * @return Lock ID to store in the file. + */ + public abstract String lockId(); + + /** + * @param lockId Existing lock id. + * @return Warning message. + */ + protected abstract String warningMessage(String lockId); + + /** + * @param lockWaitTimeMillis During which time thread will try capture file lock. + * @throws IgniteCheckedException If failed to capture file lock. + */ + public void tryLock(long lockWaitTimeMillis) throws IgniteCheckedException { + assert lockFile != null; + + FileChannel ch = lockFile.getChannel(); + + String failMsg; + + try { + String content = null; + + // Try to get lock, if not available wait 1 sec and re-try. + for (int i = 0; i < lockWaitTimeMillis; i += 1000) { + try { + lock = ch.tryLock(0, 1, false); + + if (lock != null && lock.isValid()) { + writeContent(lockId()); + + return; + } + } + catch (OverlappingFileLockException ignore) { + if (content == null) + content = readContent(); + + log.warning(warningMessage(content)); + } + + U.sleep(1000); + } + + if (content == null) + content = readContent(); + + failMsg = "Failed to acquire file lock [holder=" + content + ", time=" + (lockWaitTimeMillis / 1000) + + " sec, path=" + file.getAbsolutePath() + ']'; + } + catch (Exception e) { + throw new IgniteCheckedException(e); + } + + if (failMsg != null) + throw new IgniteCheckedException(failMsg); + } + + /** + * Write node id (who captured lock) into lock file. + * + * @param content Node id. + * @throws IOException if some fail while write node it. + */ + private void writeContent(String content) throws IOException { + FileChannel ch = lockFile.getChannel(); + + byte[] bytes = content.getBytes(); + + ByteBuffer buf = ByteBuffer.allocate(bytes.length); + buf.put(bytes); + + buf.flip(); + + ch.write(buf, 1); + + ch.force(false); + } + + /** + * + */ + private String readContent() throws IOException { + FileChannel ch = lockFile.getChannel(); + + ByteBuffer buf = ByteBuffer.allocate((int)(ch.size() - 1)); + + ch.read(buf, 1); + + String content = new String(buf.array()); + + buf.clear(); + + return content; + } + + /** + * Locked or not. + */ + public boolean isLocked() { + return lock != null && lock.isValid(); + } + + /** + * Releases file lock + */ + public void release() { + U.releaseQuiet(lock); + } + + /** + * Closes file channel + */ + @Override public void close() { + release(); + + U.closeQuiet(lockFile); + } + + /** + * @return Absolute path to lock file. + */ + public String lockPath() { + return file.getAbsolutePath(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index 8c0228eb4d22e..15557ec633782 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -19,15 +19,10 @@ import java.io.File; import java.io.IOException; -import java.io.RandomAccessFile; import java.io.Serializable; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.FileChannel; -import java.nio.channels.FileLock; -import java.nio.channels.OverlappingFileLockException; import java.nio.file.Path; -import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -299,7 +294,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan * Lock holder for compatible folders mode. Null if lock holder was created at start node.
* In this case lock is held on PDS resover manager and it is not required to manage locking here */ - @Nullable private FileLockHolder fileLockHolder; + @Nullable private NodeFileLockHolder fileLockHolder; /** Lock wait time. */ private final long lockWaitTime; @@ -554,7 +549,7 @@ private DataRegionConfiguration createDefragmentationMappingRegionConfig(long re () -> cpFreqDeviation.getOrDefault(DEFAULT_CHECKPOINT_DEVIATION) ); - final FileLockHolder preLocked = kernalCtx.pdsFolderResolver() + final NodeFileLockHolder preLocked = kernalCtx.pdsFolderResolver() .resolveFolders() .getLockedFileLockHolder(); @@ -735,12 +730,12 @@ private Collection cacheGroupContexts() { /** * @param preLocked Pre-locked file lock holder. */ - private void acquireFileLock(FileLockHolder preLocked) throws IgniteCheckedException { + private void acquireFileLock(NodeFileLockHolder preLocked) throws IgniteCheckedException { if (cctx.kernalContext().clientNode()) return; fileLockHolder = preLocked == null ? - new FileLockHolder(storeMgr.workDir().getPath(), cctx.kernalContext(), log) : preLocked; + new NodeFileLockHolder(storeMgr.workDir().getPath(), cctx.kernalContext(), log) : preLocked; if (!fileLockHolder.isLocked()) { if (log.isDebugEnabled()) @@ -2945,53 +2940,25 @@ public AtomicLong pageListCacheLimitHolder(DataRegion dataRegion) { } /** - * + * Node file lock holder. */ - public static class FileLockHolder implements AutoCloseable { - /** Lock file name. */ - private static final String lockFileName = "lock"; - - /** File. */ - private File file; - - /** Channel. */ - private RandomAccessFile lockFile; - - /** Lock. */ - private volatile FileLock lock; - + public static class NodeFileLockHolder extends FileLockHolder { /** Kernal context to generate Id of locked node in file. */ - @NotNull private GridKernalContext ctx; - - /** Logger. */ - private IgniteLogger log; + @NotNull private final GridKernalContext ctx; /** - * @param path Path. + * @param rootDir Root directory for lock file. + * @param ctx Kernal context. + * @param log Log. */ - public FileLockHolder(String path, @NotNull GridKernalContext ctx, IgniteLogger log) { - try { - file = Paths.get(path, lockFileName).toFile(); - - lockFile = new RandomAccessFile(file, "rw"); + public NodeFileLockHolder(String rootDir, @NotNull GridKernalContext ctx, IgniteLogger log) { + super(rootDir, log); - this.ctx = ctx; - this.log = log; - } - catch (IOException e) { - throw new IgniteException(e); - } + this.ctx = ctx; } - /** - * @param lockWaitTimeMillis During which time thread will try capture file lock. - * @throws IgniteCheckedException If failed to capture file lock. - */ - public void tryLock(long lockWaitTimeMillis) throws IgniteCheckedException { - assert lockFile != null; - - FileChannel ch = lockFile.getChannel(); - + /** {@inheritDoc} */ + @Override public String lockId() { SB sb = new SB(); //write node id @@ -3022,108 +2989,14 @@ public void tryLock(long lockWaitTimeMillis) throws IgniteCheckedException { sb.a("]"); - String failMsg; - - try { - String content = null; - - // Try to get lock, if not available wait 1 sec and re-try. - for (int i = 0; i < lockWaitTimeMillis; i += 1000) { - try { - lock = ch.tryLock(0, 1, false); - - if (lock != null && lock.isValid()) { - writeContent(sb.toString()); - - return; - } - } - catch (OverlappingFileLockException ignore) { - if (content == null) - content = readContent(); - - log.warning("Failed to acquire file lock. Will try again in 1s " + - "[nodeId=" + ctx.localNodeId() + ", holder=" + content + - ", path=" + file.getAbsolutePath() + ']'); - } - - U.sleep(1000); - } - - if (content == null) - content = readContent(); - - failMsg = "Failed to acquire file lock [holder=" + content + ", time=" + (lockWaitTimeMillis / 1000) + - " sec, path=" + file.getAbsolutePath() + ']'; - } - catch (Exception e) { - throw new IgniteCheckedException(e); - } - - if (failMsg != null) - throw new IgniteCheckedException(failMsg); - } - - /** - * Write node id (who captured lock) into lock file. - * - * @param content Node id. - * @throws IOException if some fail while write node it. - */ - private void writeContent(String content) throws IOException { - FileChannel ch = lockFile.getChannel(); - - byte[] bytes = content.getBytes(); - - ByteBuffer buf = ByteBuffer.allocate(bytes.length); - buf.put(bytes); - - buf.flip(); - - ch.write(buf, 1); - - ch.force(false); + return sb.toString(); } - /** - * - */ - private String readContent() throws IOException { - FileChannel ch = lockFile.getChannel(); - - ByteBuffer buf = ByteBuffer.allocate((int)(ch.size() - 1)); - - ch.read(buf, 1); - - String content = new String(buf.array()); - - buf.clear(); - - return content; - } - - /** Locked or not. */ - public boolean isLocked() { - return lock != null && lock.isValid(); - } - - /** Releases file lock */ - public void release() { - U.releaseQuiet(lock); - } - - /** Closes file channel */ - @Override public void close() { - release(); - - U.closeQuiet(lockFile); - } - - /** - * @return Absolute path to lock file. - */ - private String lockPath() { - return file.getAbsolutePath(); + /** {@inheritDoc} */ + @Override protected String warningMessage(String lockId) { + return "Failed to acquire file lock. Will try again in 1s " + + "[nodeId=" + ctx.localNodeId() + ", holder=" + lockId + + ", path=" + lockPath() + ']'; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java index 951a2e17f4145..35f06ddee3de7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java @@ -35,7 +35,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.GridProcessorAdapter; -import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.SB; @@ -179,7 +179,7 @@ private PdsFolderSettings prepareNewSettings() throws IgniteCheckedException { // If such a folder exists, we start up with this ID (compatibility mode) final String subFolder = U.maskForFileName(consistentId.toString()); - final GridCacheDatabaseSharedManager.FileLockHolder oldStyleFolderLockHolder = tryLock(new File(pstStoreBasePath, subFolder)); + final NodeFileLockHolder oldStyleFolderLockHolder = tryLock(new File(pstStoreBasePath, subFolder)); if (oldStyleFolderLockHolder != null) return new PdsFolderSettings(pstStoreBasePath, @@ -199,7 +199,7 @@ private PdsFolderSettings prepareNewSettings() throws IgniteCheckedException { } for (FolderCandidate next : getNodeIndexSortedCandidates(pstStoreBasePath)) { - final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder = tryLock(next.subFolderFile()); + final NodeFileLockHolder fileLockHolder = tryLock(next.subFolderFile()); if (fileLockHolder != null) { if (log.isInfoEnabled()) @@ -214,7 +214,7 @@ private PdsFolderSettings prepareNewSettings() throws IgniteCheckedException { } // was not able to find free slot, allocating new - try (final GridCacheDatabaseSharedManager.FileLockHolder rootDirLock = lockRootDirectory(pstStoreBasePath)) { + try (final NodeFileLockHolder rootDirLock = lockRootDirectory(pstStoreBasePath)) { final List sortedCandidates = getNodeIndexSortedCandidates(pstStoreBasePath); final int nodeIdx = sortedCandidates.isEmpty() ? 0 : (sortedCandidates.get(sortedCandidates.size() - 1).nodeIndex() + 1); @@ -327,7 +327,7 @@ private static String padStart(String str, int minLength, char padChar) { final UUID uuid = UUID.randomUUID(); final String consIdBasedFolder = genNewStyleSubfolderName(nodeIdx, uuid); final File newRandomFolder = U.resolveWorkDirectory(pstStoreBasePath.getAbsolutePath(), consIdBasedFolder, false); //mkdir here - final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder = tryLock(newRandomFolder); + final NodeFileLockHolder fileLockHolder = tryLock(newRandomFolder); if (fileLockHolder != null) { if (log.isInfoEnabled()) @@ -362,10 +362,10 @@ private static String padStart(String str, int minLength, char padChar) { * @return locked directory, should be released and closed later * @throws IgniteCheckedException if failed */ - @NotNull private GridCacheDatabaseSharedManager.FileLockHolder lockRootDirectory(File pstStoreBasePath) + @NotNull private NodeFileLockHolder lockRootDirectory(File pstStoreBasePath) throws IgniteCheckedException { - GridCacheDatabaseSharedManager.FileLockHolder rootDirLock; + NodeFileLockHolder rootDirLock; int retry = 0; while ((rootDirLock = tryLock(pstStoreBasePath)) == null) { @@ -414,13 +414,13 @@ private static String padStart(String str, int minLength, char padChar) { * @return non null holder if lock was successful, null in case lock failed. If directory does not exist method will * always fail to lock. */ - private GridCacheDatabaseSharedManager.FileLockHolder tryLock(File dbStoreDirWithSubdirectory) { + private NodeFileLockHolder tryLock(File dbStoreDirWithSubdirectory) { if (!dbStoreDirWithSubdirectory.exists()) return null; final String path = dbStoreDirWithSubdirectory.getAbsolutePath(); - final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder - = new GridCacheDatabaseSharedManager.FileLockHolder(path, ctx, log); + final NodeFileLockHolder fileLockHolder + = new NodeFileLockHolder(path, ctx, log); try { fileLockHolder.tryLock(1000); @@ -500,7 +500,7 @@ private FolderCandidate parseFileName(@NotNull final File subFolderFile) { /** {@inheritDoc} */ @Override public void stop(boolean cancel) throws IgniteCheckedException { if (settings != null) { - final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder = settings.getLockedFileLockHolder(); + final NodeFileLockHolder fileLockHolder = settings.getLockedFileLockHolder(); if (fileLockHolder != null) fileLockHolder.close(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java index c47cbc9455621..72e0720891a14 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java @@ -19,7 +19,7 @@ import java.io.File; import java.io.Serializable; -import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.NotNull; @@ -48,7 +48,7 @@ public class PdsFolderSettings { * directory. This value is to be used at activate instead of locking.
May be null in case preconfigured * consistent ID is used or in case lock holder was already taken by other processor. */ - @Nullable private final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder; + @Nullable private final NodeFileLockHolder fileLockHolder; /** * Indicates if compatible mode is enabled, in that case all sub folders are generated from consistent ID without @@ -68,7 +68,7 @@ public class PdsFolderSettings { public PdsFolderSettings(@Nullable final File persistentStoreRootPath, final String folderName, final Serializable consistentId, - @Nullable final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder, + @Nullable final NodeFileLockHolder fileLockHolder, final boolean compatible) { this.consistentId = consistentId; @@ -125,7 +125,7 @@ public boolean isCompatible() { * * @return File lock holder with prelocked db directory. */ - @Nullable public GridCacheDatabaseSharedManager.FileLockHolder getLockedFileLockHolder() { + @Nullable public NodeFileLockHolder getLockedFileLockHolder() { return fileLockHolder; } From f590a18b1379923852b6761de6b4951acf50a6ab Mon Sep 17 00:00:00 2001 From: Nikolay Date: Wed, 31 Mar 2021 10:19:06 +0300 Subject: [PATCH 04/28] IGNITE-14353 Ability to specify postfix for IgniteLogger instead of nodeId (#8923) --- .../apache/ignite/internal/IgnitionEx.java | 24 +++++++++++++-- .../ignite/internal/util/IgniteUtils.java | 14 ++++----- .../ignite/logger/LoggerNodeIdAware.java | 3 ++ .../ignite/logger/LoggerPostfixAware.java | 30 +++++++++++++++++++ .../apache/ignite/logger/java/JavaLogger.java | 27 +++++++++++++---- .../logger/java/JavaLoggerFileHandler.java | 11 ++++++- .../ignite/logger/java/JavaLoggerTest.java | 8 +++-- .../logger/GridLog4jRollingFileAppender.java | 27 ++++++++++------- .../junits/logger/GridTestLog4jLogger.java | 23 ++++++++------ .../ignite/logger/log4j/Log4JLogger.java | 18 +++++++++-- .../logger/log4j/Log4jNodeIdFilePath.java | 15 +++++----- .../ignite/logger/log4j2/Log4J2Logger.java | 19 +++++++++--- .../logger/log4j2/Log4j2LoggerSelfTest.java | 18 ++++++++--- 13 files changed, 180 insertions(+), 57 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/logger/LoggerPostfixAware.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index 310426919c068..1f22c4b30529f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -110,6 +110,7 @@ import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerPostfixAware; import org.apache.ignite.logger.java.JavaLogger; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.MarshallerUtils; @@ -2535,8 +2536,21 @@ private void initializeDefaultSpi(IgniteConfiguration cfg) { * @return Initialized logger. * @throws IgniteCheckedException If failed. */ - @SuppressWarnings("ErrorNotRethrown") private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId, String workDir) + throws IgniteCheckedException { + return initLogger(cfgLog, nodeId, null, workDir); + } + + /** + * @param cfgLog Configured logger. + * @param nodeId Local node ID. + * @param postfix Log file postfix. + * @param workDir Work directory. + * @return Initialized logger. + * @throws IgniteCheckedException If failed. + */ + @SuppressWarnings("ErrorNotRethrown") + public IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId, String postfix, String workDir) throws IgniteCheckedException { try { Exception log4jInitErr = null; @@ -2600,8 +2614,12 @@ private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId, Stri ((JavaLogger)cfgLog).setWorkDirectory(workDir); // Set node IDs for all file appenders. - if (cfgLog instanceof LoggerNodeIdAware) - ((LoggerNodeIdAware)cfgLog).setNodeId(nodeId); + if (cfgLog instanceof LoggerNodeIdAware) { + if (nodeId == null && cfgLog instanceof LoggerPostfixAware) + ((LoggerPostfixAware)cfgLog).setPostfix(postfix); + else + ((LoggerNodeIdAware)cfgLog).setNodeId(nodeId); + } if (log4jInitErr != null) U.warn(cfgLog, "Failed to initialize Log4JLogger (falling back to standard java logging): " diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 61dba92cedca7..b0e24bb88f60d 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -8826,15 +8826,15 @@ public static void removeJavaNoOpLogger(Collection rmvHnds) { } /** - * Attaches node ID to log file name. + * Attaches postfix to log file name. * - * @param nodeId Node ID. + * @param postfix Postfix. * @param fileName File name. - * @return File name with node ID. + * @return File name with postfix. */ @SuppressWarnings("IfMayBeConditional") - public static String nodeIdLogFileName(UUID nodeId, String fileName) { - assert nodeId != null; + public static String logFileName(String postfix, String fileName) { + assert postfix != null; assert fileName != null; fileName = GridFilenameUtils.separatorsToSystem(fileName); @@ -8842,9 +8842,9 @@ public static String nodeIdLogFileName(UUID nodeId, String fileName) { int dot = fileName.lastIndexOf('.'); if (dot < 0 || dot == fileName.length() - 1) - return fileName + '-' + U.id8(nodeId); + return fileName + '-' + postfix; else - return fileName.substring(0, dot) + '-' + U.id8(nodeId) + fileName.substring(dot); + return fileName.substring(0, dot) + '-' + postfix + fileName.substring(dot); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java b/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java index 492d93c507ce2..16c5125ead523 100644 --- a/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java +++ b/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java @@ -21,7 +21,10 @@ /** * Interface for Ignite file appenders to attach node ID to log file names. + * + * @deprecated Use {@link LoggerPostfixAware} instead. */ +@Deprecated public interface LoggerNodeIdAware { /** * Sets node ID. diff --git a/modules/core/src/main/java/org/apache/ignite/logger/LoggerPostfixAware.java b/modules/core/src/main/java/org/apache/ignite/logger/LoggerPostfixAware.java new file mode 100644 index 0000000000000..cb86621bddb00 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/logger/LoggerPostfixAware.java @@ -0,0 +1,30 @@ +/* + * 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.logger; + +/** + * Interface for Ignite file appenders to attach postfix to log file names. + */ +public interface LoggerPostfixAware extends LoggerNodeIdAware { + /** + * Sets postfix. + * + * @param postfix Postfix. + */ + public void setPostfix(String postfix); +} diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java index c82f01d63b0cb..89dc70799b24d 100644 --- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java +++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java @@ -34,7 +34,7 @@ import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerPostfixAware; import org.jetbrains.annotations.Nullable; import static java.util.logging.Level.FINE; @@ -95,7 +95,7 @@ * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger * injection. */ -public class JavaLogger implements IgniteLogger, LoggerNodeIdAware { +public class JavaLogger implements IgniteLogger, LoggerPostfixAware { /** */ public static final String DFLT_CONFIG_PATH = "config/java.util.logging.properties"; @@ -128,6 +128,10 @@ public class JavaLogger implements IgniteLogger, LoggerNodeIdAware { @GridToStringExclude private volatile UUID nodeId; + /** Postfix. */ + @GridToStringExclude + private volatile String postfix; + /** * Creates new logger. */ @@ -368,14 +372,27 @@ public void setWorkDirectory(String workDir) { @Override public void setNodeId(UUID nodeId) { A.notNull(nodeId, "nodeId"); - if (this.nodeId != null) + postfix(nodeId, U.id8(nodeId)); + } + + /** {@inheritDoc} */ + @Override public void setPostfix(String postfix) { + A.notNull(postfix, "postfix"); + + postfix(null, postfix); + } + + /** */ + private void postfix(UUID nodeId, String postfix) { + if (this.postfix != null) return; synchronized (mux) { // Double check. - if (this.nodeId != null) + if (this.postfix != null) return; + this.postfix = postfix; this.nodeId = nodeId; } @@ -385,7 +402,7 @@ public void setWorkDirectory(String workDir) { return; try { - fileHnd.nodeId(nodeId, workDir); + fileHnd.postfix(postfix, workDir); } catch (IgniteCheckedException | IOException e) { throw new RuntimeException("Failed to enable file handler.", e); diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java index 6320589c4793d..e729441aaae2a 100644 --- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java @@ -80,6 +80,15 @@ public final class JavaLoggerFileHandler extends StreamHandler { * @param nodeId Node id. */ public void nodeId(UUID nodeId, String workDir) throws IgniteCheckedException, IOException { + postfix(U.id8(nodeId), workDir); + } + + /** + * Sets Postfix and instantiates {@link FileHandler} delegate. + * + * @param postfix Postfix. + */ + public void postfix(String postfix, String workDir) throws IgniteCheckedException, IOException { if (delegate != null) return; @@ -90,7 +99,7 @@ public void nodeId(UUID nodeId, String workDir) throws IgniteCheckedException, I if (ptrn == null) ptrn = "ignite-%{id8}.%g.log"; - ptrn = new File(logDirectory(workDir), ptrn.replace("%{id8}", U.id8(nodeId))).getAbsolutePath(); + ptrn = new File(logDirectory(workDir), ptrn.replace("%{id8}", postfix)).getAbsolutePath(); int limit = getIntProperty(clsName + ".limit", 0); diff --git a/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java b/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java index 4687ca992c9c2..dba26500c62bc 100644 --- a/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java @@ -20,10 +20,11 @@ import java.util.UUID; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerPostfixAware; import org.apache.ignite.testframework.junits.common.GridCommonTest; import org.junit.Test; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; /** @@ -43,7 +44,10 @@ public void testLogInitialize() throws Exception { log = new JavaLogger(); ((JavaLogger)log).setWorkDirectory(U.defaultWorkDirectory()); - ((LoggerNodeIdAware)log).setNodeId(UUID.fromString("00000000-1111-2222-3333-444444444444")); + UUID id = UUID.fromString("00000000-1111-2222-3333-444444444444"); + + ((LoggerPostfixAware)log).setNodeId(id); + assertEquals(id, ((LoggerPostfixAware)log).getNodeId()); System.out.println(log.toString()); diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java index fc9f38f0a5e7d..5d73e7c09f441 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java @@ -23,16 +23,16 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerPostfixAware; import org.apache.log4j.Layout; import org.apache.log4j.RollingFileAppender; /** * Log4J {@link org.apache.log4j.RollingFileAppender} with added support for grid node IDs. */ -public class GridLog4jRollingFileAppender extends RollingFileAppender implements LoggerNodeIdAware { - /** Node ID. */ - private UUID nodeId; +public class GridLog4jRollingFileAppender extends RollingFileAppender implements LoggerPostfixAware { + /** Postfix. */ + private String postfix; /** Basic log file name. */ private String baseFileName; @@ -78,18 +78,23 @@ private void init() { GridTestLog4jLogger.addAppender(this); } + /** {@inheritDoc} */ + @Override public void setNodeId(UUID nodeId) { + setPostfix(U.id8(nodeId)); + } + /** {@inheritDoc} */ @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext") - @Override public synchronized void setNodeId(UUID nodeId) { - A.notNull(nodeId, "nodeId"); + @Override public synchronized void setPostfix(String postfix) { + A.notNull(postfix, "postfix"); - this.nodeId = nodeId; + this.postfix = postfix; if (fileName != null) { // fileName could be null if IGNITE_HOME is not defined. if (baseFileName == null) baseFileName = fileName; - fileName = U.nodeIdLogFileName(nodeId, baseFileName); + fileName = U.logFileName(postfix, baseFileName); } else { String tmpDir = IgniteSystemProperties.getString("java.io.tmpdir"); @@ -97,20 +102,20 @@ private void init() { if (tmpDir != null) { baseFileName = new File(tmpDir, "ignite.log").getAbsolutePath(); - fileName = U.nodeIdLogFileName(nodeId, baseFileName); + fileName = U.logFileName(postfix, baseFileName); } } } /** {@inheritDoc} */ @Override public synchronized UUID getNodeId() { - return nodeId; + throw new UnsupportedOperationException("getNodeId"); } /** {@inheritDoc} */ @Override public synchronized void setFile(String fileName, boolean fileAppend, boolean bufIO, int bufSize) throws IOException { - if (nodeId != null) + if (postfix != null) super.setFile(fileName, fileAppend, bufIO, bufSize); } } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java index 4464828715c5e..65d16541a3cf8 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java @@ -33,7 +33,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerPostfixAware; import org.apache.log4j.Appender; import org.apache.log4j.Category; import org.apache.log4j.ConsoleAppender; @@ -77,7 +77,7 @@ * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger * injection. */ -public class GridTestLog4jLogger implements IgniteLogger, LoggerNodeIdAware { +public class GridTestLog4jLogger implements IgniteLogger, LoggerPostfixAware { /** Appenders. */ private static Collection fileAppenders = new GridConcurrentHashSet<>(); @@ -102,9 +102,9 @@ public class GridTestLog4jLogger implements IgniteLogger, LoggerNodeIdAware { /** Quiet flag. */ private final boolean quiet; - /** Node ID. */ + /** Postfix. */ @GridToStringExclude - private UUID nodeId; + private String postfix; /** * Creates new logger and automatically detects if root logger already @@ -407,13 +407,18 @@ public static void removeAppender(FileAppender a) { /** {@inheritDoc} */ @Override public void setNodeId(UUID nodeId) { - A.notNull(nodeId, "nodeId"); + setPostfix(U.id8(nodeId)); + } + + /** {@inheritDoc} */ + @Override public void setPostfix(String postfix) { + A.notNull(postfix, "postfix"); - this.nodeId = nodeId; + this.postfix = postfix; for (FileAppender a : fileAppenders) { - if (a instanceof LoggerNodeIdAware) { - ((LoggerNodeIdAware)a).setNodeId(nodeId); + if (a instanceof LoggerPostfixAware) { + ((LoggerPostfixAware)a).setPostfix(postfix); a.activateOptions(); } @@ -422,7 +427,7 @@ public static void removeAppender(FileAppender a) { /** {@inheritDoc} */ @Override public UUID getNodeId() { - return nodeId; + throw new UnsupportedOperationException("getNodeId"); } /** diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java index ec0a5b3c79c24..fbcfc0847b524 100644 --- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java +++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java @@ -33,7 +33,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerPostfixAware; import org.apache.log4j.Appender; import org.apache.log4j.Category; import org.apache.log4j.ConsoleAppender; @@ -79,7 +79,7 @@ * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger * injection. */ -public class Log4JLogger implements IgniteLogger, LoggerNodeIdAware, Log4jFileAware { +public class Log4JLogger implements IgniteLogger, LoggerPostfixAware, Log4jFileAware { /** Appenders. */ private static Collection fileAppenders = new GridConcurrentHashSet<>(); @@ -503,9 +503,14 @@ public static void removeAppender(FileAppender a) { @Override public void setNodeId(UUID nodeId) { A.notNull(nodeId, "nodeId"); + postfix(nodeId, U.id8(nodeId)); + } + + /** */ + private void postfix(UUID nodeId, String postfix) { this.nodeId = nodeId; - updateFilePath(new Log4jNodeIdFilePath(nodeId)); + updateFilePath(new Log4jNodeIdFilePath(postfix)); } /** {@inheritDoc} */ @@ -513,6 +518,13 @@ public static void removeAppender(FileAppender a) { return nodeId; } + /** {@inheritDoc} */ + @Override public void setPostfix(String postfix) { + A.notNull(postfix, "postfix"); + + postfix(nodeId, postfix); + } + /** * Gets files for all registered file appenders. * diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java index d4d1892502750..813f1e858a893 100644 --- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java +++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java @@ -18,7 +18,6 @@ package org.apache.ignite.logger.log4j; import java.io.File; -import java.util.UUID; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -32,29 +31,29 @@ class Log4jNodeIdFilePath implements IgniteClosure { private static final long serialVersionUID = 0L; /** Node id. */ - private final UUID nodeId; + private final String postfix; /** * Creates new instance. * - * @param id Node id. + * @param postfix Postfix. */ - Log4jNodeIdFilePath(UUID id) { - nodeId = id; + Log4jNodeIdFilePath(String postfix) { + this.postfix = postfix; } /** {@inheritDoc} */ @Override public String apply(String oldPath) { if (!F.isEmpty(U.IGNITE_LOG_DIR)) - return U.nodeIdLogFileName(nodeId, new File(U.IGNITE_LOG_DIR, "ignite.log").getAbsolutePath()); + return U.logFileName(postfix, new File(U.IGNITE_LOG_DIR, "ignite.log").getAbsolutePath()); if (oldPath != null) // fileName could be null if IGNITE_HOME is not defined. - return U.nodeIdLogFileName(nodeId, oldPath); + return U.logFileName(postfix, oldPath); String tmpDir = IgniteSystemProperties.getString("java.io.tmpdir"); if (tmpDir != null) - return U.nodeIdLogFileName(nodeId, new File(tmpDir, "ignite.log").getAbsolutePath()); + return U.logFileName(postfix, new File(tmpDir, "ignite.log").getAbsolutePath()); System.err.println("Failed to get tmp directory for log file."); diff --git a/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java b/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java index 0f5c313a3f81c..665be453fbc69 100644 --- a/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java +++ b/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java @@ -31,7 +31,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerPostfixAware; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Marker; @@ -81,7 +81,7 @@ * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger * injection. */ -public class Log4J2Logger implements IgniteLogger, LoggerNodeIdAware { +public class Log4J2Logger implements IgniteLogger, LoggerPostfixAware { /** */ private static final String NODE_ID = "nodeId"; @@ -387,10 +387,14 @@ public Logger createConsoleLogger() { @Override public void setNodeId(UUID nodeId) { A.notNull(nodeId, "nodeId"); + postfix(nodeId, U.id8(nodeId)); + } + + private void postfix(UUID nodeId, String postfix) { this.nodeId = nodeId; - // Set nodeId as system variable to be used at configuration. - System.setProperty(NODE_ID, U.id8(nodeId)); + // Set postfix as system variable to be used at configuration. + System.setProperty(NODE_ID, postfix); if (inited) { final LoggerContext ctx = impl.getContext(); @@ -415,6 +419,13 @@ public Logger createConsoleLogger() { return nodeId; } + /** {@inheritDoc} */ + @Override public void setPostfix(String postfix) { + A.notNull(postfix, "nodeId"); + + postfix(null, postfix); + } + /** * Gets {@link IgniteLogger} wrapper around log4j logger for the given * category. If category is {@code null}, then root logger is returned. If diff --git a/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java b/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java index 3ebd15d656ca7..fcc40b8b36724 100644 --- a/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java +++ b/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java @@ -26,7 +26,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerPostfixAware; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.GridTestUtils; @@ -70,7 +70,9 @@ public void testFileConstructor() throws Exception { assertTrue(log.toString().contains("Log4J2Logger")); assertTrue(log.toString().contains(xml.getPath())); - ((LoggerNodeIdAware)log).setNodeId(UUID.randomUUID()); + UUID id = UUID.randomUUID(); + ((LoggerPostfixAware)log).setNodeId(id); + assertEquals(id, ((LoggerPostfixAware)log).getNodeId()); checkLog(log); } @@ -93,7 +95,9 @@ public void testUrlConstructor() throws Exception { assertTrue(log.toString().contains("Log4J2Logger")); assertTrue(log.toString().contains(url.getPath())); - ((LoggerNodeIdAware)log).setNodeId(UUID.randomUUID()); + UUID id = UUID.randomUUID(); + ((LoggerPostfixAware)log).setNodeId(id); + assertEquals(id, ((LoggerPostfixAware)log).getNodeId()); checkLog(log); } @@ -110,7 +114,9 @@ public void testPathConstructor() throws Exception { assertTrue(log.toString().contains("Log4J2Logger")); assertTrue(log.toString().contains(LOG_PATH_TEST)); - ((LoggerNodeIdAware)log).setNodeId(UUID.randomUUID()); + UUID id = UUID.randomUUID(); + ((LoggerPostfixAware)log).setNodeId(id); + assertEquals(id, ((LoggerPostfixAware)log).getNodeId()); checkLog(log); } @@ -140,6 +146,10 @@ public void testSystemNodeId() throws Exception { new Log4J2Logger(LOG_PATH_TEST).setNodeId(id); assertEquals(U.id8(id), System.getProperty("nodeId")); + + new Log4J2Logger(LOG_PATH_TEST).setPostfix("myapp"); + + assertEquals("myapp", System.getProperty("nodeId")); } /** From ab7bf2b7848b2e431aaa092c640ea337cc0fdbc6 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Wed, 31 Mar 2021 12:25:12 +0300 Subject: [PATCH 05/28] IGNITE-14435 Refactor PdsConsistentIdProcessor for reusage (#8946) --- .../filename/PdsConsistentIdProcessor.java | 453 +-------------- .../filename/PdsFolderResolver.java | 531 ++++++++++++++++++ .../filename/PdsFolderSettings.java | 10 +- .../filename/PdsFoldersResolver.java | 3 +- .../snapshot/IgniteSnapshotManager.java | 2 +- .../IgniteUidAsConsistentIdMigrationTest.java | 14 +- .../db/wal/reader/IgniteWalReaderTest.java | 2 +- .../db/wal/IgniteWalRecoveryTest.java | 4 +- 8 files changed, 565 insertions(+), 454 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java index 35f06ddee3de7..56fe06b62ba4d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java @@ -18,89 +18,32 @@ package org.apache.ignite.internal.processors.cache.persistence.filename; import java.io.File; -import java.io.FileFilter; -import java.io.IOException; import java.io.Serializable; -import java.text.SimpleDateFormat; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.UUID; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder; -import org.apache.ignite.internal.util.typedef.internal.A; -import org.apache.ignite.internal.util.typedef.internal.CU; -import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID; -import static org.apache.ignite.IgniteSystemProperties.getBoolean; - /** * Component for resolving PDS storage file names, also used for generating consistent ID for case PDS mode is enabled */ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements PdsFoldersResolver { - /** Database subfolders constant prefix. */ - private static final String DB_FOLDER_PREFIX = "node"; - - /** Node index and uid separator in subfolders name. */ - private static final String NODEIDX_UID_SEPARATOR = "-"; - - /** Constant node subfolder prefix and node index pattern (nodeII, where II - node index as decimal integer) */ - private static final String NODE_PATTERN = DB_FOLDER_PREFIX + "[0-9]*" + NODEIDX_UID_SEPARATOR; - - /** Uuid as string pattern. */ - private static final String UUID_STR_PATTERN = "[a-fA-F0-9]{8}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{12}"; - - /** - * Subdir (nodeII-UID, where II - node index as decimal integer, UID - string representation of consistent ID) - * pattern. - */ - private static final String SUBDIR_PATTERN = NODE_PATTERN + UUID_STR_PATTERN; - - /** Database subfolders for new style filter. */ - public static final FileFilter DB_SUBFOLDERS_NEW_STYLE_FILTER = new FileFilter() { - @Override public boolean accept(File pathname) { - return pathname.isDirectory() && pathname.getName().matches(SUBDIR_PATTERN); - } - }; - - /** Database subfolders for old style filter. */ - private static final FileFilter DB_SUBFOLDERS_OLD_STYLE_FILTER = new FileFilter() { - @Override public boolean accept(File pathname) { - String path = pathname.toString(); - return pathname.isDirectory() - && !"wal".equals(pathname.getName()) - && !path.contains(DataStorageConfiguration.DFLT_BINARY_METADATA_PATH) - && !path.contains(DataStorageConfiguration.DFLT_MARSHALLER_PATH) - && !pathname.getName().matches(SUBDIR_PATTERN); - } - }; - - /** Database default folder. */ - public static final String DB_DEFAULT_FOLDER = "db"; - /** Config. */ - private IgniteConfiguration cfg; + private final IgniteConfiguration cfg; /** Logger. */ - private IgniteLogger log; + private final IgniteLogger log; /** Context. */ - private GridKernalContext ctx; + private final GridKernalContext ctx; /** Cached folder settings. */ - private PdsFolderSettings settings; + private PdsFolderSettings settings; /** * Creates folders resolver @@ -122,22 +65,29 @@ public PdsConsistentIdProcessor(final GridKernalContext ctx) { * @param consistentId compatibility consistent ID * @return PDS folder settings compatible with previous versions. */ - private PdsFolderSettings compatibleResolve( + private PdsFolderSettings compatibleResolve( @Nullable final File pstStoreBasePath, @NotNull final Serializable consistentId) { if (cfg.getConsistentId() != null) { // compatible mode from configuration is used fot this case, no locking, no consitent id change - return new PdsFolderSettings(pstStoreBasePath, cfg.getConsistentId()); + return new PdsFolderSettings<>(pstStoreBasePath, cfg.getConsistentId()); } - return new PdsFolderSettings(pstStoreBasePath, consistentId); + return new PdsFolderSettings<>(pstStoreBasePath, consistentId); } /** {@inheritDoc} */ - @Override public PdsFolderSettings resolveFolders() throws IgniteCheckedException { + @Override public PdsFolderSettings resolveFolders() throws IgniteCheckedException { if (settings == null) { - settings = prepareNewSettings(); + //here deprecated method is used to get compatible version of consistentId + PdsFolderResolver resolver = + new PdsFolderResolver<>(cfg, log, ctx.discovery().consistentId(), this::tryLock); + + settings = resolver.resolve(); + + if (settings == null) + settings = resolver.generateNew(); if (!settings.isCompatible()) { if (log.isInfoEnabled()) @@ -150,262 +100,6 @@ private PdsFolderSettings compatibleResolve( return settings; } - /** - * Creates new settings when we don't have cached one. - * - * @return new settings with prelocked directory (if appropriate). - * @throws IgniteCheckedException if IO failed. - */ - private PdsFolderSettings prepareNewSettings() throws IgniteCheckedException { - final File pstStoreBasePath = resolvePersistentStoreBasePath(); - //here deprecated method is used to get compatible version of consistentId - final Serializable consistentId = ctx.discovery().consistentId(); - - if (!CU.isPersistenceEnabled(cfg)) - return compatibleResolve(pstStoreBasePath, consistentId); - - if (ctx.clientNode()) - return new PdsFolderSettings(pstStoreBasePath, UUID.randomUUID()); - - if (getBoolean(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, false)) - return compatibleResolve(pstStoreBasePath, consistentId); - - // compatible mode from configuration is used fot this case - if (cfg.getConsistentId() != null) { - // compatible mode from configuration is used fot this case, no locking, no consistent id change - return new PdsFolderSettings(pstStoreBasePath, cfg.getConsistentId()); - } - // The node scans the work directory and checks if there is a folder matching the consistent ID. - // If such a folder exists, we start up with this ID (compatibility mode) - final String subFolder = U.maskForFileName(consistentId.toString()); - - final NodeFileLockHolder oldStyleFolderLockHolder = tryLock(new File(pstStoreBasePath, subFolder)); - - if (oldStyleFolderLockHolder != null) - return new PdsFolderSettings(pstStoreBasePath, - subFolder, - consistentId, - oldStyleFolderLockHolder, - true); - - final File[] oldStyleFolders = pstStoreBasePath.listFiles(DB_SUBFOLDERS_OLD_STYLE_FILTER); - - if (oldStyleFolders != null && oldStyleFolders.length != 0) { - for (File folder : oldStyleFolders) { - final String path = getPathDisplayableInfo(folder); - - U.warn(log, "There is other non-empty storage folder under storage base directory [" + path + "]"); - } - } - - for (FolderCandidate next : getNodeIndexSortedCandidates(pstStoreBasePath)) { - final NodeFileLockHolder fileLockHolder = tryLock(next.subFolderFile()); - - if (fileLockHolder != null) { - if (log.isInfoEnabled()) - log.info("Successfully locked persistence storage folder [" + next.subFolderFile() + "]"); - - return new PdsFolderSettings(pstStoreBasePath, - next.subFolderFile().getName(), - next.uuid(), - fileLockHolder, - false); - } - } - - // was not able to find free slot, allocating new - try (final NodeFileLockHolder rootDirLock = lockRootDirectory(pstStoreBasePath)) { - final List sortedCandidates = getNodeIndexSortedCandidates(pstStoreBasePath); - final int nodeIdx = sortedCandidates.isEmpty() ? 0 : (sortedCandidates.get(sortedCandidates.size() - 1).nodeIndex() + 1); - - return generateAndLockNewDbStorage(pstStoreBasePath, nodeIdx); - } - } - - /** - * Calculate overall folder size. - * - * @param dir directory to scan. - * @return total size in bytes. - */ - private static FolderParams folderSize(File dir) { - final FolderParams params = new FolderParams(); - - visitFolder(dir, params); - - return params; - } - - /** - * Scans provided directory and its sub dirs, collects found metrics. - * - * @param dir directory to start scan from. - * @param params input/output. - */ - private static void visitFolder(final File dir, final FolderParams params) { - for (File file : dir.listFiles()) { - if (file.isDirectory()) - visitFolder(file, params); - else { - params.size += file.length(); - params.lastModified = Math.max(params.lastModified, dir.lastModified()); - } - } - } - - /** - * @param folder folder to scan. - * @return folder displayable information. - */ - @NotNull private String getPathDisplayableInfo(final File folder) { - final SB res = new SB(); - - res.a(getCanonicalPath(folder)); - res.a(", "); - final FolderParams params = folderSize(folder); - - res.a(params.size); - res.a(" bytes, modified "); - final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("MM/dd/yyyy hh:mm a"); - - res.a(simpleDateFormat.format(params.lastModified)); - res.a(" "); - - return res.toString(); - } - - /** - * Returns the canonical pathname string of this abstract pathname. - * - * @param file path to convert. - * @return canonical pathname or at leas absolute if convert to canonical failed. - */ - @NotNull private String getCanonicalPath(final File file) { - try { - return file.getCanonicalPath(); - } - catch (IOException ignored) { - return file.getAbsolutePath(); - } - } - - /** - * Pad start of string with provided character. - * - * @param str sting to pad. - * @param minLength expected length. - * @param padChar padding character. - * @return padded string. - */ - private static String padStart(String str, int minLength, char padChar) { - A.notNull(str, "String should not be empty"); - if (str.length() >= minLength) - return str; - - final SB sb = new SB(minLength); - - for (int i = str.length(); i < minLength; ++i) - sb.a(padChar); - - sb.a(str); - - return sb.toString(); - - } - - /** - * Creates new DB storage folder. - * - * @param pstStoreBasePath DB root path. - * @param nodeIdx next node index to use in folder name. - * @return new settings to be used in this node. - * @throws IgniteCheckedException if failed. - */ - @NotNull private PdsFolderSettings generateAndLockNewDbStorage(final File pstStoreBasePath, - final int nodeIdx) throws IgniteCheckedException { - - final UUID uuid = UUID.randomUUID(); - final String consIdBasedFolder = genNewStyleSubfolderName(nodeIdx, uuid); - final File newRandomFolder = U.resolveWorkDirectory(pstStoreBasePath.getAbsolutePath(), consIdBasedFolder, false); //mkdir here - final NodeFileLockHolder fileLockHolder = tryLock(newRandomFolder); - - if (fileLockHolder != null) { - if (log.isInfoEnabled()) - log.info("Successfully created new persistent storage folder [" + newRandomFolder + "]"); - - return new PdsFolderSettings(pstStoreBasePath, consIdBasedFolder, uuid, fileLockHolder, false); - } - throw new IgniteCheckedException("Unable to lock file generated randomly [" + newRandomFolder + "]"); - } - - /** - * Generates DB subfolder name for provided node index (local) and UUID (consistent ID) - * - * @param nodeIdx node index. - * @param uuid consistent ID. - * @return folder file name - */ - @NotNull public static String genNewStyleSubfolderName(final int nodeIdx, final UUID uuid) { - final String uuidAsStr = uuid.toString(); - - assert uuidAsStr.matches(UUID_STR_PATTERN); - - final String nodeIdxPadded = padStart(Integer.toString(nodeIdx), 2, '0'); - - return DB_FOLDER_PREFIX + nodeIdxPadded + NODEIDX_UID_SEPARATOR + uuidAsStr; - } - - /** - * Acquires lock to root storage directory, used to lock root directory in case creating new files is required. - * - * @param pstStoreBasePath rood DB dir to lock - * @return locked directory, should be released and closed later - * @throws IgniteCheckedException if failed - */ - @NotNull private NodeFileLockHolder lockRootDirectory(File pstStoreBasePath) - throws IgniteCheckedException { - - NodeFileLockHolder rootDirLock; - int retry = 0; - - while ((rootDirLock = tryLock(pstStoreBasePath)) == null) { - if (retry > 600) - throw new IgniteCheckedException("Unable to start under DB storage path [" + pstStoreBasePath + "]" + - ". Lock is being held to root directory"); - retry++; - } - - return rootDirLock; - } - - /** - * @param pstStoreBasePath root storage folder to scan. - * @return empty list if there is no files in folder to test. Non null value is returned for folder having - * applicable new style files. Collection is sorted ascending according to node ID, 0 node index is coming first. - */ - @Nullable private List getNodeIndexSortedCandidates(File pstStoreBasePath) { - final File[] files = pstStoreBasePath.listFiles(DB_SUBFOLDERS_NEW_STYLE_FILTER); - - if (files == null) - return Collections.emptyList(); - - final List res = new ArrayList<>(); - - for (File file : files) { - final FolderCandidate candidate = parseFileName(file); - - if (candidate != null) - res.add(candidate); - } - Collections.sort(res, new Comparator() { - @Override public int compare(FolderCandidate c1, FolderCandidate c2) { - return Integer.compare(c1.nodeIndex(), c2.nodeIndex()); - } - }); - - return res; - } - /** * Tries to lock subfolder within storage root folder. * @@ -437,66 +131,6 @@ private NodeFileLockHolder tryLock(File dbStoreDirWithSubdirectory) { } } - /** - * @return DB storage absolute root path resolved as 'db' folder in Ignite work dir (by default) or using persistent - * store configuration. Null if persistence is not enabled. Returned folder is created automatically. - * @throws IgniteCheckedException if I/O failed. - */ - @Nullable private File resolvePersistentStoreBasePath() throws IgniteCheckedException { - final DataStorageConfiguration dsCfg = cfg.getDataStorageConfiguration(); - - if (dsCfg == null) - return null; - - final String pstPath = dsCfg.getStoragePath(); - - return U.resolveWorkDirectory( - cfg.getWorkDirectory(), - pstPath != null ? pstPath : DB_DEFAULT_FOLDER, - false - ); - - } - - /** - * @param subFolderFile new style folder name to parse - * @return Pair of UUID and node index - */ - private FolderCandidate parseFileName(@NotNull final File subFolderFile) { - return parseSubFolderName(subFolderFile, log); - } - - /** - * @param subFolderFile new style file to parse. - * @param log Logger. - * @return Pair of UUID and node index. - */ - @Nullable public static FolderCandidate parseSubFolderName( - @NotNull final File subFolderFile, - @NotNull final IgniteLogger log) { - - final String fileName = subFolderFile.getName(); - final Matcher matcher = Pattern.compile(NODE_PATTERN).matcher(fileName); - if (!matcher.find()) - return null; - - int uidStart = matcher.end(); - - try { - final String uid = fileName.substring(uidStart); - final UUID uuid = UUID.fromString(uid); - final String substring = fileName.substring(DB_FOLDER_PREFIX.length(), uidStart - NODEIDX_UID_SEPARATOR.length()); - final int idx = Integer.parseInt(substring); - - return new FolderCandidate(subFolderFile, idx, uuid); - } - catch (Exception e) { - U.warn(log, "Unable to parse new style file format from [" + subFolderFile.getAbsolutePath() + "]: " + e); - - return null; - } - } - /** {@inheritDoc} */ @Override public void stop(boolean cancel) throws IgniteCheckedException { if (settings != null) { @@ -508,61 +142,6 @@ private FolderCandidate parseFileName(@NotNull final File subFolderFile) { super.stop(cancel); } - - /** Path metrics */ - private static class FolderParams { - /** Overall size in bytes. */ - private long size; - - /** Last modified. */ - private long lastModified; - } - - /** - * Represents parsed new style file and encoded parameters in this file name - */ - public static class FolderCandidate { - /** Absolute file path pointing to DB subfolder within DB storage root folder. */ - private final File subFolderFile; - - /** Node index (local, usually 0 if multiple nodes are not started at local PC). */ - private final int nodeIdx; - - /** Uuid contained in file name, is to be set as consistent ID. */ - private final UUID uuid; - - /** - * @param subFolderFile Absolute file path pointing to DB subfolder. - * @param nodeIdx Node index. - * @param uuid Uuid. - */ - public FolderCandidate(File subFolderFile, int nodeIdx, UUID uuid) { - this.subFolderFile = subFolderFile; - this.nodeIdx = nodeIdx; - this.uuid = uuid; - } - - /** - * @return Node index (local, usually 0 if multiple nodes are not started at local PC). - */ - public int nodeIndex() { - return nodeIdx; - } - - /** - * @return Uuid contained in file name, is to be set as consistent ID. - */ - public Serializable uuid() { - return uuid; - } - - /** - * @return Absolute file path pointing to DB subfolder within DB storage root folder. - */ - public File subFolderFile() { - return subFolderFile; - } - } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java new file mode 100644 index 0000000000000..225a973bad247 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java @@ -0,0 +1,531 @@ +/* + * 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.persistence.filename; + +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; +import java.io.Serializable; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.UUID; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.processors.cache.persistence.FileLockHolder; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.SB; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID; +import static org.apache.ignite.IgniteSystemProperties.getBoolean; + +/** + * This class contains logic to resolve and possibly lock PDS folder + * based on provided {@link IgniteConfiguration} and {@link #consistentId}. + * + * @param Type of the lock holder. + */ +public class PdsFolderResolver { + /** Database subfolders constant prefix. */ + private static final String DB_FOLDER_PREFIX = "node"; + + /** Node index and uid separator in subfolders name. */ + private static final String NODEIDX_UID_SEPARATOR = "-"; + + /** Constant node subfolder prefix and node index pattern (nodeII, where II - node index as decimal integer) */ + private static final String NODE_PATTERN = DB_FOLDER_PREFIX + "[0-9]*" + NODEIDX_UID_SEPARATOR; + + /** Uuid as string pattern. */ + private static final String UUID_STR_PATTERN = "[a-fA-F0-9]{8}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{12}"; + + /** + * Subdir (nodeII-UID, where II - node index as decimal integer, UID - string representation of consistent ID) + * pattern. + */ + private static final String SUBDIR_PATTERN = NODE_PATTERN + UUID_STR_PATTERN; + + /** Database subfolders for new style filter. */ + public static final FileFilter DB_SUBFOLDERS_NEW_STYLE_FILTER = new FileFilter() { + @Override public boolean accept(File pathname) { + return pathname.isDirectory() && pathname.getName().matches(SUBDIR_PATTERN); + } + }; + + /** Database subfolders for old style filter. */ + private static final FileFilter DB_SUBFOLDERS_OLD_STYLE_FILTER = new FileFilter() { + @Override public boolean accept(File pathname) { + String path = pathname.toString(); + return pathname.isDirectory() + && !"wal".equals(pathname.getName()) + && !path.contains(DataStorageConfiguration.DFLT_BINARY_METADATA_PATH) + && !path.contains(DataStorageConfiguration.DFLT_MARSHALLER_PATH) + && !pathname.getName().matches(SUBDIR_PATTERN); + } + }; + + /** Database default folder. */ + public static final String DB_DEFAULT_FOLDER = "db"; + + /** */ + private final IgniteConfiguration cfg; + + /** */ + private final IgniteLogger log; + + /** */ + private final @Nullable Serializable consistentId; + + /** */ + private final Function tryLock; + + /** + * @param cfg Ignite configuration. + * @param log Logger. + * @param consistentId Constent id. + * @param tryLock Lock function. + */ + public PdsFolderResolver( + IgniteConfiguration cfg, + IgniteLogger log, + @Nullable Serializable consistentId, + Function tryLock + ) { + this.cfg = cfg; + this.log = log; + this.consistentId = consistentId; + this.tryLock = tryLock; + } + + /** + * Prepares compatible PDS folder settings. No locking is performed, consistent ID is not overridden. + * + * @param pstStoreBasePath DB storage base path or null if persistence is not enabled. + * @param consistentId compatibility consistent ID + * @return PDS folder settings compatible with previous versions. + */ + private PdsFolderSettings compatibleResolve( + @Nullable final File pstStoreBasePath, + @Nullable final Serializable consistentId) { + + if (cfg.getConsistentId() != null) { + // compatible mode from configuration is used fot this case, no locking, no consitent id change + return new PdsFolderSettings<>(pstStoreBasePath, cfg.getConsistentId()); + } + + if (consistentId == null) + return new PdsFolderSettings<>(pstStoreBasePath, consistentId); + + return null; + } + + /** + * Resolves {@link PdsFolderSettings} according to specified {@link IgniteConfiguration}, {@link #consistentId}. + * + * @return new settings with prelocked directory (if appropriate) or null. + * @throws IgniteCheckedException if IO failed. + */ + public PdsFolderSettings resolve() throws IgniteCheckedException { + final File pstStoreBasePath = resolvePersistentStoreBasePath(); + + if (!CU.isPersistenceEnabled(cfg)) + return compatibleResolve(pstStoreBasePath, consistentId); + + if (cfg.isClientMode() || cfg.isDaemon()) + return new PdsFolderSettings<>(pstStoreBasePath, UUID.randomUUID()); + + if (getBoolean(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, false)) + return compatibleResolve(pstStoreBasePath, consistentId); + + // compatible mode from configuration is used fot this case + if (cfg.getConsistentId() != null) { + // compatible mode from configuration is used fot this case, no locking, no consistent id change + return new PdsFolderSettings<>(pstStoreBasePath, cfg.getConsistentId()); + } + + if (consistentId != null) { + // The node scans the work directory and checks if there is a folder matching the consistent ID. + // If such a folder exists, we start up with this ID (compatibility mode) + final String subFolder = U.maskForFileName(consistentId.toString()); + + final L oldStyleFolderLockHolder = tryLock.apply(new File(pstStoreBasePath, subFolder)); + + if (oldStyleFolderLockHolder != null) { + return new PdsFolderSettings<>(pstStoreBasePath, + subFolder, + consistentId, + oldStyleFolderLockHolder, + true); + } + } + + final File[] oldStyleFolders = pstStoreBasePath.listFiles(DB_SUBFOLDERS_OLD_STYLE_FILTER); + + if (oldStyleFolders != null && oldStyleFolders.length != 0) { + for (File folder : oldStyleFolders) { + final String path = getPathDisplayableInfo(folder); + + U.warn(log, "There is other non-empty storage folder under storage base directory [" + path + "]"); + } + } + + for (FolderCandidate next : getNodeIndexSortedCandidates(pstStoreBasePath)) { + final L fileLockHolder = tryLock.apply(next.subFolderFile()); + + if (fileLockHolder != null) { + if (log.isInfoEnabled()) + log.info("Successfully locked persistence storage folder [" + next.subFolderFile() + "]"); + + return new PdsFolderSettings<>(pstStoreBasePath, + next.subFolderFile().getName(), + next.uuid(), + fileLockHolder, + false); + } + } + + return null; + } + + /** + * @return New PDS folder. + * @throws IgniteCheckedException In case of error. + */ + public PdsFolderSettings generateNew() throws IgniteCheckedException { + final File pstStoreBasePath = resolvePersistentStoreBasePath(); + + // was not able to find free slot, allocating new + try (final L rootDirLock = lockRootDirectory(pstStoreBasePath)) { + final List sortedCandidates = getNodeIndexSortedCandidates(pstStoreBasePath); + final int nodeIdx = sortedCandidates.isEmpty() ? 0 : (sortedCandidates.get(sortedCandidates.size() - 1).nodeIndex() + 1); + + return generateAndLockNewDbStorage(pstStoreBasePath, nodeIdx); + } + } + + /** + * Calculate overall folder size. + * + * @param dir directory to scan. + * @return total size in bytes. + */ + private static FolderParams folderSize(File dir) { + final FolderParams params = new FolderParams(); + + visitFolder(dir, params); + + return params; + } + + /** + * Scans provided directory and its sub dirs, collects found metrics. + * + * @param dir directory to start scan from. + * @param params input/output. + */ + private static void visitFolder(final File dir, final FolderParams params) { + for (File file : dir.listFiles()) { + if (file.isDirectory()) + visitFolder(file, params); + else { + params.size += file.length(); + params.lastModified = Math.max(params.lastModified, dir.lastModified()); + } + } + } + + /** + * @param folder folder to scan. + * @return folder displayable information. + */ + @NotNull private String getPathDisplayableInfo(final File folder) { + final SB res = new SB(); + + res.a(getCanonicalPath(folder)); + res.a(", "); + final FolderParams params = folderSize(folder); + + res.a(params.size); + res.a(" bytes, modified "); + final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("MM/dd/yyyy hh:mm a"); + + res.a(simpleDateFormat.format(params.lastModified)); + res.a(" "); + + return res.toString(); + } + + /** + * Returns the canonical pathname string of this abstract pathname. + * + * @param file path to convert. + * @return canonical pathname or at leas absolute if convert to canonical failed. + */ + @NotNull private String getCanonicalPath(final File file) { + try { + return file.getCanonicalPath(); + } + catch (IOException ignored) { + return file.getAbsolutePath(); + } + } + + /** + * Pad start of string with provided character. + * + * @param str sting to pad. + * @param minLength expected length. + * @param padChar padding character. + * @return padded string. + */ + private static String padStart(String str, int minLength, char padChar) { + A.notNull(str, "String should not be empty"); + if (str.length() >= minLength) + return str; + + final SB sb = new SB(minLength); + + for (int i = str.length(); i < minLength; ++i) + sb.a(padChar); + + sb.a(str); + + return sb.toString(); + } + + /** + * Creates new DB storage folder. + * + * @param pstStoreBasePath DB root path. + * @param nodeIdx next node index to use in folder name. + * @return new settings to be used in this node. + * @throws IgniteCheckedException if failed. + */ + @NotNull private PdsFolderSettings generateAndLockNewDbStorage( + final File pstStoreBasePath, + final int nodeIdx + ) throws IgniteCheckedException { + final UUID uuid = UUID.randomUUID(); + final String consIdBasedFolder = genNewStyleSubfolderName(nodeIdx, uuid); + final File newRandomFolder = U.resolveWorkDirectory(pstStoreBasePath.getAbsolutePath(), consIdBasedFolder, false); //mkdir here + final L fileLockHolder = tryLock.apply(newRandomFolder); + + if (fileLockHolder != null) { + if (log.isInfoEnabled()) + log.info("Successfully created new persistent storage folder [" + newRandomFolder + "]"); + + return new PdsFolderSettings<>(pstStoreBasePath, consIdBasedFolder, uuid, fileLockHolder, false); + } + + throw new IgniteCheckedException("Unable to lock file generated randomly [" + newRandomFolder + "]"); + } + + /** + * Generates DB subfolder name for provided node index (local) and UUID (consistent ID) + * + * @param nodeIdx node index. + * @param uuid consistent ID. + * @return folder file name + */ + @NotNull public static String genNewStyleSubfolderName(final int nodeIdx, final UUID uuid) { + final String uuidAsStr = uuid.toString(); + + assert uuidAsStr.matches(UUID_STR_PATTERN); + + final String nodeIdxPadded = padStart(Integer.toString(nodeIdx), 2, '0'); + + return DB_FOLDER_PREFIX + nodeIdxPadded + NODEIDX_UID_SEPARATOR + uuidAsStr; + } + + /** + * Acquires lock to root storage directory, used to lock root directory in case creating new files is required. + * + * @param pstStoreBasePath rood DB dir to lock + * @return locked directory, should be released and closed later + * @throws IgniteCheckedException if failed + */ + @NotNull private L lockRootDirectory(File pstStoreBasePath) + throws IgniteCheckedException { + + L rootDirLock; + int retry = 0; + + while ((rootDirLock = tryLock.apply(pstStoreBasePath)) == null) { + if (retry > 600) + throw new IgniteCheckedException("Unable to start under DB storage path [" + pstStoreBasePath + "]" + + ". Lock is being held to root directory"); + retry++; + } + + return rootDirLock; + } + + /** + * @param pstStoreBasePath root storage folder to scan. + * @return empty list if there is no files in folder to test. Non null value is returned for folder having + * applicable new style files. Collection is sorted ascending according to node ID, 0 node index is coming first. + */ + @Nullable private List getNodeIndexSortedCandidates(File pstStoreBasePath) { + final File[] files = pstStoreBasePath.listFiles(DB_SUBFOLDERS_NEW_STYLE_FILTER); + + if (files == null) + return Collections.emptyList(); + + final List res = new ArrayList<>(); + + for (File file : files) { + final FolderCandidate candidate = parseFileName(file); + + if (candidate != null) + res.add(candidate); + } + + Collections.sort(res, new Comparator() { + @Override public int compare( + FolderCandidate c1, FolderCandidate c2) { + return Integer.compare(c1.nodeIndex(), c2.nodeIndex()); + } + }); + + return res; + } + + /** + * @return DB storage absolute root path resolved as 'db' folder in Ignite work dir (by default) or using persistent + * store configuration. Null if persistence is not enabled. Returned folder is created automatically. + * @throws IgniteCheckedException if I/O failed. + */ + @Nullable private File resolvePersistentStoreBasePath() throws IgniteCheckedException { + final DataStorageConfiguration dsCfg = cfg.getDataStorageConfiguration(); + + if (dsCfg == null) + return null; + + final String pstPath = dsCfg.getStoragePath(); + + return U.resolveWorkDirectory( + cfg.getWorkDirectory(), + pstPath != null ? pstPath : DB_DEFAULT_FOLDER, + false + ); + } + + /** + * @param subFolderFile new style folder name to parse + * @return Pair of UUID and node index + */ + private FolderCandidate parseFileName(@NotNull final File subFolderFile) { + return parseSubFolderName(subFolderFile, log); + } + + /** + * @param subFolderFile new style file to parse. + * @param log Logger. + * @return Pair of UUID and node index. + */ + @Nullable public static FolderCandidate parseSubFolderName( + @NotNull final File subFolderFile, + @NotNull final IgniteLogger log) { + + final String fileName = subFolderFile.getName(); + final Matcher matcher = Pattern.compile(NODE_PATTERN).matcher(fileName); + if (!matcher.find()) + return null; + + int uidStart = matcher.end(); + + try { + final String uid = fileName.substring(uidStart); + final UUID uuid = UUID.fromString(uid); + final String substring = fileName.substring(DB_FOLDER_PREFIX.length(), uidStart - NODEIDX_UID_SEPARATOR.length()); + final int idx = Integer.parseInt(substring); + + return new FolderCandidate(subFolderFile, idx, uuid); + } + catch (Exception e) { + U.warn(log, "Unable to parse new style file format from [" + subFolderFile.getAbsolutePath() + "]: " + e); + + return null; + } + } + + /** Path metrics */ + private static class FolderParams { + /** Overall size in bytes. */ + private long size; + + /** Last modified. */ + private long lastModified; + } + + /** + * Represents parsed new style file and encoded parameters in this file name + */ + public static class FolderCandidate { + /** Absolute file path pointing to DB subfolder within DB storage root folder. */ + private final File subFolderFile; + + /** Node index (local, usually 0 if multiple nodes are not started at local PC). */ + private final int nodeIdx; + + /** Uuid contained in file name, is to be set as consistent ID. */ + private final UUID uuid; + + /** + * @param subFolderFile Absolute file path pointing to DB subfolder. + * @param nodeIdx Node index. + * @param uuid Uuid. + */ + public FolderCandidate(File subFolderFile, int nodeIdx, UUID uuid) { + this.subFolderFile = subFolderFile; + this.nodeIdx = nodeIdx; + this.uuid = uuid; + } + + /** + * @return Node index (local, usually 0 if multiple nodes are not started at local PC). + */ + public int nodeIndex() { + return nodeIdx; + } + + /** + * @return Uuid contained in file name, is to be set as consistent ID. + */ + public Serializable uuid() { + return uuid; + } + + /** + * @return Absolute file path pointing to DB subfolder within DB storage root folder. + */ + public File subFolderFile() { + return subFolderFile; + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java index 72e0720891a14..28400c35c1059 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java @@ -19,7 +19,7 @@ import java.io.File; import java.io.Serializable; -import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder; +import org.apache.ignite.internal.processors.cache.persistence.FileLockHolder; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.NotNull; @@ -28,7 +28,7 @@ /** * Class holds information required for folder generation for ignite persistent store */ -public class PdsFolderSettings { +public class PdsFolderSettings { /** * DB storage absolute root path resolved as 'db' folder in Ignite work dir (by default) or using persistent store * configuration.
@@ -48,7 +48,7 @@ public class PdsFolderSettings { * directory. This value is to be used at activate instead of locking.
May be null in case preconfigured * consistent ID is used or in case lock holder was already taken by other processor. */ - @Nullable private final NodeFileLockHolder fileLockHolder; + @Nullable private final L fileLockHolder; /** * Indicates if compatible mode is enabled, in that case all sub folders are generated from consistent ID without @@ -68,7 +68,7 @@ public class PdsFolderSettings { public PdsFolderSettings(@Nullable final File persistentStoreRootPath, final String folderName, final Serializable consistentId, - @Nullable final NodeFileLockHolder fileLockHolder, + @Nullable final L fileLockHolder, final boolean compatible) { this.consistentId = consistentId; @@ -125,7 +125,7 @@ public boolean isCompatible() { * * @return File lock holder with prelocked db directory. */ - @Nullable public NodeFileLockHolder getLockedFileLockHolder() { + @Nullable public L getLockedFileLockHolder() { return fileLockHolder; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java index cefaa04d15a89..6122e2893646d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.cache.persistence.filename; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder; /** * Resolves folders for PDS mode, may have side effect as setting random UUID as local node consistent ID. @@ -29,5 +30,5 @@ public interface PdsFoldersResolver { * @return PDS folder settings, consistentID and prelocked DB file lock. * @throws IgniteCheckedException if failed. */ - public PdsFolderSettings resolveFolders() throws IgniteCheckedException; + public PdsFolderSettings resolveFolders() throws IgniteCheckedException; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java index d23b5843579f9..10da397bc6230 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java @@ -165,7 +165,7 @@ import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.cacheDirectories; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.getPartitionFile; -import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.DB_DEFAULT_FOLDER; +import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.DB_DEFAULT_FOLDER; import static org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId.getTypeByPartId; import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.T_DATA; import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.getPageIO; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java index 96a32f808e4fe..92408920a442b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java @@ -31,7 +31,7 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; -import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor; +import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridStringLogger; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -40,7 +40,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID; -import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.parseSubFolderName; +import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.parseSubFolderName; /** * Test for new and old style persistent storage folders generation @@ -365,7 +365,7 @@ public void testStartNodeAfterDeactivate() throws Exception { " but actual class is " + (consistentId == null ? null : consistentId.getClass()), consistentId instanceof UUID); - return PdsConsistentIdProcessor.genNewStyleSubfolderName(nodeIdx, (UUID)consistentId); + return PdsFolderResolver.genNewStyleSubfolderName(nodeIdx, (UUID)consistentId); } /** @@ -655,12 +655,12 @@ private void assertNodeIndexesInFolder(Integer... indexes) throws IgniteCheckedE * @throws IgniteCheckedException if failed. */ @NotNull private Set getAllNodeIndexesInFolder() throws IgniteCheckedException { - final File curFolder = new File(U.defaultWorkDirectory(), PdsConsistentIdProcessor.DB_DEFAULT_FOLDER); + final File curFolder = new File(U.defaultWorkDirectory(), PdsFolderResolver.DB_DEFAULT_FOLDER); final Set indexes = new TreeSet<>(); - final File[] files = curFolder.listFiles(PdsConsistentIdProcessor.DB_SUBFOLDERS_NEW_STYLE_FILTER); + final File[] files = curFolder.listFiles(PdsFolderResolver.DB_SUBFOLDERS_NEW_STYLE_FILTER); for (File file : files) { - final PdsConsistentIdProcessor.FolderCandidate uid = parseSubFolderName(file, log); + final PdsFolderResolver.FolderCandidate uid = parseSubFolderName(file, log); if (uid != null) indexes.add(uid.nodeIndex()); @@ -679,7 +679,7 @@ private void assertPdsDirsDefaultExist(String subDirName) throws IgniteCheckedEx assertDirectoryExist(DataStorageConfiguration.DFLT_BINARY_METADATA_PATH, subDirName); assertDirectoryExist(DataStorageConfiguration.DFLT_WAL_PATH, subDirName); assertDirectoryExist(DataStorageConfiguration.DFLT_WAL_ARCHIVE_PATH, subDirName); - assertDirectoryExist(PdsConsistentIdProcessor.DB_DEFAULT_FOLDER, subDirName); + assertDirectoryExist(PdsFolderResolver.DB_DEFAULT_FOLDER, subDirName); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java index 5c6c7c00526a7..60b802044456b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java @@ -96,7 +96,7 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.CREATE; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; -import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.genNewStyleSubfolderName; +import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.genNewStyleSubfolderName; /** * Test suite for WAL segments reader and event generator. diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java index 4698c006f7a33..d3b93ce7f863d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java @@ -91,7 +91,6 @@ import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointEntryType; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointMarkersStorage; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; -import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor; import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; import org.apache.ignite.internal.processors.cache.persistence.tree.io.CompactablePageIO; @@ -136,6 +135,7 @@ import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DATA_FILENAME; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; +import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.genNewStyleSubfolderName; /** * @@ -784,7 +784,7 @@ public void testWalRenameDirSimple() throws Exception { */ private File cacheDir(final String cacheName, final String consId) throws IgniteCheckedException { final String subfolderName - = PdsConsistentIdProcessor.genNewStyleSubfolderName(0, UUID.fromString(consId)); + = genNewStyleSubfolderName(0, UUID.fromString(consId)); final File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false); From d318245b181e23e2b69f0a2dc7d2a57df9a6dc85 Mon Sep 17 00:00:00 2001 From: Nikolay Izhikov Date: Thu, 1 Apr 2021 18:55:27 +0300 Subject: [PATCH 06/28] IGNITE-13581 compilation fix. --- .../persistence/FoldersReuseCompatibilityTest.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java index cbcaba4494c2b..d2bf924ccd022 100644 --- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java +++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java @@ -32,14 +32,14 @@ import org.apache.ignite.configuration.PersistentStoreConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest; -import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor; +import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.jetbrains.annotations.NotNull; import org.junit.Test; -import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.parseSubFolderName; +import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.parseSubFolderName; /** * Test for new and old style persistent storage folders generation and compatible startup of current ignite version @@ -197,12 +197,12 @@ private void assertNodeIndexesInFolder(Integer... indexes) throws IgniteCheckedE * @throws IgniteCheckedException if failed. */ @NotNull private Set getAllNodeIndexesInFolder() throws IgniteCheckedException { - final File curFolder = new File(U.defaultWorkDirectory(), PdsConsistentIdProcessor.DB_DEFAULT_FOLDER); + final File curFolder = new File(U.defaultWorkDirectory(), PdsFolderResolver.DB_DEFAULT_FOLDER); final Set indexes = new TreeSet<>(); - final File[] files = curFolder.listFiles(PdsConsistentIdProcessor.DB_SUBFOLDERS_NEW_STYLE_FILTER); + final File[] files = curFolder.listFiles(PdsFolderResolver.DB_SUBFOLDERS_NEW_STYLE_FILTER); for (File file : files) { - final PdsConsistentIdProcessor.FolderCandidate uid + final PdsFolderResolver.FolderCandidate uid = parseSubFolderName(file, log); if (uid != null) @@ -222,7 +222,7 @@ private void assertPdsDirsDefaultExist(String subDirName) throws IgniteCheckedEx assertDirectoryExist(DataStorageConfiguration.DFLT_BINARY_METADATA_PATH, subDirName); assertDirectoryExist(PersistentStoreConfiguration.DFLT_WAL_STORE_PATH, subDirName); assertDirectoryExist(PersistentStoreConfiguration.DFLT_WAL_ARCHIVE_PATH, subDirName); - assertDirectoryExist(PdsConsistentIdProcessor.DB_DEFAULT_FOLDER, subDirName); + assertDirectoryExist(PdsFolderResolver.DB_DEFAULT_FOLDER, subDirName); } /** From dec2e640042f0e96ab93c96d4989c048b9b9a941 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Fri, 14 May 2021 11:20:47 +0300 Subject: [PATCH 07/28] IGNITE-13596 Calculation of primary flag based on tx flags (#9092) --- .../GridCommandHandlerClusterByClassTest.java | 2 +- .../ignite/util/GridCommandHandlerTest.java | 2 +- .../pagemem/wal/record/DataEntry.java | 60 +++++++-- .../pagemem/wal/record/LazyDataEntry.java | 6 +- .../pagemem/wal/record/MvccDataEntry.java | 2 +- .../pagemem/wal/record/UnwrapDataEntry.java | 6 +- .../processors/cache/GridCacheAdapter.java | 3 +- .../processors/cache/GridCacheEntryEx.java | 14 ++- .../processors/cache/GridCacheMapEntry.java | 29 ++--- .../processors/cache/GridCacheUtils.java | 14 ++- .../GridDistributedTxRemoteAdapter.java | 2 +- .../distributed/dht/GridDhtCacheAdapter.java | 1 + .../distributed/dht/GridDhtLockFuture.java | 4 +- .../dht/GridDhtTxPrepareFuture.java | 1 + .../colocated/GridDhtDetachedCacheEntry.java | 6 +- .../dht/preloader/GridDhtForceKeysFuture.java | 1 + .../preloader/GridDhtPartitionDemander.java | 1 + .../distributed/near/GridNearCacheEntry.java | 5 +- .../filename/PdsConsistentIdProcessor.java | 22 ---- .../filename/PdsFolderResolver.java | 7 +- .../reader/StandaloneWalRecordsIterator.java | 2 +- .../serializer/RecordDataV1Serializer.java | 10 +- .../transactions/IgniteTxLocalAdapter.java | 2 +- .../datastreamer/DataStreamerImpl.java | 3 +- .../org/apache/ignite/TestStorageUtils.java | 2 +- .../cache/GridCacheTestEntryEx.java | 3 +- .../consistency/AbstractReadRepairTest.java | 1 + ...pointSimulationWithRealCpDisabledTest.java | 2 +- .../db/wal/IgniteWalRebalanceTest.java | 2 +- .../db/wal/reader/IgniteWalReaderTest.java | 118 +++++++++++++++++- .../IgniteClusterSnapshotCheckTest.java | 1 + .../development/utils/DataEntryWrapper.java | 2 +- .../IgniteWalConverterSensitiveDataTest.java | 2 +- 33 files changed, 247 insertions(+), 91 deletions(-) diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java index aa55b91c69703..753453d302276 100644 --- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java +++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java @@ -1524,7 +1524,7 @@ private void corruptDataEntry( 0L, partId, updateCntr, - false + DataEntry.EMPTY_FLAGS ); GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ctx.shared().database(); diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index 0b525ee8fdd2a..604ac9bf0f634 100644 --- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -2631,7 +2631,7 @@ private void corruptDataEntry( 0L, partId, updateCntr, - false + DataEntry.EMPTY_FLAGS ); GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ctx.shared().database(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java index 4ac764bed59f9..9810130ceafa4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java @@ -29,6 +29,18 @@ * Represents Data Entry ({@link #key}, {@link #val value}) pair update {@link #op operation} in WAL log. */ public class DataEntry { + /** Empty flags. */ + public static final byte EMPTY_FLAGS = 0; + + /** */ + public static final byte PRIMARY_FLAG = 0b00000001; + + /** */ + public static final byte PRELOAD_FLAG = 0b00000010; + + /** */ + public static final byte FROM_STORE_FLAG = 0b00000100; + /** Cache ID. */ @GridToStringInclude protected int cacheId; @@ -62,9 +74,16 @@ public class DataEntry { @GridToStringInclude protected long partCnt; - /** If {@code true} then change made on primary node. */ + /** + * Bit flags. + *
    + *
  • 0 bit - primary - seted when current node is primary for entry partition.
  • + *
  • 1 bit - preload - seted when entry logged during preload(rebalance).
  • + *
  • 2 bit - fromStore - seted when entry loaded from third-party store.
  • + *
+ */ @GridToStringInclude - protected boolean primary; + protected byte flags; /** Constructor. */ private DataEntry() { @@ -81,7 +100,7 @@ private DataEntry() { * @param expireTime Expire time. * @param partId Partition ID. * @param partCnt Partition counter. - * @param primary {@code True} if node is primary for partition in the moment of logging. + * @param flags Entry flags. */ public DataEntry( int cacheId, @@ -93,7 +112,7 @@ public DataEntry( long expireTime, int partId, long partCnt, - boolean primary + byte flags ) { this.cacheId = cacheId; this.key = key; @@ -104,12 +123,36 @@ public DataEntry( this.expireTime = expireTime; this.partId = partId; this.partCnt = partCnt; - this.primary = primary; + this.flags = flags; // Only READ, CREATE, UPDATE and DELETE operations should be stored in WAL. assert op == GridCacheOperation.READ || op == GridCacheOperation.CREATE || op == GridCacheOperation.UPDATE || op == GridCacheOperation.DELETE : op; } + /** + * @param primary {@code True} if node is primary for partition in the moment of logging. + * @return Flags value. + */ + public static byte flags(boolean primary) { + return flags(primary, false, false); + } + + /** + * @param primary {@code True} if node is primary for partition in the moment of logging. + * @param preload {@code True} if logged during preload(rebalance). + * @param fromStore {@code True} if logged during loading from third-party store. + * @return Flags value. + */ + public static byte flags(boolean primary, boolean preload, boolean fromStore) { + byte val = EMPTY_FLAGS; + + val |= primary ? PRIMARY_FLAG : EMPTY_FLAGS; + val |= preload ? PRELOAD_FLAG : EMPTY_FLAGS; + val |= fromStore ? FROM_STORE_FLAG : EMPTY_FLAGS; + + return val; + } + /** * @return Cache ID. */ @@ -186,10 +229,11 @@ public long expireTime() { } /** - * @return {@code True} if node is primary for partition in the moment of logging. + * Entry flags. + * @see #flags */ - public boolean primary() { - return primary; + public byte flags() { + return flags; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java index e771bdfac95be..45bd54a7325ae 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java @@ -60,7 +60,7 @@ public class LazyDataEntry extends DataEntry implements MarshalledDataEntry { * @param expireTime Expire time. * @param partId Partition ID. * @param partCnt Partition counter. - * @param primary {@code True} if node is primary for partition in the moment of logging. + * @param flags Flags. */ public LazyDataEntry( GridCacheSharedContext cctx, @@ -75,9 +75,9 @@ public LazyDataEntry( long expireTime, int partId, long partCnt, - boolean primary + byte flags ) { - super(cacheId, null, null, op, nearXidVer, writeVer, expireTime, partId, partCnt, primary); + super(cacheId, null, null, op, nearXidVer, writeVer, expireTime, partId, partCnt, flags); this.cctx = cctx; this.keyType = keyType; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MvccDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MvccDataEntry.java index cecd867c9382a..387c147bc5bae 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MvccDataEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MvccDataEntry.java @@ -56,7 +56,7 @@ public MvccDataEntry( long partCnt, MvccVersion mvccVer ) { - super(cacheId, key, val, op, nearXidVer, writeVer, expireTime, partId, partCnt, false); + super(cacheId, key, val, op, nearXidVer, writeVer, expireTime, partId, partCnt, EMPTY_FLAGS); this.mvccVer = mvccVer; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java index 93951427de2c5..216e7332fbd26 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java @@ -48,7 +48,7 @@ public class UnwrapDataEntry extends DataEntry implements UnwrappedDataEntry { * @param partCnt Partition counter. * @param cacheObjValCtx cache object value context for unwrapping objects. * @param keepBinary disable unwrapping for non primitive objects, Binary Objects would be returned instead. - * @param primary {@code True} if node is primary for partition in the moment of logging. + * @param flags Flags. */ public UnwrapDataEntry( final int cacheId, @@ -62,8 +62,8 @@ public UnwrapDataEntry( final long partCnt, final CacheObjectValueContext cacheObjValCtx, final boolean keepBinary, - final boolean primary) { - super(cacheId, key, val, op, nearXidVer, writeVer, expireTime, partId, partCnt, primary); + final byte flags) { + super(cacheId, key, val, op, nearXidVer, writeVer, expireTime, partId, partCnt, flags); this.cacheObjValCtx = cacheObjValCtx; this.keepBinary = keepBinary; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java index 2eddcad974fe9..8469434f72cc9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java @@ -3858,7 +3858,8 @@ private void loadEntry(KeyCacheObject key, false, topVer, replicate ? DR_LOAD : DR_NONE, - true); + true, + false); } catch (IgniteCheckedException e) { throw new IgniteException("Failed to put cache value: " + entry, e); 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 fb68256393a53..43220f8c6e69e 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 @@ -756,6 +756,7 @@ public boolean tmLock(IgniteInternalTx tx, * @param topVer Topology version. * @param drType DR type. * @param fromStore {@code True} if value was loaded from store. + * @param primary {@code True} if current node is primary for partition. * @return {@code True} if initial value was set. * @throws IgniteCheckedException In case of error. * @throws GridCacheEntryRemovedException If entry was removed. @@ -767,9 +768,10 @@ default boolean initialValue(CacheObject val, boolean preload, AffinityTopologyVersion topVer, GridDrType drType, - boolean fromStore) throws IgniteCheckedException, GridCacheEntryRemovedException { + boolean fromStore, + boolean primary) throws IgniteCheckedException, GridCacheEntryRemovedException { return initialValue(val, ver, null, null, TxState.NA, TxState.NA, - ttl, expireTime, preload, topVer, drType, fromStore); + ttl, expireTime, preload, topVer, drType, fromStore, primary); } /** @@ -787,6 +789,7 @@ default boolean initialValue(CacheObject val, * @param topVer Topology version. * @param drType DR type. * @param fromStore {@code True} if value was loaded from store. + * @param primary {@code True} if current node is primary for partition. * @return {@code True} if initial value was set. * @throws IgniteCheckedException In case of error. * @throws GridCacheEntryRemovedException If entry was removed. @@ -802,9 +805,10 @@ default boolean initialValue(CacheObject val, boolean preload, AffinityTopologyVersion topVer, GridDrType drType, - boolean fromStore) throws IgniteCheckedException, GridCacheEntryRemovedException { + boolean fromStore, + boolean primary) throws IgniteCheckedException, GridCacheEntryRemovedException { return initialValue(val, ver, null, null, TxState.NA, TxState.NA, - ttl, expireTime, preload, topVer, drType, fromStore, null); + ttl, expireTime, preload, topVer, drType, fromStore, primary, null); } /** @@ -822,6 +826,7 @@ default boolean initialValue(CacheObject val, * @param topVer Topology version. * @param drType DR type. * @param fromStore {@code True} if value was loaded from store. + * @param primary {@code True} if current node is primary for partition. * @param row Pre-created data row, associated with this cache entry. * @return {@code True} if initial value was set. * @throws IgniteCheckedException In case of error. @@ -839,6 +844,7 @@ public boolean initialValue(CacheObject val, AffinityTopologyVersion topVer, GridDrType drType, boolean fromStore, + boolean primary, @Nullable CacheDataRow row) throws IgniteCheckedException, GridCacheEntryRemovedException; /** 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 b301c31ef9009..731ea97d842de 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 @@ -1569,7 +1569,7 @@ else if (interceptorVal != val0) updateCntr0 = nextPartitionCounter(tx, updateCntr); if (tx != null && cctx.group().persistenceEnabled() && cctx.group().walEnabled()) - logPtr = logTxUpdate(tx, val, expireTime, updateCntr0, topVer); + logPtr = logTxUpdate(tx, val, expireTime, updateCntr0); update(val, expireTime, ttl, newVer, true); @@ -1795,7 +1795,7 @@ protected Object keyValue(boolean cpy) { updateCntr0 = nextPartitionCounter(tx, updateCntr); if (tx != null && cctx.group().persistenceEnabled() && cctx.group().walEnabled()) - logPtr = logTxUpdate(tx, null, 0, updateCntr0, topVer); + logPtr = logTxUpdate(tx, null, 0, updateCntr0); drReplicate(drType, null, newVer, topVer); @@ -2152,7 +2152,7 @@ else if (ttl != CU.TTL_ZERO) update(updated, expireTime, ttl, ver, true); - logUpdate(op, updated, ver, expireTime, 0, cctx.affinity().affinityTopologyVersion()); + logUpdate(op, updated, ver, expireTime, 0, true); if (evt) { CacheObject evtOld = null; @@ -2184,7 +2184,7 @@ else if (ttl != CU.TTL_ZERO) update(null, CU.TTL_ETERNAL, CU.EXPIRE_TIME_ETERNAL, ver, true); - logUpdate(op, null, ver, CU.EXPIRE_TIME_ETERNAL, 0, cctx.affinity().affinityTopologyVersion()); + logUpdate(op, null, ver, CU.EXPIRE_TIME_ETERNAL, 0, true); if (evt) { CacheObject evtOld = null; @@ -3346,8 +3346,11 @@ private boolean skipInterceptor(@Nullable GridCacheVersion explicitVer) { AffinityTopologyVersion topVer, GridDrType drType, boolean fromStore, + boolean primary, CacheDataRow row ) throws IgniteCheckedException, GridCacheEntryRemovedException { + assert !primary || !(preload || fromStore); + ensureFreeSpace(); boolean deferred = false; @@ -3500,7 +3503,7 @@ else if (deletedUnlocked()) expireTime, partition(), updateCntr, - cctx.affinity().primaryByPartition(cctx.localNode(), partition(), topVer) + DataEntry.flags(primary, preload, fromStore) ))); } } @@ -4328,7 +4331,7 @@ protected boolean storeValue( * @param writeVer Write version. * @param expireTime Expire time. * @param updCntr Update counter. - * @param topVer Topology version. + * @param primary {@code True} if node is primary for entry in the moment of logging. */ protected void logUpdate( GridCacheOperation op, @@ -4336,7 +4339,7 @@ protected void logUpdate( GridCacheVersion writeVer, long expireTime, long updCntr, - AffinityTopologyVersion topVer + boolean primary ) throws IgniteCheckedException { // We log individual updates only in ATOMIC cache. assert cctx.atomic(); @@ -4353,7 +4356,7 @@ protected void logUpdate( expireTime, partition(), updCntr, - cctx.affinity().primaryByPartition(cctx.localNode(), partition(), topVer)))); + DataEntry.flags(primary)))); } catch (StorageException e) { throw new IgniteCheckedException("Failed to log ATOMIC cache update [key=" + key + ", op=" + op + @@ -4366,15 +4369,13 @@ protected void logUpdate( * @param val Value. * @param expireTime Expire time (or 0 if not applicable). * @param updCntr Update counter. - * @param topVer Topology version. * @throws IgniteCheckedException In case of log failure. */ protected WALPointer logTxUpdate( IgniteInternalTx tx, CacheObject val, long expireTime, - long updCntr, - AffinityTopologyVersion topVer + long updCntr ) throws IgniteCheckedException { assert cctx.transactional() && !cctx.transactionalSnapshot(); @@ -4395,7 +4396,7 @@ protected WALPointer logTxUpdate( expireTime, key.partition(), updCntr, - cctx.affinity().primaryByPartition(cctx.localNode(), partition(), topVer)))); + DataEntry.flags(CU.txOnPrimary(tx))))); } else return null; @@ -6504,7 +6505,7 @@ else if (interceptorVal != updated0) { long updateCntr0 = entry.nextPartitionCounter(topVer, primary, false, updateCntr); - entry.logUpdate(op, updated, newVer, newExpireTime, updateCntr0, topVer); + entry.logUpdate(op, updated, newVer, newExpireTime, updateCntr0, primary); if (!entry.isNear()) { newRow = entry.localPartition().dataStore().createRow( @@ -6591,7 +6592,7 @@ private void remove(@Nullable GridCacheVersionConflictContext conflictCtx, long updateCntr0 = entry.nextPartitionCounter(topVer, primary, false, updateCntr); - entry.logUpdate(op, null, newVer, 0, updateCntr0, topVer); + entry.logUpdate(op, null, newVer, 0, updateCntr0, primary); if (oldVal != null) { assert !entry.deletedUnlocked(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 76f8fcbb21e73..8e78b8368cd3e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -845,6 +845,17 @@ public static GridNearTxLocal txStartInternal(GridCacheContext ctx, IgniteIntern return prj.txStartEx(concurrency, isolation); } + /** + * @param tx Transaction. + * @return {@code True} if transaction is on primary node. + */ + public static boolean txOnPrimary(IgniteInternalTx tx) { + if (tx.near() && tx.local() && ((GridNearTxLocal)tx).colocatedLocallyMapped()) + return true; + + return tx.dht() && tx.local(); + } + /** * Alias for {@link #txString(IgniteInternalTx)}. */ @@ -1831,7 +1842,8 @@ private void process(KeyCacheObject key, CacheObject val, GridCacheVersion ver, true, topVer, GridDrType.DR_BACKUP, - true); + true, + false); break; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index 7cfe869407eec..423640e4cb5a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -632,7 +632,7 @@ else if (conflictCtx.isMerge()) { 0, txEntry.key().partition(), txEntry.updateCounter(), - cacheCtx.affinity().primaryByPartition(cctx.localNode(), txEntry.key().partition(), topVer) + DataEntry.flags(CU.txOnPrimary(this)) ), txEntry ) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java index 4bfb4fa11406c..28f61ffce2a3c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java @@ -586,6 +586,7 @@ private void loadEntry(KeyCacheObject key, false, topVer, replicate ? DR_LOAD : DR_NONE, + true, false); } catch (IgniteCheckedException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java index 06e554c36de2a..95a8bb390d3f3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java @@ -1146,7 +1146,8 @@ private void loadMissingFromStore() { false, topVer, GridDrType.DR_LOAD, - true); + true, + false); } catch (GridCacheEntryRemovedException e) { assert false : "Should not get removed exception while holding lock on entry " + @@ -1400,6 +1401,7 @@ void onResult(GridDhtLockResponse res) { true, topVer, replicate ? DR_PRELOAD : DR_NONE, + false, false)) { if (rec && !entry.isInternal()) cctx.events().addEvent(entry.partition(), entry.key(), cctx.localNodeId(), null, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java index 86074587ff74a..0ec8b32ff66f3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java @@ -1995,6 +1995,7 @@ void onResult(GridDhtTxPrepareResponse res) { true, topVer, drType, + false, false)) { if (rec && !entry.isInternal()) cacheCtx.events().addEvent(entry.partition(), entry.key(), cctx.localNodeId(), null, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java index 8e46629acb2cc..d7202e205ec16 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.colocated; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheOperation; @@ -75,13 +74,12 @@ public void resetFromPrimary(CacheObject val, GridCacheVersion ver) { /** {@inheritDoc} */ @Override protected void logUpdate(GridCacheOperation op, CacheObject val, GridCacheVersion writeVer, - long expireTime, long updCntr, AffinityTopologyVersion topVer) { + long expireTime, long updCntr, boolean primary) { // No-op for detached entries, index is updated on primary or backup nodes. } /** {@inheritDoc} */ - @Override protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr, - AffinityTopologyVersion topVer) { + @Override protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr) { return null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java index 7d16884785933..c0cecdc037c97 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java @@ -547,6 +547,7 @@ void onResult(GridDhtForceKeysResponse res) { true, topVer, replicate ? DR_PRELOAD : DR_NONE, + false, false )) { if (rec && !entry.isInternal()) 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 c012b5a4dff11..683da8a29db6d 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 @@ -927,6 +927,7 @@ private boolean preloadEntry(CacheDataRow row, AffinityTopologyVersion topVer) t topVer, cctx.isDrEnabled() ? DR_PRELOAD : DR_NONE, false, + false, row )) { cached.touch(); // Start tracking. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java index fc70aa3d6c420..a948dab2e0535 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java @@ -472,13 +472,12 @@ public boolean loadedValue(@Nullable IgniteInternalTx tx, /** {@inheritDoc} */ @Override protected void logUpdate(GridCacheOperation op, CacheObject val, GridCacheVersion ver, long expireTime, - long updCntr, AffinityTopologyVersion topVer) { + long updCntr, boolean primary) { // No-op: queries are disabled for near cache. } /** {@inheritDoc} */ - @Override protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr, - AffinityTopologyVersion topVer) { + @Override protected WALPointer logTxUpdate(IgniteInternalTx tx, CacheObject val, long expireTime, long updCntr) { return null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java index 56fe06b62ba4d..fb60746809fd5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.cache.persistence.filename; import java.io.File; -import java.io.Serializable; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.configuration.IgniteConfiguration; @@ -26,8 +25,6 @@ import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder; import org.apache.ignite.internal.util.typedef.internal.U; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; /** * Component for resolving PDS storage file names, also used for generating consistent ID for case PDS mode is enabled @@ -58,25 +55,6 @@ public PdsConsistentIdProcessor(final GridKernalContext ctx) { this.ctx = ctx; } - /** - * Prepares compatible PDS folder settings. No locking is performed, consistent ID is not overridden. - * - * @param pstStoreBasePath DB storage base path or null if persistence is not enabled. - * @param consistentId compatibility consistent ID - * @return PDS folder settings compatible with previous versions. - */ - private PdsFolderSettings compatibleResolve( - @Nullable final File pstStoreBasePath, - @NotNull final Serializable consistentId) { - - if (cfg.getConsistentId() != null) { - // compatible mode from configuration is used fot this case, no locking, no consitent id change - return new PdsFolderSettings<>(pstStoreBasePath, cfg.getConsistentId()); - } - - return new PdsFolderSettings<>(pstStoreBasePath, consistentId); - } - /** {@inheritDoc} */ @Override public PdsFolderSettings resolveFolders() throws IgniteCheckedException { if (settings == null) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java index 225a973bad247..0d9e79f9c126a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java @@ -131,17 +131,14 @@ public PdsFolderResolver( */ private PdsFolderSettings compatibleResolve( @Nullable final File pstStoreBasePath, - @Nullable final Serializable consistentId) { + @NotNull final Serializable consistentId) { if (cfg.getConsistentId() != null) { // compatible mode from configuration is used fot this case, no locking, no consitent id change return new PdsFolderSettings<>(pstStoreBasePath, cfg.getConsistentId()); } - if (consistentId == null) - return new PdsFolderSettings<>(pstStoreBasePath, consistentId); - - return null; + return new PdsFolderSettings<>(pstStoreBasePath, consistentId); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java index cfaa67031349a..cf0f2ba44b032 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java @@ -501,7 +501,7 @@ private DataEntry unwrapDataEntry(CacheObjectContext coCtx, DataEntry dataEntry, dataEntry.partitionCounter(), coCtx, keepBinary, - dataEntry.primary()); + dataEntry.flags()); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java index 6ad08ddd2044c..e7456c1baf433 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java @@ -1964,7 +1964,7 @@ else if (!entry.value().putValue(buf)) buf.putLong(entry.expireTime()); if (!(entry instanceof MvccDataEntry)) - buf.put(entry.primary() ? (byte)1 : 0); + buf.put(entry.flags()); } /** @@ -2073,7 +2073,7 @@ DataEntry readPlainDataEntry(ByteBufferBackedDataInput in, RecordType type) thro int partId = in.readInt(); long partCntr = in.readLong(); long expireTime = in.readLong(); - boolean primary = type == DATA_RECORD_V2 && in.readByte() == (byte)1; + byte flags = type == DATA_RECORD_V2 ? in.readByte() : (byte)0; GridCacheContext cacheCtx = cctx.cacheContext(cacheId); @@ -2097,7 +2097,7 @@ DataEntry readPlainDataEntry(ByteBufferBackedDataInput in, RecordType type) thro expireTime, partId, partCntr, - primary + flags ); } else @@ -2114,7 +2114,7 @@ DataEntry readPlainDataEntry(ByteBufferBackedDataInput in, RecordType type) thro expireTime, partId, partCntr, - primary + flags ); } @@ -2285,7 +2285,7 @@ private int cacheStatesSize(Map states) { public static class EncryptedDataEntry extends DataEntry { /** Constructor. */ EncryptedDataEntry() { - super(0, null, null, READ, null, null, 0, 0, 0, false); + super(0, null, null, READ, null, null, 0, 0, 0, EMPTY_FLAGS); } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index c08aa91c5ea93..b06a451785168 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -863,7 +863,7 @@ else if (op == READ) { 0, txEntry.key().partition(), txEntry.updateCounter(), - cacheCtx.affinity().primaryByPartition(cctx.localNode(), txEntry.key().partition(), topVer)))); + DataEntry.flags(CU.txOnPrimary(this))))); } ExpiryPolicy expiry = cacheCtx.expiryForTxEntry(txEntry); 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 452789d35a18d..534096935d914 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 @@ -2315,7 +2315,8 @@ else if (ttl == CU.TTL_NOT_CHANGED) false, topVer, primary ? GridDrType.DR_LOAD : GridDrType.DR_PRELOAD, - false); + false, + primary); entry.touch(); diff --git a/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java b/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java index e0f415d44e8ed..c9282b42c1893 100644 --- a/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java +++ b/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java @@ -83,7 +83,7 @@ public static void corruptDataEntry( 0L, partId, updateCntr, - false + DataEntry.EMPTY_FLAGS ); IgniteCacheDatabaseSharedManager db = ctx.shared().database(); 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 c40605406a389..4d86ed30bc266 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 @@ -84,7 +84,7 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr /** {@inheritDoc} */ @Override public boolean initialValue(CacheObject val, GridCacheVersion ver, long ttl, long expireTime, - boolean preload, AffinityTopologyVersion topVer, GridDrType drType, boolean fromStore) { + boolean preload, AffinityTopologyVersion topVer, GridDrType drType, boolean fromStore, boolean primary) { assert false; return false; @@ -711,6 +711,7 @@ void recheckLock() { AffinityTopologyVersion topVer, GridDrType drType, boolean fromStore, + boolean primary, CacheDataRow row ) throws IgniteCheckedException, GridCacheEntryRemovedException { assert false; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistency/AbstractReadRepairTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistency/AbstractReadRepairTest.java index 03548c2fba2ce..de8734e30b3c5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistency/AbstractReadRepairTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/consistency/AbstractReadRepairTest.java @@ -259,6 +259,7 @@ private InconsistentMapping setDifferentValuesForSameKey( false, AffinityTopologyVersion.NONE, GridDrType.DR_NONE, + false, false); assertTrue("iterableKey " + key + " already inited", init); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java index a8dd46c141132..ead3b85c6828d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java @@ -358,7 +358,7 @@ private void checkDataWalEntries(boolean mvcc) throws Exception { cctx.affinity().partition(i), i, new MvccVersionImpl(1000L, 10L, i + 1 /* Non-zero */)) : new DataEntry(cctx.cacheId(), key, val, op, null, cctx.cache().nextVersion(), 0L, - cctx.affinity().partition(i), i, false)); + cctx.affinity().partition(i), i, DataEntry.EMPTY_FLAGS)); } UUID cpId = UUID.randomUUID(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java index 869385713bea8..9d0ce52b616ba 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java @@ -772,7 +772,7 @@ public void testSwitchHistoricalRebalanceToFullWhileIteratingOverWAL() throws Ex 0, 0, 0, - false + DataEntry.EMPTY_FLAGS ))); File walDir = U.field(walMgr, "walWorkDir"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java index 60b802044456b..94544835a8615 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java @@ -79,6 +79,7 @@ import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.logger.NullLogger; import org.apache.ignite.testframework.MvccFeatureChecker; +import org.apache.ignite.testframework.junits.WithSystemProperty; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.transactions.Transaction; import org.jetbrains.annotations.NotNull; @@ -138,6 +139,9 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest { /** DataEntry from primary flag. */ private boolean primary = true; + /** DataEntry during rebalacne flag. */ + private boolean rebalance; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); @@ -280,8 +284,10 @@ private int iterateAndCount(WALIterator walIter) throws IgniteCheckedException { KeyCacheObject key = entry.key(); CacheObject val = entry.value(); - if (walRecord.type() == DATA_RECORD_V2) - assertEquals(primary, entry.primary()); + if (walRecord.type() == DATA_RECORD_V2) { + assertEquals(primary, (entry.flags() & DataEntry.PRIMARY_FLAG) != 0); + assertEquals(rebalance, (entry.flags() & DataEntry.PRELOAD_FLAG) != 0); + } if (DUMP_RECORDS) log.info("Op: " + entry.op() + ", Key: " + key + ", Value: " + val); @@ -1131,6 +1137,108 @@ public void testPrimaryFlagOnTwoNodes() throws Exception { ); } + /** + * Tests transaction generation and WAL for putAll cache operation. + * + * @throws Exception if failed. + */ + @Test + @WithSystemProperty(key = IgniteSystemProperties.IGNITE_DISABLE_WAL_DURING_REBALANCING, value = "false") + public void testRebalanceFlag() throws Exception { + backupCnt = 1; + + IgniteEx ignite = startGrid("node0"); + Ignite ignite1 = startGrid(1); + + ignite.cluster().state(ACTIVE); + + IgniteCache cache = ignite.cache(CACHE_NAME); + + int cntEntries = 100; + + List keys = findKeys(ignite.localNode(), cache, cntEntries, 0, 0); + + Map map = new TreeMap<>(); + + for (Integer key : keys) + map.putIfAbsent(key, new IndexedObject(key)); + + cache.putAll(map); + + Ignite ignite2 = startGrid(2); + + ignite.cluster().setBaselineTopology(ignite2.cluster().topologyVersion()); + + backupCnt = 0; + + awaitPartitionMapExchange(false, true, null); + + String subfolderName1 = genDbSubfolderName(ignite, 0); + String subfolderName2 = genDbSubfolderName(ignite1, 1); + String subfolderName3 = genDbSubfolderName(ignite2, 2); + + stopAllGrids(); + + String workDir = U.defaultWorkDirectory(); + + IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log); + + Map operationsFound = new EnumMap<>(GridCacheOperation.class); + + IgniteInClosure drHnd = dataRecord -> { + List entries = dataRecord.writeEntries(); + + for (DataEntry entry : entries) { + GridCacheOperation op = entry.op(); + Integer cnt = operationsFound.get(op); + + operationsFound.put(op, cnt == null ? 1 : (cnt + 1)); + } + }; + + scanIterateAndCount( + factory, + createIteratorParametersBuilder(workDir, subfolderName1) + .filesOrDirs( + workDir + "/db/wal/" + subfolderName1, + workDir + "/db/wal/archive/" + subfolderName1 + ), + 1, + 1, + null, drHnd + ); + + primary = false; + + scanIterateAndCount( + factory, + createIteratorParametersBuilder(workDir, subfolderName2) + .filesOrDirs( + workDir + "/db/wal/" + subfolderName2, + workDir + "/db/wal/archive/" + subfolderName2 + ), + 1, + 1, + null, + drHnd + ); + + rebalance = true; + + scanIterateAndCount( + factory, + createIteratorParametersBuilder(workDir, subfolderName3) + .filesOrDirs( + workDir + "/db/wal/" + subfolderName3, + workDir + "/db/wal/archive/" + subfolderName3 + ), + 1, + 0, + null, + drHnd + ); + } + /** * Tests transaction generation and WAL for putAll cache operation. * @@ -1501,8 +1609,10 @@ private Map iterateAndCountDataRecord( List entries = dataRecord.writeEntries(); for (DataEntry entry : entries) { - if (walRecord.type() == DATA_RECORD_V2) - assertEquals(primary, entry.primary()); + if (walRecord.type() == DATA_RECORD_V2) { + assertEquals(primary, (entry.flags() & DataEntry.PRIMARY_FLAG) != 0); + assertEquals(rebalance, (entry.flags() & DataEntry.PRELOAD_FLAG) != 0); + } GridCacheVersion globalTxId = entry.nearXidVersion(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java index 4527125faa3dd..108b26dfc75fc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotCheckTest.java @@ -421,6 +421,7 @@ public void testClusterSnapshotCheckFailsOnPartitionDataDiffers() throws Excepti topVer, DR_NONE, false, + false, null); assertTrue(success); diff --git a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java index 25e404f0abbf8..fd660de8e9f9d 100644 --- a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java +++ b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java @@ -66,7 +66,7 @@ public DataEntryWrapper( dataEntry.expireTime(), dataEntry.partitionId(), dataEntry.partitionCounter(), - dataEntry.primary() + dataEntry.flags() ); this.source = dataEntry; diff --git a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java index 487f791b5479d..00133ce676624 100644 --- a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java +++ b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteWalConverterSensitiveDataTest.java @@ -299,7 +299,7 @@ private Collection withSensitiveData() { 0, 0, 0, - false + DataEntry.EMPTY_FLAGS ); byte[] sensitiveDataBytes = SENSITIVE_DATA_VALUE_PREFIX.getBytes(StandardCharsets.UTF_8); From 92e6ff164d1f3885e374e208faf6e4f790fe76d4 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Fri, 14 May 2021 15:01:07 +0300 Subject: [PATCH 08/28] ignite-cdc Code review fixes. --- .../persistence/wal/serializer/RecordDataV1Serializer.java | 1 - .../persistence/wal/serializer/RecordDataV2Serializer.java | 2 -- 2 files changed, 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java index e7456c1baf433..ff4a199c69767 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java @@ -1356,7 +1356,6 @@ void writePlainRecord(WALRecord rec, ByteBuffer buf) throws IgniteCheckedExcepti break; - case DATA_RECORD: case DATA_RECORD_V2: DataRecord dataRec = (DataRecord)rec; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java index 08f132acd648d..56568cd0faf4b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java @@ -93,7 +93,6 @@ public RecordDataV2Serializer(GridCacheSharedContext cctx) { case MVCC_DATA_RECORD: return 4/*entry count*/ + 8/*timestamp*/ + dataSize((DataRecord)rec); - case DATA_RECORD: case DATA_RECORD_V2: return super.plainSize(rec) + 8/*timestamp*/; @@ -261,7 +260,6 @@ public RecordDataV2Serializer(GridCacheSharedContext cctx) { break; case MVCC_DATA_RECORD: - case DATA_RECORD: case DATA_RECORD_V2: DataRecord dataRec = (DataRecord)rec; From 197f3e216c19ad344dee362d9742abeeaa1f071a Mon Sep 17 00:00:00 2001 From: Nikolay Date: Fri, 14 May 2021 15:03:09 +0300 Subject: [PATCH 09/28] ignite-cdc Code review fixes. --- .../persistence/filename/PdsConsistentIdProcessor.java | 10 ++-------- .../cache/persistence/filename/PdsFolderResolver.java | 3 +-- 2 files changed, 3 insertions(+), 10 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java index fb60746809fd5..f6a9752ffe4ae 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java @@ -20,7 +20,6 @@ import java.io.File; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.NodeFileLockHolder; @@ -30,9 +29,6 @@ * Component for resolving PDS storage file names, also used for generating consistent ID for case PDS mode is enabled */ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements PdsFoldersResolver { - /** Config. */ - private final IgniteConfiguration cfg; - /** Logger. */ private final IgniteLogger log; @@ -50,7 +46,6 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd public PdsConsistentIdProcessor(final GridKernalContext ctx) { super(ctx); - this.cfg = ctx.config(); this.log = ctx.log(PdsFoldersResolver.class); this.ctx = ctx; } @@ -60,7 +55,7 @@ public PdsConsistentIdProcessor(final GridKernalContext ctx) { if (settings == null) { //here deprecated method is used to get compatible version of consistentId PdsFolderResolver resolver = - new PdsFolderResolver<>(cfg, log, ctx.discovery().consistentId(), this::tryLock); + new PdsFolderResolver<>(ctx.config(), log, ctx.discovery().consistentId(), this::tryLock); settings = resolver.resolve(); @@ -91,8 +86,7 @@ private NodeFileLockHolder tryLock(File dbStoreDirWithSubdirectory) { return null; final String path = dbStoreDirWithSubdirectory.getAbsolutePath(); - final NodeFileLockHolder fileLockHolder - = new NodeFileLockHolder(path, ctx, log); + final NodeFileLockHolder fileLockHolder = new NodeFileLockHolder(path, ctx, log); try { fileLockHolder.tryLock(1000); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java index 0d9e79f9c126a..aad5fc0cbf649 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java @@ -403,8 +403,7 @@ private static String padStart(String str, int minLength, char padChar) { } Collections.sort(res, new Comparator() { - @Override public int compare( - FolderCandidate c1, FolderCandidate c2) { + @Override public int compare(FolderCandidate c1, FolderCandidate c2) { return Integer.compare(c1.nodeIndex(), c2.nodeIndex()); } }); From f364a88b357d009aaf9988981169a551e172bb6d Mon Sep 17 00:00:00 2001 From: Nikolay Date: Sat, 15 May 2021 20:13:15 +0300 Subject: [PATCH 10/28] ignite-cdc Code review fixes. --- .../wal/FileWriteAheadLogManager.java | 56 +++++++++---------- .../visor/VisorDataTransferObject.java | 3 + .../node/VisorDataStorageConfiguration.java | 8 ++- .../visor/node/VisorGridConfiguration.java | 2 +- .../visor/node/VisorPersistenceMetrics.java | 2 +- .../ignite/internal/visor/tx/VisorTxInfo.java | 2 +- .../visor/verify/VisorIdleVerifyTaskArg.java | 2 +- .../ignite/development/utils/WalStat.java | 4 +- .../db/wal/IgniteWalRecoveryTest.java | 3 +- 9 files changed, 42 insertions(+), 40 deletions(-) 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 3982cb2d4734e..17e62464f0e5d 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 @@ -772,21 +772,19 @@ private void scheduleNextRolloverCheck() { assert timeoutRolloverMux != null; synchronized (timeoutRolloverMux) { - long nextEndTime; - - if (walForceArchiveTimeout > 0) { - long lastRollover = lastRolloverMs.get(); - nextEndTime = lastRollover == 0 ? U.currentTimeMillis() : lastRollover + walForceArchiveTimeout; - } - else { - long lastRecMs = lastRecordLoggedMs.get(); - nextEndTime = lastRecMs <= 0 ? U.currentTimeMillis() : lastRecMs + walAutoArchiveAfterInactivity; - } + long nextEndTime = walForceArchiveTimeout > 0 + ? nextTimeout(lastRolloverMs.get(), walForceArchiveTimeout) + : nextTimeout(lastRecordLoggedMs.get(), walAutoArchiveAfterInactivity); cctx.time().addTimeoutObject(timeoutRollover = new TimeoutRollover(nextEndTime)); } } + /** */ + private long nextTimeout(long lastEvt, long timeout) { + return lastEvt <= 0 ? U.currentTimeMillis() : lastEvt + timeout; + } + /** {@inheritDoc} */ @Override public int serializerVersion() { return serializerVer; @@ -801,30 +799,15 @@ private void checkWalRolloverRequired() { if (walAutoArchiveAfterInactivity <= 0 && walForceArchiveTimeout <= 0) return; // feature not configured, nothing to do. - final long lastRecMs = lastRecordLoggedMs.get(); - - if (lastRecMs == 0) + if (lastRecordLoggedMs.get() == 0) return; //no records were logged to current segment, does not consider inactivity. if (walForceArchiveTimeout > 0) { - final long lastRollover = lastRolloverMs.get(); - final long elapsedMs = U.currentTimeMillis() - lastRollover; - - if (elapsedMs < walForceArchiveTimeout) - return; // not enough time elapsed since last rollover. - - if (!lastRolloverMs.compareAndSet(lastRollover, 0)) - return; // record write occurred concurrently. - } - else { - final long elapsedMs = U.currentTimeMillis() - lastRecMs; - - if (elapsedMs <= walAutoArchiveAfterInactivity) - return; // not enough time elapsed since last write. - - if (!lastRecordLoggedMs.compareAndSet(lastRecMs, 0)) - return; // record write occurred concurrently. + if (!checkTimeout(lastRolloverMs, walForceArchiveTimeout)) + return; } + else if (!checkTimeout(lastRecordLoggedMs, walAutoArchiveAfterInactivity)) + return; final FileWriteHandle handle = currentHandle(); @@ -838,6 +821,19 @@ private void checkWalRolloverRequired() { } } + /** */ + private boolean checkTimeout(AtomicLong lastEvt, long timeout) { + final long lastEvtMs = lastEvt.get(); + + final long elapsedMs = U.currentTimeMillis() - lastEvtMs; + + if (elapsedMs <= timeout) + return false; // not enough time elapsed since last write. + + // Will return false if record write occurred concurrently. + return lastEvt.compareAndSet(lastEvtMs, 0); + } + /** {@inheritDoc} */ @Override public WALPointer log(WALRecord rec) throws IgniteCheckedException { return log(rec, RolloverType.NONE); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorDataTransferObject.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorDataTransferObject.java index 2424b2ee10dff..d77f3b6bbd776 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorDataTransferObject.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorDataTransferObject.java @@ -53,6 +53,9 @@ public abstract class VisorDataTransferObject implements Externalizable { /** Version 4. */ protected static final byte V4 = 4; + /** Version 5. */ + protected static final byte V5 = 5; + /** * @param col Source collection. * @param Collection type. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java index 243127c183e99..160b0fcb621ff 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java @@ -116,7 +116,7 @@ public class VisorDataStorageConfiguration extends VisorDataTransferObject { /** Time interval (in milliseconds) for rate-based metrics. */ private long metricsRateTimeInterval; - /** Time interval (in milliseconds) for running auto archiving for incompletely WAL segment. */ + /** Time interval of inactivity (in milliseconds) for running auto archiving for incompletely WAL segment. */ private long walAutoArchiveAfterInactivity; /** Time interval (in milliseconds) for running auto archiving for incompletely WAL segment. */ @@ -436,10 +436,10 @@ public boolean isWalCompactionEnabled() { out.writeInt(metricsSubIntervalCount); out.writeLong(metricsRateTimeInterval); out.writeLong(walAutoArchiveAfterInactivity); - out.writeLong(walForceArchiveTimeout); out.writeBoolean(writeThrottlingEnabled); out.writeInt(walBufSize); out.writeBoolean(walCompactionEnabled); + out.writeLong(walForceArchiveTimeout); } /** {@inheritDoc} */ @@ -472,13 +472,15 @@ public boolean isWalCompactionEnabled() { metricsSubIntervalCount = in.readInt(); metricsRateTimeInterval = in.readLong(); walAutoArchiveAfterInactivity = in.readLong(); - walForceArchiveTimeout = in.readLong(); writeThrottlingEnabled = in.readBoolean(); if (protoVer > V1) { walBufSize = in.readInt(); walCompactionEnabled = in.readBoolean(); } + + if (protoVer >= V5) + walForceArchiveTimeout = in.readLong(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java index 13c0e72e1ed83..0f5abc6ec75ee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java @@ -390,7 +390,7 @@ public VisorMvccConfiguration getMvccConfiguration() { /** {@inheritDoc} */ @Override public byte getProtocolVersion() { - return V4; + return V5; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java index dffe840460426..6f506ca7fd736 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java @@ -373,7 +373,7 @@ public long getSparseStorageSize() { /** {@inheritDoc} */ @Override public byte getProtocolVersion() { - return V4; + return V5; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java index 399e5c2b67ab0..20ed1944fff73 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java @@ -144,7 +144,7 @@ public VisorTxInfo(IgniteUuid xid, TransactionState state) { /** {@inheritDoc} */ @Override public byte getProtocolVersion() { - return V4; + return V5; } /** */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java index c474387076fd3..579eabe03c478 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java @@ -117,7 +117,7 @@ public Set excludeCaches() { /** {@inheritDoc} */ @Override public byte getProtocolVersion() { - return V4; + return V5; } /** {@inheritDoc} */ diff --git a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/WalStat.java b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/WalStat.java index cbed361d8026d..84be2256e4f46 100644 --- a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/WalStat.java +++ b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/WalStat.java @@ -127,7 +127,9 @@ void registerRecord(WALRecord record, WALPointer walPointer, boolean workDir) { if (type == WALRecord.RecordType.PAGE_RECORD) registerPageSnapshot((PageSnapshot)record); - else if (type == WALRecord.RecordType.DATA_RECORD || type == WALRecord.RecordType.DATA_RECORD_V2 || type == WALRecord.RecordType.MVCC_DATA_RECORD) + else if (type == WALRecord.RecordType.DATA_RECORD + || type == WALRecord.RecordType.DATA_RECORD_V2 + || type == WALRecord.RecordType.MVCC_DATA_RECORD) registerDataRecord((DataRecord)record); else if (type == WALRecord.RecordType.TX_RECORD || type == WALRecord.RecordType.MVCC_TX_RECORD) registerTxRecord((TxRecord)record); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java index f57b3f38d9b2c..8e19f99b101fe 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java @@ -784,8 +784,7 @@ public void testWalRenameDirSimple() throws Exception { * @throws IgniteCheckedException If fail. */ private File cacheDir(final String cacheName, final String consId) throws IgniteCheckedException { - final String subfolderName - = genNewStyleSubfolderName(0, UUID.fromString(consId)); + final String subfolderName = genNewStyleSubfolderName(0, UUID.fromString(consId)); final File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false); From a9d09f4bf208a12a1f70e07d70d0d577a4b0bcdd Mon Sep 17 00:00:00 2001 From: Nikolay Date: Mon, 17 May 2021 10:11:17 +0300 Subject: [PATCH 11/28] ignite-cdc Code review fixes. --- .../cache/persistence/db/wal/reader/IgniteWalReaderTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java index 94544835a8615..4e8904e72d3ea 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java @@ -395,7 +395,9 @@ public void testForceArchiveSegment() throws Exception { waitingForEvt.set(true); // Flag for skipping regular log() and rollOver(). - boolean recordedAfterSleep = forceArchiveSegment.await(forceArchiveSegmentMs + 1001, TimeUnit.MILLISECONDS); + putDummyRecords(ignite, 1); + + boolean recordedAfterSleep = forceArchiveSegment.await(forceArchiveSegmentMs + getTestTimeout(), TimeUnit.MILLISECONDS); stopGrid(); From 16bea82a51c7fc8126747b4ea0475fa52c225822 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Mon, 17 May 2021 11:26:20 +0300 Subject: [PATCH 12/28] Revert "IGNITE-14353 Ability to specify postfix for IgniteLogger instead of nodeId (#8923)" This reverts commit f590a18b --- .../apache/ignite/internal/IgnitionEx.java | 24 ++------------- .../ignite/internal/util/IgniteUtils.java | 14 ++++----- .../ignite/logger/LoggerNodeIdAware.java | 3 -- .../ignite/logger/LoggerPostfixAware.java | 30 ------------------- .../apache/ignite/logger/java/JavaLogger.java | 27 ++++------------- .../ignite/logger/java/JavaLoggerTest.java | 8 ++--- .../logger/GridLog4jRollingFileAppender.java | 27 +++++++---------- .../junits/logger/GridTestLog4jLogger.java | 23 ++++++-------- .../ignite/logger/log4j/Log4JLogger.java | 18 ++--------- .../logger/log4j/Log4jNodeIdFilePath.java | 15 +++++----- .../ignite/logger/log4j2/Log4J2Logger.java | 19 +++--------- .../logger/log4j2/Log4j2LoggerSelfTest.java | 18 +++-------- 12 files changed, 56 insertions(+), 170 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/logger/LoggerPostfixAware.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index 95ef6984dc2d0..f63b8618e7415 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -110,7 +110,6 @@ import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.logger.LoggerNodeIdAware; -import org.apache.ignite.logger.LoggerPostfixAware; import org.apache.ignite.logger.java.JavaLogger; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.MarshallerUtils; @@ -2558,21 +2557,8 @@ private void initializeDefaultSpi(IgniteConfiguration cfg) { * @return Initialized logger. * @throws IgniteCheckedException If failed. */ - private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId, String workDir) - throws IgniteCheckedException { - return initLogger(cfgLog, nodeId, null, workDir); - } - - /** - * @param cfgLog Configured logger. - * @param nodeId Local node ID. - * @param postfix Log file postfix. - * @param workDir Work directory. - * @return Initialized logger. - * @throws IgniteCheckedException If failed. - */ @SuppressWarnings("ErrorNotRethrown") - public IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId, String postfix, String workDir) + private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId, String workDir) throws IgniteCheckedException { try { Exception log4jInitErr = null; @@ -2636,12 +2622,8 @@ public IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId, Strin ((JavaLogger)cfgLog).setWorkDirectory(workDir); // Set node IDs for all file appenders. - if (cfgLog instanceof LoggerNodeIdAware) { - if (nodeId == null && cfgLog instanceof LoggerPostfixAware) - ((LoggerPostfixAware)cfgLog).setPostfix(postfix); - else - ((LoggerNodeIdAware)cfgLog).setNodeId(nodeId); - } + if (cfgLog instanceof LoggerNodeIdAware) + ((LoggerNodeIdAware)cfgLog).setNodeId(nodeId); if (log4jInitErr != null) U.warn(cfgLog, "Failed to initialize Log4JLogger (falling back to standard java logging): " diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 3a9961964b4c9..de79bf8da35db 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -8826,15 +8826,15 @@ public static void removeJavaNoOpLogger(Collection rmvHnds) { } /** - * Attaches postfix to log file name. + * Attaches node ID to log file name. * - * @param postfix Postfix. + * @param nodeId Node ID. * @param fileName File name. - * @return File name with postfix. + * @return File name with node ID. */ @SuppressWarnings("IfMayBeConditional") - public static String logFileName(String postfix, String fileName) { - assert postfix != null; + public static String nodeIdLogFileName(UUID nodeId, String fileName) { + assert nodeId != null; assert fileName != null; fileName = GridFilenameUtils.separatorsToSystem(fileName); @@ -8842,9 +8842,9 @@ public static String logFileName(String postfix, String fileName) { int dot = fileName.lastIndexOf('.'); if (dot < 0 || dot == fileName.length() - 1) - return fileName + '-' + postfix; + return fileName + '-' + U.id8(nodeId); else - return fileName.substring(0, dot) + '-' + postfix + fileName.substring(dot); + return fileName.substring(0, dot) + '-' + U.id8(nodeId) + fileName.substring(dot); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java b/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java index 16c5125ead523..492d93c507ce2 100644 --- a/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java +++ b/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java @@ -21,10 +21,7 @@ /** * Interface for Ignite file appenders to attach node ID to log file names. - * - * @deprecated Use {@link LoggerPostfixAware} instead. */ -@Deprecated public interface LoggerNodeIdAware { /** * Sets node ID. diff --git a/modules/core/src/main/java/org/apache/ignite/logger/LoggerPostfixAware.java b/modules/core/src/main/java/org/apache/ignite/logger/LoggerPostfixAware.java deleted file mode 100644 index cb86621bddb00..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/logger/LoggerPostfixAware.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.logger; - -/** - * Interface for Ignite file appenders to attach postfix to log file names. - */ -public interface LoggerPostfixAware extends LoggerNodeIdAware { - /** - * Sets postfix. - * - * @param postfix Postfix. - */ - public void setPostfix(String postfix); -} diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java index 89dc70799b24d..c82f01d63b0cb 100644 --- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java +++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java @@ -34,7 +34,7 @@ import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.LoggerPostfixAware; +import org.apache.ignite.logger.LoggerNodeIdAware; import org.jetbrains.annotations.Nullable; import static java.util.logging.Level.FINE; @@ -95,7 +95,7 @@ * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger * injection. */ -public class JavaLogger implements IgniteLogger, LoggerPostfixAware { +public class JavaLogger implements IgniteLogger, LoggerNodeIdAware { /** */ public static final String DFLT_CONFIG_PATH = "config/java.util.logging.properties"; @@ -128,10 +128,6 @@ public class JavaLogger implements IgniteLogger, LoggerPostfixAware { @GridToStringExclude private volatile UUID nodeId; - /** Postfix. */ - @GridToStringExclude - private volatile String postfix; - /** * Creates new logger. */ @@ -372,27 +368,14 @@ public void setWorkDirectory(String workDir) { @Override public void setNodeId(UUID nodeId) { A.notNull(nodeId, "nodeId"); - postfix(nodeId, U.id8(nodeId)); - } - - /** {@inheritDoc} */ - @Override public void setPostfix(String postfix) { - A.notNull(postfix, "postfix"); - - postfix(null, postfix); - } - - /** */ - private void postfix(UUID nodeId, String postfix) { - if (this.postfix != null) + if (this.nodeId != null) return; synchronized (mux) { // Double check. - if (this.postfix != null) + if (this.nodeId != null) return; - this.postfix = postfix; this.nodeId = nodeId; } @@ -402,7 +385,7 @@ private void postfix(UUID nodeId, String postfix) { return; try { - fileHnd.postfix(postfix, workDir); + fileHnd.nodeId(nodeId, workDir); } catch (IgniteCheckedException | IOException e) { throw new RuntimeException("Failed to enable file handler.", e); diff --git a/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java b/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java index dba26500c62bc..4687ca992c9c2 100644 --- a/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java @@ -20,11 +20,10 @@ import java.util.UUID; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.LoggerPostfixAware; +import org.apache.ignite.logger.LoggerNodeIdAware; import org.apache.ignite.testframework.junits.common.GridCommonTest; import org.junit.Test; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; /** @@ -44,10 +43,7 @@ public void testLogInitialize() throws Exception { log = new JavaLogger(); ((JavaLogger)log).setWorkDirectory(U.defaultWorkDirectory()); - UUID id = UUID.fromString("00000000-1111-2222-3333-444444444444"); - - ((LoggerPostfixAware)log).setNodeId(id); - assertEquals(id, ((LoggerPostfixAware)log).getNodeId()); + ((LoggerNodeIdAware)log).setNodeId(UUID.fromString("00000000-1111-2222-3333-444444444444")); System.out.println(log.toString()); diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java index 5d73e7c09f441..fc9f38f0a5e7d 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java @@ -23,16 +23,16 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.LoggerPostfixAware; +import org.apache.ignite.logger.LoggerNodeIdAware; import org.apache.log4j.Layout; import org.apache.log4j.RollingFileAppender; /** * Log4J {@link org.apache.log4j.RollingFileAppender} with added support for grid node IDs. */ -public class GridLog4jRollingFileAppender extends RollingFileAppender implements LoggerPostfixAware { - /** Postfix. */ - private String postfix; +public class GridLog4jRollingFileAppender extends RollingFileAppender implements LoggerNodeIdAware { + /** Node ID. */ + private UUID nodeId; /** Basic log file name. */ private String baseFileName; @@ -78,23 +78,18 @@ private void init() { GridTestLog4jLogger.addAppender(this); } - /** {@inheritDoc} */ - @Override public void setNodeId(UUID nodeId) { - setPostfix(U.id8(nodeId)); - } - /** {@inheritDoc} */ @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext") - @Override public synchronized void setPostfix(String postfix) { - A.notNull(postfix, "postfix"); + @Override public synchronized void setNodeId(UUID nodeId) { + A.notNull(nodeId, "nodeId"); - this.postfix = postfix; + this.nodeId = nodeId; if (fileName != null) { // fileName could be null if IGNITE_HOME is not defined. if (baseFileName == null) baseFileName = fileName; - fileName = U.logFileName(postfix, baseFileName); + fileName = U.nodeIdLogFileName(nodeId, baseFileName); } else { String tmpDir = IgniteSystemProperties.getString("java.io.tmpdir"); @@ -102,20 +97,20 @@ private void init() { if (tmpDir != null) { baseFileName = new File(tmpDir, "ignite.log").getAbsolutePath(); - fileName = U.logFileName(postfix, baseFileName); + fileName = U.nodeIdLogFileName(nodeId, baseFileName); } } } /** {@inheritDoc} */ @Override public synchronized UUID getNodeId() { - throw new UnsupportedOperationException("getNodeId"); + return nodeId; } /** {@inheritDoc} */ @Override public synchronized void setFile(String fileName, boolean fileAppend, boolean bufIO, int bufSize) throws IOException { - if (postfix != null) + if (nodeId != null) super.setFile(fileName, fileAppend, bufIO, bufSize); } } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java index 65d16541a3cf8..4464828715c5e 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java @@ -33,7 +33,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; -import org.apache.ignite.logger.LoggerPostfixAware; +import org.apache.ignite.logger.LoggerNodeIdAware; import org.apache.log4j.Appender; import org.apache.log4j.Category; import org.apache.log4j.ConsoleAppender; @@ -77,7 +77,7 @@ * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger * injection. */ -public class GridTestLog4jLogger implements IgniteLogger, LoggerPostfixAware { +public class GridTestLog4jLogger implements IgniteLogger, LoggerNodeIdAware { /** Appenders. */ private static Collection fileAppenders = new GridConcurrentHashSet<>(); @@ -102,9 +102,9 @@ public class GridTestLog4jLogger implements IgniteLogger, LoggerPostfixAware { /** Quiet flag. */ private final boolean quiet; - /** Postfix. */ + /** Node ID. */ @GridToStringExclude - private String postfix; + private UUID nodeId; /** * Creates new logger and automatically detects if root logger already @@ -407,18 +407,13 @@ public static void removeAppender(FileAppender a) { /** {@inheritDoc} */ @Override public void setNodeId(UUID nodeId) { - setPostfix(U.id8(nodeId)); - } - - /** {@inheritDoc} */ - @Override public void setPostfix(String postfix) { - A.notNull(postfix, "postfix"); + A.notNull(nodeId, "nodeId"); - this.postfix = postfix; + this.nodeId = nodeId; for (FileAppender a : fileAppenders) { - if (a instanceof LoggerPostfixAware) { - ((LoggerPostfixAware)a).setPostfix(postfix); + if (a instanceof LoggerNodeIdAware) { + ((LoggerNodeIdAware)a).setNodeId(nodeId); a.activateOptions(); } @@ -427,7 +422,7 @@ public static void removeAppender(FileAppender a) { /** {@inheritDoc} */ @Override public UUID getNodeId() { - throw new UnsupportedOperationException("getNodeId"); + return nodeId; } /** diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java index fbcfc0847b524..ec0a5b3c79c24 100644 --- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java +++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java @@ -33,7 +33,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; -import org.apache.ignite.logger.LoggerPostfixAware; +import org.apache.ignite.logger.LoggerNodeIdAware; import org.apache.log4j.Appender; import org.apache.log4j.Category; import org.apache.log4j.ConsoleAppender; @@ -79,7 +79,7 @@ * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger * injection. */ -public class Log4JLogger implements IgniteLogger, LoggerPostfixAware, Log4jFileAware { +public class Log4JLogger implements IgniteLogger, LoggerNodeIdAware, Log4jFileAware { /** Appenders. */ private static Collection fileAppenders = new GridConcurrentHashSet<>(); @@ -503,14 +503,9 @@ public static void removeAppender(FileAppender a) { @Override public void setNodeId(UUID nodeId) { A.notNull(nodeId, "nodeId"); - postfix(nodeId, U.id8(nodeId)); - } - - /** */ - private void postfix(UUID nodeId, String postfix) { this.nodeId = nodeId; - updateFilePath(new Log4jNodeIdFilePath(postfix)); + updateFilePath(new Log4jNodeIdFilePath(nodeId)); } /** {@inheritDoc} */ @@ -518,13 +513,6 @@ private void postfix(UUID nodeId, String postfix) { return nodeId; } - /** {@inheritDoc} */ - @Override public void setPostfix(String postfix) { - A.notNull(postfix, "postfix"); - - postfix(nodeId, postfix); - } - /** * Gets files for all registered file appenders. * diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java index 813f1e858a893..d4d1892502750 100644 --- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java +++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java @@ -18,6 +18,7 @@ package org.apache.ignite.logger.log4j; import java.io.File; +import java.util.UUID; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -31,29 +32,29 @@ class Log4jNodeIdFilePath implements IgniteClosure { private static final long serialVersionUID = 0L; /** Node id. */ - private final String postfix; + private final UUID nodeId; /** * Creates new instance. * - * @param postfix Postfix. + * @param id Node id. */ - Log4jNodeIdFilePath(String postfix) { - this.postfix = postfix; + Log4jNodeIdFilePath(UUID id) { + nodeId = id; } /** {@inheritDoc} */ @Override public String apply(String oldPath) { if (!F.isEmpty(U.IGNITE_LOG_DIR)) - return U.logFileName(postfix, new File(U.IGNITE_LOG_DIR, "ignite.log").getAbsolutePath()); + return U.nodeIdLogFileName(nodeId, new File(U.IGNITE_LOG_DIR, "ignite.log").getAbsolutePath()); if (oldPath != null) // fileName could be null if IGNITE_HOME is not defined. - return U.logFileName(postfix, oldPath); + return U.nodeIdLogFileName(nodeId, oldPath); String tmpDir = IgniteSystemProperties.getString("java.io.tmpdir"); if (tmpDir != null) - return U.logFileName(postfix, new File(tmpDir, "ignite.log").getAbsolutePath()); + return U.nodeIdLogFileName(nodeId, new File(tmpDir, "ignite.log").getAbsolutePath()); System.err.println("Failed to get tmp directory for log file."); diff --git a/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java b/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java index 665be453fbc69..0f5c313a3f81c 100644 --- a/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java +++ b/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java @@ -31,7 +31,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; -import org.apache.ignite.logger.LoggerPostfixAware; +import org.apache.ignite.logger.LoggerNodeIdAware; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Marker; @@ -81,7 +81,7 @@ * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger * injection. */ -public class Log4J2Logger implements IgniteLogger, LoggerPostfixAware { +public class Log4J2Logger implements IgniteLogger, LoggerNodeIdAware { /** */ private static final String NODE_ID = "nodeId"; @@ -387,14 +387,10 @@ public Logger createConsoleLogger() { @Override public void setNodeId(UUID nodeId) { A.notNull(nodeId, "nodeId"); - postfix(nodeId, U.id8(nodeId)); - } - - private void postfix(UUID nodeId, String postfix) { this.nodeId = nodeId; - // Set postfix as system variable to be used at configuration. - System.setProperty(NODE_ID, postfix); + // Set nodeId as system variable to be used at configuration. + System.setProperty(NODE_ID, U.id8(nodeId)); if (inited) { final LoggerContext ctx = impl.getContext(); @@ -419,13 +415,6 @@ private void postfix(UUID nodeId, String postfix) { return nodeId; } - /** {@inheritDoc} */ - @Override public void setPostfix(String postfix) { - A.notNull(postfix, "nodeId"); - - postfix(null, postfix); - } - /** * Gets {@link IgniteLogger} wrapper around log4j logger for the given * category. If category is {@code null}, then root logger is returned. If diff --git a/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java b/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java index fcc40b8b36724..3ebd15d656ca7 100644 --- a/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java +++ b/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java @@ -26,7 +26,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.LoggerPostfixAware; +import org.apache.ignite.logger.LoggerNodeIdAware; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.GridTestUtils; @@ -70,9 +70,7 @@ public void testFileConstructor() throws Exception { assertTrue(log.toString().contains("Log4J2Logger")); assertTrue(log.toString().contains(xml.getPath())); - UUID id = UUID.randomUUID(); - ((LoggerPostfixAware)log).setNodeId(id); - assertEquals(id, ((LoggerPostfixAware)log).getNodeId()); + ((LoggerNodeIdAware)log).setNodeId(UUID.randomUUID()); checkLog(log); } @@ -95,9 +93,7 @@ public void testUrlConstructor() throws Exception { assertTrue(log.toString().contains("Log4J2Logger")); assertTrue(log.toString().contains(url.getPath())); - UUID id = UUID.randomUUID(); - ((LoggerPostfixAware)log).setNodeId(id); - assertEquals(id, ((LoggerPostfixAware)log).getNodeId()); + ((LoggerNodeIdAware)log).setNodeId(UUID.randomUUID()); checkLog(log); } @@ -114,9 +110,7 @@ public void testPathConstructor() throws Exception { assertTrue(log.toString().contains("Log4J2Logger")); assertTrue(log.toString().contains(LOG_PATH_TEST)); - UUID id = UUID.randomUUID(); - ((LoggerPostfixAware)log).setNodeId(id); - assertEquals(id, ((LoggerPostfixAware)log).getNodeId()); + ((LoggerNodeIdAware)log).setNodeId(UUID.randomUUID()); checkLog(log); } @@ -146,10 +140,6 @@ public void testSystemNodeId() throws Exception { new Log4J2Logger(LOG_PATH_TEST).setNodeId(id); assertEquals(U.id8(id), System.getProperty("nodeId")); - - new Log4J2Logger(LOG_PATH_TEST).setPostfix("myapp"); - - assertEquals("myapp", System.getProperty("nodeId")); } /** From 1b2de827764701788bd024cc71c42081d37d86c7 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Thu, 20 May 2021 12:44:17 +0300 Subject: [PATCH 13/28] ignite-cdc revert Logger changes --- .../ignite/logger/java/JavaLoggerFileHandler.java | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java index e729441aaae2a..6320589c4793d 100644 --- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java @@ -80,15 +80,6 @@ public final class JavaLoggerFileHandler extends StreamHandler { * @param nodeId Node id. */ public void nodeId(UUID nodeId, String workDir) throws IgniteCheckedException, IOException { - postfix(U.id8(nodeId), workDir); - } - - /** - * Sets Postfix and instantiates {@link FileHandler} delegate. - * - * @param postfix Postfix. - */ - public void postfix(String postfix, String workDir) throws IgniteCheckedException, IOException { if (delegate != null) return; @@ -99,7 +90,7 @@ public void postfix(String postfix, String workDir) throws IgniteCheckedExceptio if (ptrn == null) ptrn = "ignite-%{id8}.%g.log"; - ptrn = new File(logDirectory(workDir), ptrn.replace("%{id8}", postfix)).getAbsolutePath(); + ptrn = new File(logDirectory(workDir), ptrn.replace("%{id8}", U.id8(nodeId))).getAbsolutePath(); int limit = getIntProperty(clsName + ".limit", 0); From 4204deb4053a9f8ed9ba8a4c6f8e8dc5bb413c63 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Thu, 20 May 2021 14:25:42 +0300 Subject: [PATCH 14/28] IGNITE-14353 Ability to specify application name for IgniteLogger (#9101) --- config/ignite-log4j2.xml | 4 +- config/java.util.logging.properties | 2 +- .../_docs/code-snippets/xml/log4j2-config.xml | 4 +- .../apache/ignite/internal/IgnitionEx.java | 9 +++-- .../LoggerNodeIdAndApplicationAware.java | 39 +++++++++++++++++++ .../ignite/logger/LoggerNodeIdAware.java | 3 ++ .../apache/ignite/logger/java/JavaLogger.java | 8 ++-- .../logger/java/JavaLoggerFileHandler.java | 18 ++++++++- modules/core/src/test/config/log4j2-test.xml | 4 +- .../src/test/config/log4j2-verbose-test.xml | 4 +- .../ignite/logger/java/JavaLoggerTest.java | 20 +++++++++- .../logger/GridLog4jRollingFileAppender.java | 8 ++-- .../junits/logger/GridTestLog4jLogger.java | 10 ++--- .../ignite/logger/log4j/Log4JLogger.java | 8 ++-- .../logger/log4j/Log4jNodeIdFilePath.java | 14 +++++-- .../ignite/logger/log4j2/Log4J2Logger.java | 10 +++-- .../logger/log4j2/Log4j2LoggerSelfTest.java | 16 +++++--- 17 files changed, 137 insertions(+), 44 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAndApplicationAware.java diff --git a/config/ignite-log4j2.xml b/config/ignite-log4j2.xml index b55c563716f16..5beee0b22a2a2 100644 --- a/config/ignite-log4j2.xml +++ b/config/ignite-log4j2.xml @@ -31,8 +31,8 @@ - + diff --git a/config/java.util.logging.properties b/config/java.util.logging.properties index f68af5b1f8521..3e34a50d62716 100644 --- a/config/java.util.logging.properties +++ b/config/java.util.logging.properties @@ -68,7 +68,7 @@ java.util.logging.ConsoleHandler.level=INFO # under `$IGNITE_HOME/work/log/` directory. The placeholder `%{id8}` is a truncated node ID. # org.apache.ignite.logger.java.JavaLoggerFileHandler.formatter=org.apache.ignite.logger.java.JavaLoggerFormatter -org.apache.ignite.logger.java.JavaLoggerFileHandler.pattern=ignite-%{id8}.%g.log +org.apache.ignite.logger.java.JavaLoggerFileHandler.pattern=%{app}-%{id8}.%g.log org.apache.ignite.logger.java.JavaLoggerFileHandler.level=INFO org.apache.ignite.logger.java.JavaLoggerFileHandler.limit=10485760 org.apache.ignite.logger.java.JavaLoggerFileHandler.count=10 diff --git a/docs/_docs/code-snippets/xml/log4j2-config.xml b/docs/_docs/code-snippets/xml/log4j2-config.xml index 2b412285942b5..1728ed9d845c9 100644 --- a/docs/_docs/code-snippets/xml/log4j2-config.xml +++ b/docs/_docs/code-snippets/xml/log4j2-config.xml @@ -29,8 +29,8 @@ - + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index f1e5b69f464f3..9aff1acf29344 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -109,6 +109,7 @@ import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware; import org.apache.ignite.logger.LoggerNodeIdAware; import org.apache.ignite.logger.java.JavaLogger; import org.apache.ignite.marshaller.Marshaller; @@ -2301,7 +2302,7 @@ private IgniteConfiguration initializeConfiguration(IgniteConfiguration cfg) if (!F.isEmpty(predefineConsistentId)) myCfg.setConsistentId(predefineConsistentId); - IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), nodeId, workDir); + IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), null, nodeId, workDir); assert cfgLog != null; @@ -2565,7 +2566,7 @@ private void initializeDefaultSpi(IgniteConfiguration cfg) { * @throws IgniteCheckedException If failed. */ @SuppressWarnings("ErrorNotRethrown") - private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId, String workDir) + private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, @Nullable String app, UUID nodeId, String workDir) throws IgniteCheckedException { try { Exception log4jInitErr = null; @@ -2629,7 +2630,9 @@ private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId, Stri ((JavaLogger)cfgLog).setWorkDirectory(workDir); // Set node IDs for all file appenders. - if (cfgLog instanceof LoggerNodeIdAware) + if (cfgLog instanceof LoggerNodeIdAndApplicationAware) + ((LoggerNodeIdAndApplicationAware)cfgLog).setApplicationAndNode(app, nodeId); + else if (cfgLog instanceof LoggerNodeIdAware) ((LoggerNodeIdAware)cfgLog).setNodeId(nodeId); if (log4jInitErr != null) diff --git a/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAndApplicationAware.java b/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAndApplicationAware.java new file mode 100644 index 0000000000000..2b2ba1e9773c9 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAndApplicationAware.java @@ -0,0 +1,39 @@ +/* + * 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.logger; + +import java.util.UUID; +import org.jetbrains.annotations.Nullable; + +/** + * Interface for Ignite file appenders to attach postfix to log file names. + */ +public interface LoggerNodeIdAndApplicationAware extends LoggerNodeIdAware { + /** {@inheritDoc} */ + @Override public default void setNodeId(UUID nodeId) { + setApplicationAndNode(null, nodeId); + } + + /** + * Sets application name and node ID. + * + * @param application Application. + * @param nodeId Node ID. + */ + public void setApplicationAndNode(@Nullable String application, UUID nodeId); +} diff --git a/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java b/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java index 492d93c507ce2..a17e3409925c3 100644 --- a/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java +++ b/modules/core/src/main/java/org/apache/ignite/logger/LoggerNodeIdAware.java @@ -21,7 +21,10 @@ /** * Interface for Ignite file appenders to attach node ID to log file names. + * + * @deprecated Use {@link LoggerNodeIdAndApplicationAware} instead. */ +@Deprecated public interface LoggerNodeIdAware { /** * Sets node ID. diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java index c82f01d63b0cb..b7c36e3635701 100644 --- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java +++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java @@ -34,7 +34,7 @@ import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware; import org.jetbrains.annotations.Nullable; import static java.util.logging.Level.FINE; @@ -95,7 +95,7 @@ * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger * injection. */ -public class JavaLogger implements IgniteLogger, LoggerNodeIdAware { +public class JavaLogger implements IgniteLogger, LoggerNodeIdAndApplicationAware { /** */ public static final String DFLT_CONFIG_PATH = "config/java.util.logging.properties"; @@ -365,7 +365,7 @@ public void setWorkDirectory(String workDir) { } /** {@inheritDoc} */ - @Override public void setNodeId(UUID nodeId) { + @Override public void setApplicationAndNode(@Nullable String application, UUID nodeId) { A.notNull(nodeId, "nodeId"); if (this.nodeId != null) @@ -385,7 +385,7 @@ public void setWorkDirectory(String workDir) { return; try { - fileHnd.nodeId(nodeId, workDir); + fileHnd.nodeId(application, nodeId, workDir); } catch (IgniteCheckedException | IOException e) { throw new RuntimeException("Failed to enable file handler.", e); diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java index 6320589c4793d..5560e43fc3eec 100644 --- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java @@ -80,6 +80,16 @@ public final class JavaLoggerFileHandler extends StreamHandler { * @param nodeId Node id. */ public void nodeId(UUID nodeId, String workDir) throws IgniteCheckedException, IOException { + nodeId(null, nodeId, workDir); + } + + /** + * Sets Node id and instantiates {@link FileHandler} delegate. + * + * @param app Application name. + * @param nodeId Node id. + */ + public void nodeId(@Nullable String app, UUID nodeId, String workDir) throws IgniteCheckedException, IOException { if (delegate != null) return; @@ -88,9 +98,13 @@ public void nodeId(UUID nodeId, String workDir) throws IgniteCheckedException, I String ptrn = manager.getProperty(clsName + ".pattern"); if (ptrn == null) - ptrn = "ignite-%{id8}.%g.log"; + ptrn = "%{app}-%{id8}.%g.log"; + + String fileName = ptrn + .replace("%{app}", app != null ? app : "ignite") + .replace("%{id8}", U.id8(nodeId)); - ptrn = new File(logDirectory(workDir), ptrn.replace("%{id8}", U.id8(nodeId))).getAbsolutePath(); + ptrn = new File(logDirectory(workDir), fileName).getAbsolutePath(); int limit = getIntProperty(clsName + ".limit", 0); diff --git a/modules/core/src/test/config/log4j2-test.xml b/modules/core/src/test/config/log4j2-test.xml index 8b46f00bca50e..9740c2ce27e08 100644 --- a/modules/core/src/test/config/log4j2-test.xml +++ b/modules/core/src/test/config/log4j2-test.xml @@ -31,8 +31,8 @@ - + diff --git a/modules/core/src/test/config/log4j2-verbose-test.xml b/modules/core/src/test/config/log4j2-verbose-test.xml index ff6325b290513..d2d7a7805c258 100644 --- a/modules/core/src/test/config/log4j2-verbose-test.xml +++ b/modules/core/src/test/config/log4j2-verbose-test.xml @@ -26,8 +26,8 @@ - + diff --git a/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java b/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java index 4687ca992c9c2..fa9f5363d617a 100644 --- a/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java @@ -20,7 +20,8 @@ import java.util.UUID; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonTest; import org.junit.Test; @@ -43,7 +44,7 @@ public void testLogInitialize() throws Exception { log = new JavaLogger(); ((JavaLogger)log).setWorkDirectory(U.defaultWorkDirectory()); - ((LoggerNodeIdAware)log).setNodeId(UUID.fromString("00000000-1111-2222-3333-444444444444")); + ((LoggerNodeIdAndApplicationAware)log).setApplicationAndNode(null, UUID.fromString("00000000-1111-2222-3333-444444444444")); System.out.println(log.toString()); @@ -67,5 +68,20 @@ public void testLogInitialize() throws Exception { // Ensure we don't get pattern, only actual file name is allowed here. assert !log.fileName().contains("%"); + assert log.fileName().contains("ignite"); + + System.clearProperty("java.util.logging.config.file"); + GridTestUtils.setFieldValue(JavaLogger.class, JavaLogger.class, "inited", false); + + log = new JavaLogger(); + + ((JavaLogger)log).setWorkDirectory(U.defaultWorkDirectory()); + ((LoggerNodeIdAndApplicationAware)log).setApplicationAndNode("other-app", UUID.fromString("00000000-1111-2222-3333-444444444444")); + + assert log.fileName() != null; + + // Ensure we don't get pattern, only actual file name is allowed here. + assert !log.fileName().contains("%"); + assert log.fileName().contains("other-app"); } } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java index fc9f38f0a5e7d..98841d867291a 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridLog4jRollingFileAppender.java @@ -23,14 +23,15 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware; import org.apache.log4j.Layout; import org.apache.log4j.RollingFileAppender; +import org.jetbrains.annotations.Nullable; /** * Log4J {@link org.apache.log4j.RollingFileAppender} with added support for grid node IDs. */ -public class GridLog4jRollingFileAppender extends RollingFileAppender implements LoggerNodeIdAware { +public class GridLog4jRollingFileAppender extends RollingFileAppender implements LoggerNodeIdAndApplicationAware { /** Node ID. */ private UUID nodeId; @@ -79,8 +80,7 @@ private void init() { } /** {@inheritDoc} */ - @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext") - @Override public synchronized void setNodeId(UUID nodeId) { + @Override public synchronized void setApplicationAndNode(@Nullable String application, UUID nodeId) { A.notNull(nodeId, "nodeId"); this.nodeId = nodeId; diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java index 4464828715c5e..6563e1c84c71f 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java @@ -33,7 +33,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware; import org.apache.log4j.Appender; import org.apache.log4j.Category; import org.apache.log4j.ConsoleAppender; @@ -77,7 +77,7 @@ * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger * injection. */ -public class GridTestLog4jLogger implements IgniteLogger, LoggerNodeIdAware { +public class GridTestLog4jLogger implements IgniteLogger, LoggerNodeIdAndApplicationAware { /** Appenders. */ private static Collection fileAppenders = new GridConcurrentHashSet<>(); @@ -406,14 +406,14 @@ public static void removeAppender(FileAppender a) { } /** {@inheritDoc} */ - @Override public void setNodeId(UUID nodeId) { + @Override public void setApplicationAndNode(@Nullable String application, UUID nodeId) { A.notNull(nodeId, "nodeId"); this.nodeId = nodeId; for (FileAppender a : fileAppenders) { - if (a instanceof LoggerNodeIdAware) { - ((LoggerNodeIdAware)a).setNodeId(nodeId); + if (a instanceof LoggerNodeIdAndApplicationAware) { + ((LoggerNodeIdAndApplicationAware)a).setApplicationAndNode(application, nodeId); a.activateOptions(); } diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java index ec0a5b3c79c24..682919b8a703a 100644 --- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java +++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java @@ -33,7 +33,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware; import org.apache.log4j.Appender; import org.apache.log4j.Category; import org.apache.log4j.ConsoleAppender; @@ -79,7 +79,7 @@ * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger * injection. */ -public class Log4JLogger implements IgniteLogger, LoggerNodeIdAware, Log4jFileAware { +public class Log4JLogger implements IgniteLogger, LoggerNodeIdAndApplicationAware, Log4jFileAware { /** Appenders. */ private static Collection fileAppenders = new GridConcurrentHashSet<>(); @@ -500,12 +500,12 @@ public static void removeAppender(FileAppender a) { } /** {@inheritDoc} */ - @Override public void setNodeId(UUID nodeId) { + @Override public void setApplicationAndNode(@Nullable String application, UUID nodeId) { A.notNull(nodeId, "nodeId"); this.nodeId = nodeId; - updateFilePath(new Log4jNodeIdFilePath(nodeId)); + updateFilePath(new Log4jNodeIdFilePath(application, nodeId)); } /** {@inheritDoc} */ diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java index d4d1892502750..715684dd3455e 100644 --- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java +++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4jNodeIdFilePath.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; +import org.jetbrains.annotations.Nullable; /** * Closure that generates file path adding node id to filename as a suffix. @@ -31,22 +32,29 @@ class Log4jNodeIdFilePath implements IgniteClosure { /** */ private static final long serialVersionUID = 0L; + /** Applictaion name. */ + private final @Nullable String app; + /** Node id. */ private final UUID nodeId; /** * Creates new instance. * + * @param app Application name. * @param id Node id. */ - Log4jNodeIdFilePath(UUID id) { + Log4jNodeIdFilePath(@Nullable String app, UUID id) { + this.app = app; nodeId = id; } /** {@inheritDoc} */ @Override public String apply(String oldPath) { + String fileName = (app != null ? app : "ignite") + ".log"; + if (!F.isEmpty(U.IGNITE_LOG_DIR)) - return U.nodeIdLogFileName(nodeId, new File(U.IGNITE_LOG_DIR, "ignite.log").getAbsolutePath()); + return U.nodeIdLogFileName(nodeId, new File(U.IGNITE_LOG_DIR, fileName).getAbsolutePath()); if (oldPath != null) // fileName could be null if IGNITE_HOME is not defined. return U.nodeIdLogFileName(nodeId, oldPath); @@ -54,7 +62,7 @@ class Log4jNodeIdFilePath implements IgniteClosure { String tmpDir = IgniteSystemProperties.getString("java.io.tmpdir"); if (tmpDir != null) - return U.nodeIdLogFileName(nodeId, new File(tmpDir, "ignite.log").getAbsolutePath()); + return U.nodeIdLogFileName(nodeId, new File(tmpDir, fileName).getAbsolutePath()); System.err.println("Failed to get tmp directory for log file."); diff --git a/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java b/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java index 0f5c313a3f81c..66050c27d6e09 100644 --- a/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java +++ b/modules/log4j2/src/main/java/org/apache/ignite/logger/log4j2/Log4J2Logger.java @@ -31,7 +31,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Marker; @@ -81,10 +81,13 @@ * logger in your task/job code. See {@link org.apache.ignite.resources.LoggerResource} annotation about logger * injection. */ -public class Log4J2Logger implements IgniteLogger, LoggerNodeIdAware { +public class Log4J2Logger implements IgniteLogger, LoggerNodeIdAndApplicationAware { /** */ private static final String NODE_ID = "nodeId"; + /** */ + private static final String APP_ID = "appId"; + /** */ private static final String CONSOLE_APPENDER = "autoConfiguredIgniteConsoleAppender"; @@ -384,13 +387,14 @@ public Logger createConsoleLogger() { } /** {@inheritDoc} */ - @Override public void setNodeId(UUID nodeId) { + @Override public void setApplicationAndNode(@Nullable String application, UUID nodeId) { A.notNull(nodeId, "nodeId"); this.nodeId = nodeId; // Set nodeId as system variable to be used at configuration. System.setProperty(NODE_ID, U.id8(nodeId)); + System.setProperty(APP_ID, application != null ? application : "ignite"); if (inited) { final LoggerContext ctx = impl.getContext(); diff --git a/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java b/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java index 3ebd15d656ca7..8d0b6fcec2275 100644 --- a/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java +++ b/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/Log4j2LoggerSelfTest.java @@ -26,7 +26,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.GridTestUtils; @@ -70,7 +70,7 @@ public void testFileConstructor() throws Exception { assertTrue(log.toString().contains("Log4J2Logger")); assertTrue(log.toString().contains(xml.getPath())); - ((LoggerNodeIdAware)log).setNodeId(UUID.randomUUID()); + ((LoggerNodeIdAndApplicationAware)log).setApplicationAndNode(null, UUID.randomUUID()); checkLog(log); } @@ -93,7 +93,7 @@ public void testUrlConstructor() throws Exception { assertTrue(log.toString().contains("Log4J2Logger")); assertTrue(log.toString().contains(url.getPath())); - ((LoggerNodeIdAware)log).setNodeId(UUID.randomUUID()); + ((LoggerNodeIdAndApplicationAware)log).setApplicationAndNode(null, UUID.randomUUID()); checkLog(log); } @@ -110,7 +110,7 @@ public void testPathConstructor() throws Exception { assertTrue(log.toString().contains("Log4J2Logger")); assertTrue(log.toString().contains(LOG_PATH_TEST)); - ((LoggerNodeIdAware)log).setNodeId(UUID.randomUUID()); + ((LoggerNodeIdAndApplicationAware)log).setApplicationAndNode(null, UUID.randomUUID()); checkLog(log); } @@ -137,9 +137,15 @@ private void checkLog(IgniteLogger log) { public void testSystemNodeId() throws Exception { UUID id = UUID.randomUUID(); - new Log4J2Logger(LOG_PATH_TEST).setNodeId(id); + new Log4J2Logger(LOG_PATH_TEST).setApplicationAndNode(null, id); assertEquals(U.id8(id), System.getProperty("nodeId")); + assertEquals("ignite", System.getProperty("appId")); + + new Log4J2Logger(LOG_PATH_TEST).setApplicationAndNode("other-app", id); + + assertEquals(U.id8(id), System.getProperty("nodeId")); + assertEquals("other-app", System.getProperty("appId")); } /** From 7acff016cf8e42b11822d06205a21a888bcc6b2b Mon Sep 17 00:00:00 2001 From: Nikolay Date: Mon, 7 Jun 2021 13:29:15 +0300 Subject: [PATCH 15/28] IGNITE-13581 Capture Data Change implementation (#8909) --- bin/ignite-cdc.sh | 27 + .../ignite/cache/CacheEntryVersion.java | 66 +++ .../cdc/ChangeDataCaptureConfiguration.java | 101 ++++ .../ignite/cdc/ChangeDataCaptureConsumer.java | 77 +++ .../ignite/cdc/ChangeDataCaptureEvent.java | 79 +++ .../ignite/cdc/ChangeDataCaptureLoader.java | 74 +++ .../DataStorageConfiguration.java | 59 ++ .../apache/ignite/internal/IgniteKernal.java | 2 +- .../apache/ignite/internal/IgnitionEx.java | 95 +-- .../internal/cdc/ChangeDataCapture.java | 544 ++++++++++++++++++ .../cdc/ChangeDataCaptureConsumerState.java | 114 ++++ .../cdc/ChangeDataCaptureEventImpl.java | 106 ++++ .../cdc/ChangeDataCaptureFileLockHolder.java | 55 ++ .../internal/cdc/WalRecordsConsumer.java | 174 ++++++ .../cache/persistence/FileLockHolder.java | 14 +- .../GridCacheDatabaseSharedManager.java | 6 +- .../filename/PdsFolderResolver.java | 3 +- .../wal/FileWriteAheadLogManager.java | 15 + .../cache/version/GridCacheVersion.java | 31 +- .../version/GridCacheVersionManager.java | 5 + .../processors/resource/GridResourceIoc.java | 2 +- .../ignite/internal/util/IgniteUtils.java | 95 +++ .../ChangeDataCaptureCommandLineStartup.java | 157 +++++ .../startup/cmdline/CommandLineStartup.java | 2 +- .../cdc/AbstractChangeDataCaptureTest.java | 250 ++++++++ .../ChangeDataCaptureCacheVersionTest.java | 232 ++++++++ .../ignite/cdc/ChangeDataCaptureSelfTest.java | 417 ++++++++++++++ .../plugin/AbstractCachePluginProvider.java | 70 +++ .../testsuites/IgnitePdsTestSuite2.java | 6 + .../PlatformTestCachePluginProvider.java | 49 +- .../cdc/SqlChangeDataCaptureTest.java | 179 ++++++ .../IgniteBinaryCacheQueryTestSuite.java | 6 +- .../cdc/cdc-config-without-persistence.xml | 30 + .../test/config/cdc/correct-cdc-config.xml | 53 ++ .../src/test/config/cdc/double-cdc-config.xml | 30 + .../test/config/cdc/double-ignite-config.xml | 30 + .../ChangeDataCaptureConfigurationTest.java | 140 +++++ .../testsuites/IgniteSpringTestSuite.java | 6 +- parent/pom.xml | 4 + 39 files changed, 3237 insertions(+), 168 deletions(-) create mode 100755 bin/ignite-cdc.sh create mode 100644 modules/core/src/main/java/org/apache/ignite/cache/CacheEntryVersion.java create mode 100644 modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConfiguration.java create mode 100644 modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConsumer.java create mode 100644 modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureEvent.java create mode 100644 modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureLoader.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCapture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureConsumerState.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureEventImpl.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureFileLockHolder.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java create mode 100644 modules/core/src/main/java/org/apache/ignite/startup/cmdline/ChangeDataCaptureCommandLineStartup.java create mode 100644 modules/core/src/test/java/org/apache/ignite/cdc/AbstractChangeDataCaptureTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureCacheVersionTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/plugin/AbstractCachePluginProvider.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlChangeDataCaptureTest.java create mode 100644 modules/spring/src/test/config/cdc/cdc-config-without-persistence.xml create mode 100644 modules/spring/src/test/config/cdc/correct-cdc-config.xml create mode 100644 modules/spring/src/test/config/cdc/double-cdc-config.xml create mode 100644 modules/spring/src/test/config/cdc/double-ignite-config.xml create mode 100644 modules/spring/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureConfigurationTest.java diff --git a/bin/ignite-cdc.sh b/bin/ignite-cdc.sh new file mode 100755 index 0000000000000..8f806a4eacfde --- /dev/null +++ b/bin/ignite-cdc.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +# +# 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. +# + +export MAIN_CLASS="org.apache.ignite.startup.cmdline.ChangeDataCaptureCommandLineStartup" + +if [ "${IGNITE_HOME:-}" = "" ]; + then IGNITE_HOME_TMP="$(dirname "$(cd "$(dirname "$0")"; "pwd")")"; + else IGNITE_HOME_TMP=${IGNITE_HOME}; +fi + +${IGNITE_HOME_TMP}/bin/ignite.sh "$@" diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheEntryVersion.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheEntryVersion.java new file mode 100644 index 0000000000000..f22a4e0f7f5b5 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheEntryVersion.java @@ -0,0 +1,66 @@ +/* + * 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.cache; + +import java.io.Serializable; +import java.util.Map; +import org.apache.ignite.internal.processors.cache.CacheConflictResolutionManager; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager; +import org.apache.ignite.lang.IgniteExperimental; + +/** + * Entry event order. + * Two concurrent updates of the same entry can be ordered based on {@link CacheEntryVersion} comparsion. + * Greater value means that event occurs later. + * + * @see CacheConflictResolutionManager + * @see GridCacheVersionManager#dataCenterId(byte) + */ +@IgniteExperimental +public interface CacheEntryVersion extends Comparable, Serializable { + /** + * Order of the update. Value is an incremental counter value. Scope of counter is node. + * @return Version order. + */ + public long order(); + + /** @return Node order on which this version was assigned. */ + public int nodeOrder(); + + /** + * Cluster id is a value to distinguish updates in case user wants to aggregate and sort updates from several + * Ignite clusters. {@code clusterId} id can be set for the node using + * {@link GridCacheVersionManager#dataCenterId(byte)}. + * + * @return Cluster id. + */ + public byte clusterId(); + + /** @return Topology version plus number of seconds from the start time of the first grid node. */ + public int topologyVersion(); + + /** + * If source of the update is "local" cluster then {@code this} will be returned. + * If updated comes from the other cluster using {@link IgniteInternalCache#putAllConflict(Map)} then + * @return Replication version. + * @see IgniteInternalCache#putAllConflict(Map) + * @see IgniteInternalCache#removeAllConflict(Map) + */ + public CacheEntryVersion otherClusterVersion(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConfiguration.java b/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConfiguration.java new file mode 100644 index 0000000000000..302d70f9adb09 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConfiguration.java @@ -0,0 +1,101 @@ +/* + * 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.cdc; + +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.lang.IgniteExperimental; + +/** + * This class defines {@link ChangeDataCapture} runtime configuration. + * Configuration is passed to {@link ChangeDataCapture} constructor. + */ +@IgniteExperimental +public class ChangeDataCaptureConfiguration { + /** */ + private static final int DFLT_LOCK_TIMEOUT = 1000; + + /** */ + private static final long DFLT_CHECK_FREQ = 1000L; + + /** */ + private static final boolean DFLT_KEEP_BINARY = true; + + /** Change Data Capture consumer. */ + private ChangeDataCaptureConsumer consumer; + + /** Keep binary flag.
Default value {@code true}. */ + private boolean keepBinary = DFLT_KEEP_BINARY; + + /** + * {@link ChangeDataCapture} acquire file lock on startup to ensure exclusive consumption. + * This property specifies amount of time to wait for lock acquisition.
+ * Default is {@code 1000 ms}. + */ + private long lockTimeout = DFLT_LOCK_TIMEOUT; + + /** + * CDC application periodically scans {@link DataStorageConfiguration#getChangeDataCaptureWalPath()} folder to find new WAL segments. + * This frequency specify amount of time application sleeps between subsequent checks when no new files available. + * Default is {@code 1000 ms}. + */ + private long checkFreq = DFLT_CHECK_FREQ; + + /** @return CDC consumer. */ + public ChangeDataCaptureConsumer getConsumer() { + return consumer; + } + + /** @param consumer CDC consumer. */ + public void setConsumer(ChangeDataCaptureConsumer consumer) { + this.consumer = consumer; + } + + /** @return keep binary value. */ + public boolean isKeepBinary() { + return keepBinary; + } + + /** @param keepBinary keep binary value. */ + public void setKeepBinary(boolean keepBinary) { + this.keepBinary = keepBinary; + } + + /** @return Amount of time to wait for lock acquisition. */ + public long getLockTimeout() { + return lockTimeout; + } + + /** @param lockTimeout Amount of time to wait for lock acquisition. */ + public void setLockTimeout(long lockTimeout) { + this.lockTimeout = lockTimeout; + } + + /** @return Amount of time application sleeps between subsequent checks when no new files available. */ + public long getCheckFrequency() { + return checkFreq; + } + + /** + * @param checkFreq Amount of time application sleeps between subsequent checks when no new + * files available. + */ + public void setCheckFrequency(long checkFreq) { + this.checkFreq = checkFreq; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConsumer.java b/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConsumer.java new file mode 100644 index 0000000000000..aa58002046e29 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConsumer.java @@ -0,0 +1,77 @@ +/* + * 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.cdc; + +import java.util.Iterator; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.CacheEntryVersion; +import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.lang.IgniteExperimental; +import org.apache.ignite.resources.LoggerResource; + +/** + * Consumer of WAL data change events. + * This consumer will receive data change events during {@link ChangeDataCapture} application invocation. + * The lifecycle of the consumer is the following: + *
    + *
  • Start of the consumer {@link #start()}.
  • + *
  • Notification of the consumer by the {@link #onEvents(Iterator)} call.
  • + *
  • Stop of the consumer {@link #stop()}.
  • + *
+ * + * In case consumer implementation wants to user {@link IgniteLogger}, please, use, {@link LoggerResource} annotation: + *
 {@code
+ * public class ChangeDataCaptureConsumer implements ChangeDataCaptureConsumer {
+ *     @LoggerReource
+ *     private IgniteLogger log;
+ *
+ *     ...
+ * }
+ * }
+ * + * Note, consumption of the {@link ChangeDataCaptureEvent} will be started from the last saved offset. + * The offset of consumptions is saved on the disk every time {@link #onEvents(Iterator)} returns {@code true}. + * + * @see ChangeDataCapture + * @see ChangeDataCaptureEvent + * @see CacheEntryVersion + */ +@IgniteExperimental +public interface ChangeDataCaptureConsumer { + /** + * Starts the consumer. + */ + public void start(); + + /** + * Handles entry changes events. + * If this method return {@code true} then current offset will be stored + * and ongoing notifications after CDC application fail/restart will be started from it. + * + * @param events Entry change events. + * @return {@code True} if current offset should be saved on the disk + * to continue from it in case any failures or restart. + */ + public boolean onEvents(Iterator events); + + /** + * Stops the consumer. + * This methods can be invoked only after {@link #start()}. + */ + public void stop(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureEvent.java b/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureEvent.java new file mode 100644 index 0000000000000..90540c160aa5c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureEvent.java @@ -0,0 +1,79 @@ +/* + * 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.cdc; + +import java.io.Serializable; +import org.apache.ignite.cache.CacheEntryVersion; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.lang.IgniteExperimental; +import org.apache.ignite.spi.systemview.view.CacheView; +import org.jetbrains.annotations.Nullable; + +/** + * Event of single entry change. + * Instance presents new value of modified entry. + * + * @see ChangeDataCapture + * @see ChangeDataCaptureConsumer + */ +@IgniteExperimental +public interface ChangeDataCaptureEvent extends Serializable { + /** + * @return Key for the changed entry. + */ + public Object key(); + + /** + * @return Value for the changed entry or {@code null} in case of entry removal. + */ + @Nullable public Object value(); + + /** + * @return {@code True} if event fired on primary node for partition containing this entry. + * @see + * Configuring partition backups. + */ + public boolean primary(); + + /** + * Ignite split dataset into smaller chunks to distribute them across the cluster. + * {@link ChangeDataCaptureConsumer} implementations can use {@link #partition()} to split changes processing + * in the same way as it done for the cache. + * + * @return Partition number. + * @see Affinity#partition(Object) + * @see Affinity#partitions() + * @see Data partitioning + * @see Affinity collocation + */ + public int partition(); + + /** + * @return Version of the entry. + */ + public CacheEntryVersion version(); + + /** + * @return Cache ID. + * @see org.apache.ignite.internal.util.typedef.internal.CU#cacheId(String) + * @see CacheView#cacheId() + */ + public int cacheId(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureLoader.java b/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureLoader.java new file mode 100644 index 0000000000000..17b95e7be00ac --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureLoader.java @@ -0,0 +1,74 @@ +/* + * 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.cdc; + +import java.net.URL; +import java.util.Collection; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.processors.resource.GridSpringResourceContext; +import org.apache.ignite.internal.util.spring.IgniteSpringHelper; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; + +import static org.apache.ignite.internal.IgniteComponentType.SPRING; + +/** + * Utility class to load {@link ChangeDataCapture} from Spring XML configuration. + */ +public class ChangeDataCaptureLoader { + /** + * Loads {@link ChangeDataCapture} from XML configuration file and possible error message. + * If load fails then error message wouldn't be null. + * + * @param springXmlPath Path to XML configuration file. + * @return Tuple of {@code ChangeDataCapture} and error message. + * @throws IgniteCheckedException If failed. + */ + public static ChangeDataCapture loadChangeDataCapture(String springXmlPath) throws IgniteCheckedException { + URL cfgUrl = U.resolveSpringUrl(springXmlPath); + + IgniteSpringHelper spring = SPRING.create(false); + + IgniteBiTuple, ? extends GridSpringResourceContext> cfgTuple = + spring.loadConfigurations(cfgUrl); + + if (cfgTuple.get1().size() > 1) { + throw new IgniteCheckedException( + "Exact 1 IgniteConfiguration should be defined. Found " + cfgTuple.get1().size() + ); + } + + IgniteBiTuple, ? extends GridSpringResourceContext> cdcCfgs = + spring.loadConfigurations(cfgUrl, ChangeDataCaptureConfiguration.class); + + if (cdcCfgs.get1().size() > 1) { + throw new IgniteCheckedException( + "Exact 1 CaptureDataChangeConfiguration configuration should be defined. " + + "Found " + cdcCfgs.get1().size() + ); + } + + return new ChangeDataCapture( + cfgTuple.get1().iterator().next(), + cfgTuple.get2(), + cdcCfgs.get1().iterator().next() + ); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java index 4574febe54be3..3fd33d5b23b5a 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java @@ -165,6 +165,9 @@ public class DataStorageConfiguration implements Serializable { /** Default wal archive directory. */ public static final String DFLT_WAL_ARCHIVE_PATH = "db/wal/archive"; + /** Default change data capture directory. */ + public static final String DFLT_WAL_CDC_PATH = "db/wal/cdc"; + /** Default path (relative to working directory) of binary metadata folder */ public static final String DFLT_BINARY_METADATA_PATH = "db/binary_meta"; @@ -242,6 +245,14 @@ public class DataStorageConfiguration implements Serializable { /** WAL archive path. */ private String walArchivePath = DFLT_WAL_ARCHIVE_PATH; + /** Change Data Capture path. */ + @IgniteExperimental + private String changeDataCaptureWalPath = DFLT_WAL_CDC_PATH; + + /** Change Data Capture enabled flag. */ + @IgniteExperimental + private boolean changeDataCaptureEnabled; + /** Metrics enabled flag. */ private boolean metricsEnabled = DFLT_METRICS_ENABLED; @@ -727,6 +738,54 @@ public DataStorageConfiguration setWalArchivePath(String walArchivePath) { return this; } + /** + * Gets a path to the CDC directory. + * If this path is relative, it will be resolved relatively to Ignite work directory. + * + * @return CDC directory. + */ + @IgniteExperimental + public String getChangeDataCaptureWalPath() { + return changeDataCaptureWalPath; + } + + /** + * Sets a path for the CDC directory. + * Hard link to every WAL Archive segment will be created in it for CDC processing purpose. + * + * @param changeDataCaptureWalPath CDC directory. + * @return {@code this} for chaining. + */ + @IgniteExperimental + public DataStorageConfiguration setChangeDataCaptureWalPath(String changeDataCaptureWalPath) { + this.changeDataCaptureWalPath = changeDataCaptureWalPath; + + return this; + } + + /** + * Sets flag indicating whether CDC enabled. + * + * @param changeDataCaptureEnabled CDC enabled flag. + */ + @IgniteExperimental + public DataStorageConfiguration setChangeDataCaptureEnabled(boolean changeDataCaptureEnabled) { + this.changeDataCaptureEnabled = changeDataCaptureEnabled; + + return this; + } + + /** + * Gets flag indicating whether CDC is enabled. + * Default value is {@code false}. + * + * @return Metrics enabled flag. + */ + @IgniteExperimental + public boolean isChangeDataCaptureEnabled() { + return changeDataCaptureEnabled; + } + /** * Gets flag indicating whether persistence metrics collection is enabled. * Default value is {@link #DFLT_METRICS_ENABLED}. 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 a370b4f0622df..e206ce7232c3a 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 @@ -347,7 +347,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { public static final String SITE = "ignite.apache.org"; /** System line separator. */ - private static final String NL = U.nl(); + public static final String NL = U.nl(); /** System megabyte. */ private static final int MEGABYTE = 1024 * 1024; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index 9aff1acf29344..a9a6e95e894c8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -22,8 +22,6 @@ import java.io.InputStream; import java.lang.Thread.UncaughtExceptionHandler; import java.lang.management.ManagementFactory; -import java.lang.reflect.Constructor; -import java.net.MalformedURLException; import java.net.URL; import java.util.ArrayList; import java.util.Collection; @@ -109,9 +107,6 @@ import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteInClosure; -import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware; -import org.apache.ignite.logger.LoggerNodeIdAware; -import org.apache.ignite.logger.java.JavaLogger; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.marshaller.jdk.JdkMarshaller; @@ -2302,7 +2297,7 @@ private IgniteConfiguration initializeConfiguration(IgniteConfiguration cfg) if (!F.isEmpty(predefineConsistentId)) myCfg.setConsistentId(predefineConsistentId); - IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), null, nodeId, workDir); + IgniteLogger cfgLog = U.initLogger(cfg.getGridLogger(), null, nodeId, workDir); assert cfgLog != null; @@ -2558,94 +2553,6 @@ private void initializeDefaultSpi(IgniteConfiguration cfg) { cfg.setTracingSpi(new NoopTracingSpi()); } - /** - * @param cfgLog Configured logger. - * @param nodeId Local node ID. - * @param workDir Work directory. - * @return Initialized logger. - * @throws IgniteCheckedException If failed. - */ - @SuppressWarnings("ErrorNotRethrown") - private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, @Nullable String app, UUID nodeId, String workDir) - throws IgniteCheckedException { - try { - Exception log4jInitErr = null; - - if (cfgLog == null) { - Class log4jCls; - - try { - log4jCls = Class.forName("org.apache.ignite.logger.log4j.Log4JLogger"); - } - catch (ClassNotFoundException | NoClassDefFoundError ignored) { - log4jCls = null; - } - - if (log4jCls != null) { - try { - URL url = U.resolveIgniteUrl("config/ignite-log4j.xml"); - - if (url == null) { - File cfgFile = new File("config/ignite-log4j.xml"); - - if (!cfgFile.exists()) - cfgFile = new File("../config/ignite-log4j.xml"); - - if (cfgFile.exists()) { - try { - url = cfgFile.toURI().toURL(); - } - catch (MalformedURLException ignore) { - // No-op. - } - } - } - - if (url != null) { - boolean configured = (Boolean)log4jCls.getMethod("isConfigured").invoke(null); - - if (configured) - url = null; - } - - if (url != null) { - Constructor ctor = log4jCls.getConstructor(URL.class); - - cfgLog = (IgniteLogger)ctor.newInstance(url); - } - else - cfgLog = (IgniteLogger)log4jCls.newInstance(); - } - catch (Exception e) { - log4jInitErr = e; - } - } - - if (log4jCls == null || log4jInitErr != null) - cfgLog = new JavaLogger(); - } - - // Special handling for Java logger which requires work directory. - if (cfgLog instanceof JavaLogger) - ((JavaLogger)cfgLog).setWorkDirectory(workDir); - - // Set node IDs for all file appenders. - if (cfgLog instanceof LoggerNodeIdAndApplicationAware) - ((LoggerNodeIdAndApplicationAware)cfgLog).setApplicationAndNode(app, nodeId); - else if (cfgLog instanceof LoggerNodeIdAware) - ((LoggerNodeIdAware)cfgLog).setNodeId(nodeId); - - if (log4jInitErr != null) - U.warn(cfgLog, "Failed to initialize Log4JLogger (falling back to standard java logging): " - + log4jInitErr.getCause()); - - return cfgLog; - } - catch (Exception e) { - throw new IgniteCheckedException("Failed to create logger.", e); - } - } - /** * Creates utility system cache configuration. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCapture.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCapture.java new file mode 100644 index 0000000000000..5fb3e183c0e24 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCapture.java @@ -0,0 +1,544 @@ +/* + * 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.cdc; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Stream; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cdc.ChangeDataCaptureConfiguration; +import org.apache.ignite.cdc.ChangeDataCaptureConsumer; +import org.apache.ignite.cdc.ChangeDataCaptureEvent; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.MarshallerContextImpl; +import org.apache.ignite.internal.pagemem.wal.WALIterator; +import org.apache.ignite.internal.pagemem.wal.record.DataRecord; +import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; +import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver; +import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory; +import org.apache.ignite.internal.processors.resource.GridResourceIoc; +import org.apache.ignite.internal.processors.resource.GridResourceLoggerInjector; +import org.apache.ignite.internal.processors.resource.GridSpringResourceContext; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.resources.SpringApplicationContextResource; +import org.apache.ignite.resources.SpringResource; +import org.apache.ignite.startup.cmdline.ChangeDataCaptureCommandLineStartup; + +import static org.apache.ignite.internal.IgniteKernal.NL; +import static org.apache.ignite.internal.IgniteKernal.SITE; +import static org.apache.ignite.internal.IgniteVersionUtils.ACK_VER_STR; +import static org.apache.ignite.internal.IgniteVersionUtils.COPYRIGHT; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD_V2; +import static org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER; + +/** + * Change Data Capture (CDC) application. + * The application runs independently of Ignite node process and provides the ability + * for the {@link ChangeDataCaptureConsumer} to consume events({@link ChangeDataCaptureEvent}) from WAL segments. + * The user should provide {@link ChangeDataCaptureConsumer} implementation with custom consumption logic. + * + * Ignite node should be explicitly configured for using {@link ChangeDataCapture}. + *
    + *
  1. Set {@link DataStorageConfiguration#setChangeDataCaptureEnabled(boolean)} to true.
  2. + *
  3. Optional: Set {@link DataStorageConfiguration#setChangeDataCaptureWalPath(String)} to path to the directory + * to store WAL segments for CDC.
  4. + *
  5. Optional: Set {@link DataStorageConfiguration#setWalForceArchiveTimeout(long)} to configure timeout for + * force WAL rollover, so new events will be available for consumptions with the predicted time.
  6. + *
+ * + * When {@link DataStorageConfiguration#getChangeDataCaptureWalPath()} is true then Ignite node on each WAL segment + * rollover creates hard link to archive WAL segment in + * {@link DataStorageConfiguration#getChangeDataCaptureWalPath()} directory. {@link ChangeDataCapture} application takes + * segment file and consumes events from it. + * After successful consumption (see {@link ChangeDataCaptureConsumer#onEvents(Iterator)}) WAL segment will be deleted + * from directory. + * + * Several Ignite nodes can be started on the same host. + * If your deployment done with custom consistent id then you should specify it via + * {@link IgniteConfiguration#setConsistentId(Serializable)} in provided {@link IgniteConfiguration}. + * + * Application works as follows: + *
    + *
  1. Searches node work directory based on provided {@link IgniteConfiguration}.
  2. + *
  3. Awaits for the creation of CDC directory if it not exists.
  4. + *
  5. Acquires file lock to ensure exclusive consumption.
  6. + *
  7. Loads state of consumption if it exists.
  8. + *
  9. Infinitely waits for new available segment and processes it.
  10. + *
+ * + * @see DataStorageConfiguration#setChangeDataCaptureEnabled(boolean) + * @see DataStorageConfiguration#setChangeDataCaptureWalPath(String) + * @see DataStorageConfiguration#setWalForceArchiveTimeout(long) + * @see ChangeDataCaptureCommandLineStartup + * @see ChangeDataCaptureConsumer + * @see DataStorageConfiguration#DFLT_WAL_CDC_PATH + */ +public class ChangeDataCapture implements Runnable { + /** */ + public static final String ERR_MSG = "Persistence disabled. Capture Data Change can't run!"; + + /** State dir. */ + public static final String STATE_DIR = "state"; + + /** Ignite configuration. */ + private final IgniteConfiguration igniteCfg; + + /** Spring resource context. */ + private final GridSpringResourceContext ctx; + + /** Change Data Capture configuration. */ + private final ChangeDataCaptureConfiguration cdcCfg; + + /** WAL iterator factory. */ + private final IgniteWalIteratorFactory factory; + + /** Events consumer. */ + private final WalRecordsConsumer consumer; + + /** Logger. */ + private final IgniteLogger log; + + /** Change Data Capture directory. */ + private Path cdcDir; + + /** Binary meta directory. */ + private File binaryMeta; + + /** Marshaller directory. */ + private File marshaller; + + /** Change Data Capture state. */ + private ChangeDataCaptureConsumerState state; + + /** Save state to start from. */ + private WALPointer initState; + + /** Stopped flag. */ + private volatile boolean stopped; + + /** Already processed segments. */ + private final Set processedSegments = new HashSet<>(); + + /** + * @param igniteCfg Ignite configuration. + * @param ctx Spring resource context. + * @param cdcCfg Change Data Capture configuration. + */ + public ChangeDataCapture( + IgniteConfiguration igniteCfg, + GridSpringResourceContext ctx, + ChangeDataCaptureConfiguration cdcCfg) { + this.igniteCfg = new IgniteConfiguration(igniteCfg); + this.ctx = ctx; + this.cdcCfg = cdcCfg; + + try { + initWorkDir(this.igniteCfg); + + log = U.initLogger( + igniteCfg.getGridLogger(), + "ignite-cdc", + igniteCfg.getNodeId() != null ? igniteCfg.getNodeId() : UUID.randomUUID(), + igniteCfg.getWorkDirectory() + ); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + + consumer = new WalRecordsConsumer<>(cdcCfg.getConsumer(), log); + + factory = new IgniteWalIteratorFactory(log); + } + + /** Runs Change Data Capture. */ + @Override public void run() { + synchronized (this) { + if (stopped) + return; + } + + try { + runX(); + } + catch (Throwable e) { + e.printStackTrace(); + + throw new IgniteException(e); + } + } + + /** Runs Change Data Capture application with possible exception. */ + public void runX() throws Exception { + if (!CU.isPersistenceEnabled(igniteCfg)) { + log.error(ERR_MSG); + + throw new IllegalArgumentException(ERR_MSG); + } + + PdsFolderSettings settings = + new PdsFolderResolver<>(igniteCfg, log, null, this::tryLock).resolve(); + + if (settings == null) { + throw new IgniteException("Can't find folder to read WAL segments from based on provided configuration! " + + "[workDir=" + igniteCfg.getWorkDirectory() + ", consistentId=" + igniteCfg.getConsistentId() + ']'); + } + + ChangeDataCaptureFileLockHolder lock = settings.getLockedFileLockHolder(); + + if (lock == null) { + File consIdDir = new File(settings.persistentStoreRootPath(), settings.folderName()); + + lock = tryLock(consIdDir); + + if (lock == null) { + throw new IgniteException( + "Can't acquire lock for Change Data Capture folder [dir=" + consIdDir.getAbsolutePath() + ']' + ); + } + } + + try { + String consIdDir = cdcDir.getName(cdcDir.getNameCount() - 1).toString(); + + Files.createDirectories(cdcDir.resolve(STATE_DIR)); + + binaryMeta = CacheObjectBinaryProcessorImpl.binaryWorkDir(igniteCfg.getWorkDirectory(), consIdDir); + + marshaller = MarshallerContextImpl.mappingFileStoreWorkDir(igniteCfg.getWorkDirectory()); + + injectResources(consumer.consumer()); + + ackAsciiLogo(); + + state = new ChangeDataCaptureConsumerState(cdcDir.resolve(STATE_DIR)); + + initState = state.load(); + + if (initState != null && log.isInfoEnabled()) + log.info("Initial state loaded [state=" + initState + ']'); + + consumer.start(); + + try { + consumeWalSegmentsUntilStopped(); + } + finally { + consumer.stop(); + + if (log.isInfoEnabled()) + log.info("Ignite Change Data Capture Application stopped."); + } + } + finally { + U.closeQuiet(lock); + } + } + + /** Waits and consumes new WAL segments until stopped. */ + public void consumeWalSegmentsUntilStopped() { + try { + Set seen = new HashSet<>(); + + AtomicLong lastSgmnt = new AtomicLong(-1); + + while (!stopped) { + try (Stream cdcFiles = Files.walk(cdcDir, 1)) { + Set exists = new HashSet<>(); + + cdcFiles + .peek(exists::add) // Store files that exists in cdc dir. + // Need unseen WAL segments only. + .filter(p -> WAL_SEGMENT_FILE_FILTER.accept(p.toFile()) && !seen.contains(p)) + .peek(seen::add) // Adds to seen. + .sorted(Comparator.comparingLong(this::segmentIndex)) // Sort by segment index. + .peek(p -> { + long nextSgmnt = segmentIndex(p); + + assert lastSgmnt.get() == -1 || nextSgmnt - lastSgmnt.get() == 1; + + lastSgmnt.set(nextSgmnt); + }) + .forEach(this::consumeSegment); // Consuming segments. + + seen.removeIf(p -> !exists.contains(p)); // Clean up seen set. + } + + if (!stopped) + U.sleep(cdcCfg.getCheckFrequency()); + } + } + catch (IOException | IgniteInterruptedCheckedException e) { + throw new IgniteException(e); + } + } + + /** Reads all available records from segment. */ + private void consumeSegment(Path segment) { + if (log.isInfoEnabled()) + log.info("Processing WAL segment [segment=" + segment + ']'); + + IgniteWalIteratorFactory.IteratorParametersBuilder builder = + new IgniteWalIteratorFactory.IteratorParametersBuilder() + .log(log) + .binaryMetadataFileStoreDir(binaryMeta) + .marshallerMappingFileStoreDir(marshaller) + .keepBinary(cdcCfg.isKeepBinary()) + .filesOrDirs(segment.toFile()) + .addFilter((type, ptr) -> type == DATA_RECORD_V2); + + if (initState != null) { + long segmentIdx = segmentIndex(segment); + + if (segmentIdx > initState.index()) { + throw new IgniteException("Found segment greater then saved state. Some events are missed. Exiting! " + + "[state=" + initState + ", segment=" + segmentIdx + ']'); + } + + if (segmentIdx < initState.index()) { + if (log.isInfoEnabled()) { + log.info("Already processed segment found. Skipping and deleting the file [segment=" + + segmentIdx + ", state=" + initState.index() + ']'); + } + + // WAL segment is a hard link to a segment file in the special Change Data Capture folder. + // So, we can safely delete it after processing. + try { + Files.delete(segment); + + return; + } + catch (IOException e) { + throw new IgniteException(e); + } + } + + builder.from(initState); + + initState = null; + } + + try (WALIterator it = factory.iterator(builder)) { + boolean interrupted = Thread.interrupted(); + + while (it.hasNext() && !interrupted) { + Iterator iter = F.iterator(it.iterator(), t -> (DataRecord)t.get2(), true); + + boolean commit = consumer.onRecords(iter); + + if (commit) { + assert it.lastRead().isPresent(); + + state.save(it.lastRead().get()); + + // Can delete after new file state save. + if (!processedSegments.isEmpty()) { + // WAL segment is a hard link to a segment file in a specifal Change Data Capture folder. + // So we can safely delete it after success processing. + for (Path processedSegment : processedSegments) { + // Can't delete current segment, because state points to it. + if (processedSegment.equals(segment)) + continue; + + Files.delete(processedSegment); + } + + processedSegments.clear(); + } + } + + interrupted = Thread.interrupted(); + } + + if (interrupted) + throw new IgniteException("Change Data Capture Application interrupted"); + + processedSegments.add(segment); + } catch (IgniteCheckedException | IOException e) { + throw new IgniteException(e); + } + } + + /** + * Try locks Change Data Capture directory. + * + * @param dbStoreDirWithSubdirectory Root PDS directory. + * @return Lock or null if lock failed. + */ + private ChangeDataCaptureFileLockHolder tryLock(File dbStoreDirWithSubdirectory) { + if (!dbStoreDirWithSubdirectory.exists()) { + log.warning("DB store directory not exists [dir=" + dbStoreDirWithSubdirectory + ']'); + + return null; + } + + File cdcRoot = new File(igniteCfg.getDataStorageConfiguration().getChangeDataCaptureWalPath()); + + if (!cdcRoot.isAbsolute()) { + cdcRoot = new File( + igniteCfg.getWorkDirectory(), + igniteCfg.getDataStorageConfiguration().getChangeDataCaptureWalPath() + ); + } + + if (!cdcRoot.exists()) { + log.warning("CDC root directory not exists. Should be created by Ignite Node. " + + "Is Change Data Capture enabled in IgniteConfiguration? [dir=" + cdcRoot + ']'); + + return null; + } + + Path cdcDir = Paths.get(cdcRoot.getAbsolutePath(), dbStoreDirWithSubdirectory.getName()); + + if (!Files.exists(cdcDir)) { + log.warning("CDC directory not exists. Should be created by Ignite Node. " + + "Is Change Data Capture enabled in IgniteConfiguration? [dir=" + cdcDir + ']'); + + return null; + } + + this.cdcDir = cdcDir; + + ChangeDataCaptureFileLockHolder lock = new ChangeDataCaptureFileLockHolder(cdcDir.toString(), "cdc.lock", log); + + try { + lock.tryLock(cdcCfg.getLockTimeout()); + + return lock; + } + catch (IgniteCheckedException e) { + U.closeQuiet(lock); + + if (log.isInfoEnabled()) { + log.info("Unable to acquire lock to lock CDC folder [dir=" + cdcRoot + "]" + NL + + "Reason: " + e.getMessage()); + } + + return null; + } + } + + /** Resolves work directory. */ + private static void initWorkDir(IgniteConfiguration cfg) throws IgniteCheckedException { + String igniteHome = cfg.getIgniteHome(); + + // Set Ignite home. + if (igniteHome == null) + igniteHome = U.getIgniteHome(); + + String userProvidedWorkDir = cfg.getWorkDirectory(); + + // Correctly resolve work directory and set it back to configuration. + cfg.setWorkDirectory(U.workDirectory(userProvidedWorkDir, igniteHome)); + } + + /** + * @param segment WAL segment file. + * @return Segment index. + */ + public long segmentIndex(Path segment) { + String fn = segment.getFileName().toString(); + + return Long.parseLong(fn.substring(0, fn.indexOf('.'))); + } + + /** Stops the application. */ + public void stop() { + synchronized (this) { + if (log.isInfoEnabled()) + log.info("Stopping Change Data Capture service instance"); + + stopped = true; + } + } + + /** */ + private void injectResources(ChangeDataCaptureConsumer dataConsumer) throws IgniteCheckedException { + GridResourceIoc ioc = new GridResourceIoc(); + + ioc.inject( + dataConsumer, + LoggerResource.class, + new GridResourceLoggerInjector(log), + null, + null + ); + + if (ctx != null) { + ioc.inject( + dataConsumer, + SpringResource.class, + ctx.springBeanInjector(), + null, + null + ); + + ioc.inject( + dataConsumer, + SpringApplicationContextResource.class, + ctx.springContextInjector(), + null, + null + ); + } + } + + /** */ + private void ackAsciiLogo() { + if (!log.isInfoEnabled()) + return; + + String ver = "ver. " + ACK_VER_STR; + + log.info(NL + NL + + ">>> __________ ________________ ________ _____" + NL + + ">>> / _/ ___/ |/ / _/_ __/ __/ / ___/ _ \\/ ___/" + NL + + ">>> _/ // (7 7 // / / / / _/ / /__/ // / /__ " + NL + + ">>> /___/\\___/_/|_/___/ /_/ /___/ \\___/____/\\___/ " + NL + + ">>> " + NL + + ">>> " + ver + NL + + ">>> " + COPYRIGHT + NL + + ">>> " + NL + + ">>> Ignite documentation: " + "http://" + SITE + NL + + ">>> Consumer: " + consumer.consumer().toString() + NL + + ">>> ConsistentId: " + igniteCfg.getConsistentId() + NL + + ">>> Change Data Capture: " + cdcDir + NL + + ">>> Ignite node Binary meta: " + binaryMeta + NL + + ">>> Ignite node Marshaller: " + marshaller + NL + ); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureConsumerState.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureConsumerState.java new file mode 100644 index 0000000000000..cc6f67cfac5e1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureConsumerState.java @@ -0,0 +1,114 @@ +/* + * 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.cdc; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.Iterator; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cdc.ChangeDataCaptureConsumer; +import org.apache.ignite.cdc.ChangeDataCaptureEvent; +import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; + +import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; +import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX; +import static org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer.POINTER_SIZE; + +/** + * Change Data Capture Consumer state. + * + * Each time {@link ChangeDataCaptureConsumer#onEvents(Iterator)} returns {@code true} + * current offset in WAL segment saved to file. + * This allows to the {@link ChangeDataCaptureConsumer} to continue consumption of the {@link ChangeDataCaptureEvent} + * from the last saved offset in case of fail or restart. + * + * @see ChangeDataCaptureConsumer#onEvents(Iterator) + * @see ChangeDataCapture + */ +public class ChangeDataCaptureConsumerState { + /** */ + public static final String STATE_FILE_NAME = "cdc-state" + FILE_SUFFIX; + + /** State file. */ + private final Path state; + + /** Temp state file. */ + private final Path tmp; + + /** + * @param stateDir State directory. + */ + public ChangeDataCaptureConsumerState(Path stateDir) { + state = stateDir.resolve(STATE_FILE_NAME); + tmp = stateDir.resolve(STATE_FILE_NAME + TMP_SUFFIX); + } + + /** + * Saves state to file. + * @param ptr WAL pointer. + */ + public void save(WALPointer ptr) throws IOException { + ByteBuffer buf = ByteBuffer.allocate(POINTER_SIZE); + + buf.putLong(ptr.index()); + buf.putInt(ptr.fileOffset()); + buf.putInt(ptr.length()); + buf.flip(); + + try (FileChannel ch = FileChannel.open(tmp, StandardOpenOption.CREATE, StandardOpenOption.WRITE)) { + ch.write(buf); + + ch.force(true); + } + + Files.move(tmp, state, ATOMIC_MOVE, REPLACE_EXISTING); + } + + /** + * Loads CDC state from file. + * @return Saved state. + */ + public WALPointer load() { + if (!Files.exists(state)) + return null; + + try (FileChannel ch = FileChannel.open(state, StandardOpenOption.READ)) { + ByteBuffer buf = ByteBuffer.allocate(POINTER_SIZE); + + ch.read(buf); + + buf.flip(); + + long idx = buf.getLong(); + int offset = buf.getInt(); + int length = buf.getInt(); + + return new WALPointer(idx, offset, length); + } + catch (IOException e) { + throw new IgniteException("Failed to read state [file=" + state + ']', e); + } + + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureEventImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureEventImpl.java new file mode 100644 index 0000000000000..ff4ceec323da6 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureEventImpl.java @@ -0,0 +1,106 @@ +/* + * 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.cdc; + +import org.apache.ignite.cache.CacheEntryVersion; +import org.apache.ignite.cdc.ChangeDataCaptureConsumer; +import org.apache.ignite.cdc.ChangeDataCaptureEvent; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Event of single entry change. + * Instance presents new value of modified entry. + * + * @see ChangeDataCapture + * @see ChangeDataCaptureConsumer + */ +public class ChangeDataCaptureEventImpl implements ChangeDataCaptureEvent { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Key. */ + private final Object key; + + /** Value. */ + private final Object val; + + /** {@code True} if changes made on primary node. */ + private final boolean primary; + + /** Partition. */ + private final int part; + + /** Order of the entry change. */ + private final CacheEntryVersion ord; + + /** Cache id. */ + private final int cacheId; + + /** + * @param key Key. + * @param val Value. + * @param primary {@code True} if changes made on primary node. + * @param part Partition. + * @param ord Order of the entry change. + * @param cacheId Cache id. + */ + public ChangeDataCaptureEventImpl(Object key, Object val, boolean primary, int part, + CacheEntryVersion ord, int cacheId) { + this.key = key; + this.val = val; + this.primary = primary; + this.part = part; + this.ord = ord; + this.cacheId = cacheId; + } + + /** {@inheritDoc} */ + @Override public Object key() { + return key; + } + + /** {@inheritDoc} */ + @Override public Object value() { + return val; + } + + /** {@inheritDoc} */ + @Override public boolean primary() { + return primary; + } + + /** {@inheritDoc} */ + @Override public int partition() { + return part; + } + + /** {@inheritDoc} */ + @Override public CacheEntryVersion version() { + return ord; + } + + /** {@inheritDoc} */ + @Override public int cacheId() { + return cacheId; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ChangeDataCaptureEventImpl.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureFileLockHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureFileLockHolder.java new file mode 100644 index 0000000000000..5d91b145def66 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureFileLockHolder.java @@ -0,0 +1,55 @@ +/* + * 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.cdc; + +import java.lang.management.ManagementFactory; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.cache.persistence.FileLockHolder; + +/** + * Lock file holder for Change Data Capture application. + * + * @see ChangeDataCapture + * @see ChangeDataCaptureConsumerState + */ +public class ChangeDataCaptureFileLockHolder extends FileLockHolder { + /** Consumer ID. */ + private final String consumerId; + + /** + * @param rootDir Root directory for lock file. + * @param log Log. + */ + public ChangeDataCaptureFileLockHolder(String rootDir, String consumerId, IgniteLogger log) { + super(rootDir, log); + + this.consumerId = consumerId; + } + + /** {@inheritDoc} */ + @Override public String lockInfo() { + return "[consumerId=" + consumerId + ", proc=" + ManagementFactory.getRuntimeMXBean().getName() + ']'; + } + + /** {@inheritDoc} */ + @Override protected String warningMessage(String lockInfo) { + return "Failed to acquire file lock. Will try again in 1s " + + "[proc=" + ManagementFactory.getRuntimeMXBean().getName() + ", holder=" + lockInfo + + ", path=" + lockPath() + ']'; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java new file mode 100644 index 0000000000000..1b28f2b1b0fbb --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java @@ -0,0 +1,174 @@ +/* + * 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.cdc; + +import java.util.EnumSet; +import java.util.Iterator; +import java.util.NoSuchElementException; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cdc.ChangeDataCaptureConsumer; +import org.apache.ignite.cdc.ChangeDataCaptureEvent; +import org.apache.ignite.internal.pagemem.wal.record.DataEntry; +import org.apache.ignite.internal.pagemem.wal.record.DataRecord; +import org.apache.ignite.internal.pagemem.wal.record.UnwrappedDataEntry; +import org.apache.ignite.internal.processors.cache.GridCacheOperation; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgnitePredicate; + +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.CREATE; +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE; +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE; + +/** + * Transform {@link DataEntry} to {@link ChangeDataCaptureEvent} and sends it to {@link ChangeDataCaptureConsumer}. + * + * @see ChangeDataCapture + * @see ChangeDataCaptureConsumer + */ +public class WalRecordsConsumer { + /** Ignite logger. */ + private final IgniteLogger log; + + /** Data change events consumer. */ + private final ChangeDataCaptureConsumer consumer; + + /** Operations types we interested in. */ + private static final EnumSet OPERATIONS_TYPES = EnumSet.of(CREATE, UPDATE, DELETE, TRANSFORM); + + /** Operations filter. */ + private static final IgnitePredicate OPERATIONS_FILTER = e -> { + if (!(e instanceof UnwrappedDataEntry)) + throw new IllegalStateException("Unexpected data entry [type=" + e.getClass().getName() + ']'); + + if ((e.flags() & DataEntry.PRELOAD_FLAG) != 0 || + (e.flags() & DataEntry.FROM_STORE_FLAG) != 0) + return false; + + return OPERATIONS_TYPES.contains(e.op()); + }; + + /** + * @param consumer User provided CDC consumer. + * @param log Logger. + */ + public WalRecordsConsumer(ChangeDataCaptureConsumer consumer, IgniteLogger log) { + this.consumer = consumer; + this.log = log; + } + + /** + * Handles record from the WAL. + * If this method return {@code true} then current offset in WAL will be stored and WAL iteration will be + * started from it on CDC application fail/restart. + * + * @param recs WAL records iterator. + * @return {@code True} if current offset in WAL should be commited. + */ + public boolean onRecords(Iterator recs) { + Iterator evts = new Iterator() { + /** */ + private Iterator entries; + + @Override public boolean hasNext() { + advance(); + + return hasCurrent(); + } + + @Override public ChangeDataCaptureEvent next() { + advance(); + + if (!hasCurrent()) + throw new NoSuchElementException(); + + return entries.next(); + } + + private void advance() { + if (hasCurrent()) + return; + + while (recs.hasNext()) { + entries = + F.iterator(recs.next().writeEntries().iterator(), this::transform, true, OPERATIONS_FILTER); + + if (entries.hasNext()) + break; + + entries = null; + } + } + + private boolean hasCurrent() { + return entries != null && entries.hasNext(); + } + + /** */ + private ChangeDataCaptureEvent transform(DataEntry e) { + UnwrappedDataEntry ue = (UnwrappedDataEntry)e; + + return new ChangeDataCaptureEventImpl( + ue.unwrappedKey(), + ue.unwrappedValue(), + (e.flags() & DataEntry.PRIMARY_FLAG) != 0, + e.partitionId(), + e.writeVersion(), + e.cacheId() + ); + } + }; + + return consumer.onEvents(evts); + } + + /** + * Starts the consumer. + * + * @throws IgniteCheckedException If failed. + */ + public void start() throws IgniteCheckedException { + consumer.start(); + + if (log.isDebugEnabled()) + log.debug("WalRecordsConsumer started [consumer=" + consumer.getClass() + ']'); + } + + /** + * Stops the consumer. + * This methods can be invoked only after {@link #start()}. + */ + public void stop() { + consumer.stop(); + + if (log.isInfoEnabled()) + log.info("WalRecordsConsumer stopped [consumer=" + consumer.getClass() + ']'); + } + + /** @return Change Data Capture Consumer. */ + public ChangeDataCaptureConsumer consumer() { + return consumer; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(WalRecordsConsumer.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FileLockHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FileLockHolder.java index b7b3d5cd7f440..2e8a88b45743b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FileLockHolder.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FileLockHolder.java @@ -32,7 +32,7 @@ /** * Abstract file lock holder. - * Implementations should provide {@link #lockId()} that will appear in error message for concurrent processes + * Implementations should provide {@link #lockInfo()} that will appear in error message for concurrent processes * that will try to lock the same file and {@link #warningMessage(String)} to print on each lock try. * * @see GridCacheDatabaseSharedManager.NodeFileLockHolder @@ -71,17 +71,17 @@ protected FileLockHolder(String rootDir, IgniteLogger log) { } /** - * This id will appear in error message of concurrent processes that will try to lock on the same file. + * This info will appear in error message of concurrent processes that will try to lock on the same file. * - * @return Lock ID to store in the file. + * @return Lock info to store in the file. */ - public abstract String lockId(); + public abstract String lockInfo(); /** - * @param lockId Existing lock id. + * @param lockInfo Existing lock info. * @return Warning message. */ - protected abstract String warningMessage(String lockId); + protected abstract String warningMessage(String lockInfo); /** * @param lockWaitTimeMillis During which time thread will try capture file lock. @@ -103,7 +103,7 @@ public void tryLock(long lockWaitTimeMillis) throws IgniteCheckedException { lock = ch.tryLock(0, 1, false); if (lock != null && lock.isValid()) { - writeContent(lockId()); + writeContent(lockInfo()); return; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index 402e3e414b74f..aab2def7220eb 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -3021,7 +3021,7 @@ public NodeFileLockHolder(String rootDir, @NotNull GridKernalContext ctx, Ignite } /** {@inheritDoc} */ - @Override public String lockId() { + @Override public String lockInfo() { SB sb = new SB(); //write node id @@ -3056,9 +3056,9 @@ public NodeFileLockHolder(String rootDir, @NotNull GridKernalContext ctx, Ignite } /** {@inheritDoc} */ - @Override protected String warningMessage(String lockId) { + @Override protected String warningMessage(String lockInfo) { return "Failed to acquire file lock. Will try again in 1s " + - "[nodeId=" + ctx.localNodeId() + ", holder=" + lockId + + "[nodeId=" + ctx.localNodeId() + ", holder=" + lockInfo + ", path=" + lockPath() + ']'; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java index aad5fc0cbf649..ce0a8ae2e78d4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderResolver.java @@ -42,6 +42,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import static java.lang.Boolean.TRUE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID; import static org.apache.ignite.IgniteSystemProperties.getBoolean; @@ -153,7 +154,7 @@ public PdsFolderSettings resolve() throws IgniteCheckedException { if (!CU.isPersistenceEnabled(cfg)) return compatibleResolve(pstStoreBasePath, consistentId); - if (cfg.isClientMode() || cfg.isDaemon()) + if (cfg.isClientMode() == TRUE || cfg.isDaemon()) return new PdsFolderSettings<>(pstStoreBasePath, UUID.randomUUID()); if (getBoolean(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, false)) 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 17e62464f0e5d..c27dbe675c194 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 @@ -273,6 +273,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl /** WAL archive directory (including consistent ID as subfolder). */ private File walArchiveDir; + /** WAL cdc directory (including consistent ID as subfolder) */ + private File walCdcDir; + /** Serializer of latest version, used to read header record and for write records */ private RecordSerializer serializer; @@ -466,6 +469,15 @@ public void setFileIOFactory(FileIOFactory ioFactory) { "write ahead log archive directory" ); + if (dsCfg.isChangeDataCaptureEnabled()) { + walCdcDir = initDirectory( + dsCfg.getChangeDataCaptureWalPath(), + DataStorageConfiguration.DFLT_WAL_CDC_PATH, + resolveFolders.folderName(), + "change data capture directory" + ); + } + serializer = new RecordSerializerFactoryImpl(cctx).createSerializer(serializerVer); GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)cctx.database(); @@ -2024,6 +2036,9 @@ public SegmentArchiveResult archiveSegment(long absIdx) throws StorageException Files.move(dstTmpFile.toPath(), dstFile.toPath()); + if (dsCfg.isChangeDataCaptureEnabled()) + Files.createLink(walCdcDir.toPath().resolve(dstFile.getName()), dstFile.toPath()); + if (mode != WALMode.NONE) { try (FileIO f0 = ioFactory.create(dstFile, CREATE, READ, WRITE)) { f0.force(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java index 4f4f9d0b1e713..1d5b3456cedb1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java @@ -23,6 +23,7 @@ import java.io.ObjectOutput; import java.nio.ByteBuffer; import java.util.UUID; +import org.apache.ignite.cache.CacheEntryVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageReader; @@ -31,7 +32,7 @@ /** * Grid unique version. */ -public class GridCacheVersion implements Message, Comparable, Externalizable { +public class GridCacheVersion implements Message, Externalizable, CacheEntryVersion { /** */ private static final long serialVersionUID = 0L; @@ -92,10 +93,8 @@ public GridCacheVersion(int topVer, int nodeOrderDrId, long order) { this.order = order; } - /** - * @return Topology version plus number of seconds from the start time of the first grid node.. - */ - public int topologyVersion() { + /** {@inheritDoc} */ + @Override public int topologyVersion() { return topVer; } @@ -111,17 +110,25 @@ public int nodeOrderAndDrIdRaw() { /** * @return Version order. */ - public long order() { + @Override public long order() { return order; } - /** - * @return Node order on which this version was assigned. - */ - public int nodeOrder() { + /** {@inheritDoc} */ + @Override public CacheEntryVersion otherClusterVersion() { + return conflictVersion(); + } + + /** {@inheritDoc} */ + @Override public int nodeOrder() { return nodeOrderDrId & NODE_ORDER_MASK; } + /** {@inheritDoc} */ + @Override public byte clusterId() { + return dataCenterId(); + } + /** * @return DR mask. */ @@ -219,13 +226,13 @@ public IgniteUuid asIgniteUuid() { } /** {@inheritDoc} */ - @Override public int compareTo(GridCacheVersion other) { + @Override public int compareTo(CacheEntryVersion other) { int res = Integer.compare(topologyVersion(), other.topologyVersion()); if (res != 0) return res; - res = Long.compare(order, other.order); + res = Long.compare(order(), other.order()); if (res != 0) return res; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java index e389662dc7449..fb6716060a3a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java @@ -136,6 +136,11 @@ public void dataCenterId(byte dataCenterId) { startVer = new GridCacheVersion(0, 0, 0, dataCenterId); } + /** @return Data center ID. */ + public byte dataCenterId() { + return dataCenterId; + } + /** * @param nodeId Node ID. * @param ver Remote version. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java index 4b43e0414548c..9bc47fbb4307d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java @@ -106,7 +106,7 @@ void undeployAll() { * @return {@code True} if resource was injected. * @throws IgniteCheckedException Thrown in case of any errors during injection. */ - boolean inject(Object target, + public boolean inject(Object target, Class annCls, GridResourceInjector injector, @Nullable GridDeployment dep, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 3ddea5d081b3f..3fde13a1d1711 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -256,6 +256,9 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.lifecycle.LifecycleAware; +import org.apache.ignite.logger.LoggerNodeIdAndApplicationAware; +import org.apache.ignite.logger.LoggerNodeIdAware; +import org.apache.ignite.logger.java.JavaLogger; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.PluginProvider; import org.apache.ignite.plugin.extensions.communication.Message; @@ -4636,6 +4639,98 @@ public static void log(@Nullable IgniteLogger log, Object longMsg, Object shortM quiet(false, shortMsg); } + /** + * @param cfgLog Configured logger. + * @param nodeId Local node ID. + * @param workDir Work directory. + * @return Initialized logger. + * @throws IgniteCheckedException If failed. + */ + @SuppressWarnings("ErrorNotRethrown") + public static IgniteLogger initLogger( + @Nullable IgniteLogger cfgLog, + @Nullable String app, + UUID nodeId, + String workDir + ) throws IgniteCheckedException { + try { + Exception log4jInitErr = null; + + if (cfgLog == null) { + Class log4jCls; + + try { + log4jCls = Class.forName("org.apache.ignite.logger.log4j.Log4JLogger"); + } + catch (ClassNotFoundException | NoClassDefFoundError ignored) { + log4jCls = null; + } + + if (log4jCls != null) { + try { + URL url = U.resolveIgniteUrl("config/ignite-log4j.xml"); + + if (url == null) { + File cfgFile = new File("config/ignite-log4j.xml"); + + if (!cfgFile.exists()) + cfgFile = new File("../config/ignite-log4j.xml"); + + if (cfgFile.exists()) { + try { + url = cfgFile.toURI().toURL(); + } + catch (MalformedURLException ignore) { + // No-op. + } + } + } + + if (url != null) { + boolean configured = (Boolean)log4jCls.getMethod("isConfigured").invoke(null); + + if (configured) + url = null; + } + + if (url != null) { + Constructor ctor = log4jCls.getConstructor(URL.class); + + cfgLog = (IgniteLogger)ctor.newInstance(url); + } + else + cfgLog = (IgniteLogger)log4jCls.newInstance(); + } + catch (Exception e) { + log4jInitErr = e; + } + } + + if (log4jCls == null || log4jInitErr != null) + cfgLog = new JavaLogger(); + } + + // Special handling for Java logger which requires work directory. + if (cfgLog instanceof JavaLogger) + ((JavaLogger)cfgLog).setWorkDirectory(workDir); + + // Set node IDs for all file appenders. + if (cfgLog instanceof LoggerNodeIdAndApplicationAware) + ((LoggerNodeIdAndApplicationAware)cfgLog).setApplicationAndNode(app, nodeId); + else if (cfgLog instanceof LoggerNodeIdAware) + ((LoggerNodeIdAware)cfgLog).setNodeId(nodeId); + + if (log4jInitErr != null) + U.warn(cfgLog, "Failed to initialize Log4JLogger (falling back to standard java logging): " + + log4jInitErr.getCause()); + + return cfgLog; + } + catch (Exception e) { + throw new IgniteCheckedException("Failed to create logger.", e); + } + } + /** * Depending on whether or not log is provided and quiet mode is enabled logs given * messages as quiet message or normal log INF0 message. diff --git a/modules/core/src/main/java/org/apache/ignite/startup/cmdline/ChangeDataCaptureCommandLineStartup.java b/modules/core/src/main/java/org/apache/ignite/startup/cmdline/ChangeDataCaptureCommandLineStartup.java new file mode 100644 index 0000000000000..8eb5b565cb6b3 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/startup/cmdline/ChangeDataCaptureCommandLineStartup.java @@ -0,0 +1,157 @@ +/* + * 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.startup.cmdline; + +import java.net.URL; +import java.util.Map; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cdc.ChangeDataCaptureConfiguration; +import org.apache.ignite.cdc.ChangeDataCaptureLoader; +import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.util.spring.IgniteSpringHelper; +import org.apache.ignite.internal.util.typedef.X; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_NO_SHUTDOWN_HOOK; +import static org.apache.ignite.internal.IgniteKernal.NL; +import static org.apache.ignite.internal.IgniteVersionUtils.ACK_VER_STR; +import static org.apache.ignite.internal.IgniteVersionUtils.COPYRIGHT; +import static org.apache.ignite.startup.cmdline.CommandLineStartup.isHelp; + +/** + * This class defines command-line Ignite Capture Data Change startup. This startup can be used to start Ignite + * Capture Data Change application outside of any hosting environment from command line. + * This startup is a Java application with {@link #main(String[])} method that accepts command line arguments. + * It accepts just one parameter which is Spring XML configuration file path. + * You can run this class from command line without parameters to get help message. + *

+ * Note that scripts {@code ${IGNITE_HOME}/bin/cdc.{sh|bat}} shipped with Ignite use + * this startup and you can use them as an example. + *

+ * + * @see ChangeDataCapture + */ +public class ChangeDataCaptureCommandLineStartup { + /** Quite log flag. */ + private static final boolean QUITE = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_QUIET); + + /** + * Main entry point. + * + * @param args Command line arguments. + */ + public static void main(String[] args) { + if (!QUITE) { + X.println(" __________ ________________ ________ _____" + NL + + " / _/ ___/ |/ / _/_ __/ __/ / ___/ _ \\/ ___/" + NL + + " _/ // (7 7 // / / / / _/ / /__/ // / /__ " + NL + + " /___/\\___/_/|_/___/ /_/ /___/ \\___/____/\\___/"); + X.println("Ignite Change Data Capture Command Line Startup, ver. " + ACK_VER_STR); + X.println(COPYRIGHT); + X.println(); + } + + if (args.length > 1) + exit("Too many arguments.", true, -1); + + if (args.length > 0 && isHelp(args[0])) + exit(null, true, 0); + + if (args.length > 0 && args[0].isEmpty()) + exit("Empty argument.", true, 1); + + if (args.length > 0 && args[0].charAt(0) == '-') + exit("Invalid arguments: " + args[0], true, -1); + + try { + ChangeDataCapture cdc = ChangeDataCaptureLoader.loadChangeDataCapture(args[0]); + + if (!IgniteSystemProperties.getBoolean(IGNITE_NO_SHUTDOWN_HOOK, false)) { + Runtime.getRuntime().addShutdownHook(new Thread("cdc-shutdown-hook") { + @Override public void run() { + cdc.stop(); + } + }); + } + + Thread appThread = new Thread(cdc); + + appThread.start(); + + appThread.join(); + } + catch (Throwable e) { + e.printStackTrace(); + + String note = ""; + + if (X.hasCause(e, ClassNotFoundException.class)) + note = "\nNote! You may use 'USER_LIBS' environment variable to specify your classpath."; + + exit("Failed to run CDC: " + e.getMessage() + note, false, -1); + } + } + + /** + * @param cfgUrl String configuration URL. + * @param spring Ignite spring helper. + * @return CDC consumer defined in spring configuration. + * @throws IgniteCheckedException in case of load error. + */ + private static ChangeDataCaptureConfiguration consumerConfig( + URL cfgUrl, + IgniteSpringHelper spring + ) throws IgniteCheckedException { + Map, Object> cdcCfgs = spring.loadBeans(cfgUrl, ChangeDataCaptureConfiguration.class); + + if (cdcCfgs == null || cdcCfgs.size() != 1) + exit("Exact 1 CaptureDataChangeConfiguration configuration should be defined", false, 1); + + return (ChangeDataCaptureConfiguration)cdcCfgs.values().iterator().next(); + } + + /** + * Exists with optional error message, usage show and exit code. + * + * @param errMsg Optional error message. + * @param showUsage Whether or not to show usage information. + * @param exitCode Exit code. + */ + private static void exit(@Nullable String errMsg, boolean showUsage, int exitCode) { + if (errMsg != null) + X.error(errMsg); + + if (showUsage) { + X.error( + "Usage:", + " ignite-cdc.{sh|bat} [?]|[path]", + " Where:", + " ?, /help, -help, - show this message.", + " -v - verbose mode (quiet by default).", + " path - path to Spring XML configuration file.", + " Path can be absolute or relative to IGNITE_HOME.", + " ", + "Spring file should contain bean definition of 'org.apache.ignite.configuration.IgniteConfiguration' " + + "And one or more implementations of 'org.apache.ignite.cdc.CDCConsumer'." + + "Note that bean will be fetched by the type and its ID is not used."); + } + + System.exit(exitCode); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CommandLineStartup.java b/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CommandLineStartup.java index c12365b858e70..460f7c37b9c3f 100644 --- a/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CommandLineStartup.java +++ b/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CommandLineStartup.java @@ -255,7 +255,7 @@ private static void exit(@Nullable String errMsg, boolean showUsage, int exitCod * @param arg Command line argument. * @return {@code true} if given argument is a help argument, {@code false} otherwise. */ - private static boolean isHelp(String arg) { + public static boolean isHelp(String arg) { String s; if (arg.startsWith("--")) diff --git a/modules/core/src/test/java/org/apache/ignite/cdc/AbstractChangeDataCaptureTest.java b/modules/core/src/test/java/org/apache/ignite/cdc/AbstractChangeDataCaptureTest.java new file mode 100644 index 0000000000000..268e0ac2df8f6 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cdc/AbstractChangeDataCaptureTest.java @@ -0,0 +1,250 @@ +/* + * 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.cdc; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.util.typedef.CI3; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.ChangeEventType.DELETE; +import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.ChangeEventType.UPDATE; +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheId; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; + +/** */ +public abstract class AbstractChangeDataCaptureTest extends GridCommonAbstractTest { + /** */ + public static final String JOHN = "John Connor"; + + /** */ + public static final int WAL_ARCHIVE_TIMEOUT = 5_000; + + /** Keys count. */ + public static final int KEYS_CNT = 50; + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + + super.beforeTest(); + } + + /** */ + public void addAndWaitForConsumption( + UserCdcConsumer cnsmr, + ChangeDataCapture cdc, + IgniteCache cache, + IgniteCache txCache, + CI3, Integer, Integer> addData, + int from, + int to, + long timeout + ) throws IgniteCheckedException { + IgniteInternalFuture fut = runAsync(cdc); + + addData.apply(cache, from, to); + + if (txCache != null) + addData.apply(txCache, from, to); + + assertTrue(waitForSize(to - from, cache.getName(), UPDATE, timeout, cnsmr)); + + if (txCache != null) + assertTrue(waitForSize(to - from, txCache.getName(), UPDATE, timeout, cnsmr)); + + fut.cancel(); + + List keys = cnsmr.data(UPDATE, cacheId(cache.getName())); + + assertEquals(to - from, keys.size()); + + for (int i = from; i < to; i++) + assertTrue(Integer.toString(i), keys.contains(i)); + + assertTrue(cnsmr.stopped()); + } + + /** */ + public boolean waitForSize( + int expSz, + String cacheName, + ChangeDataCaptureSelfTest.ChangeEventType evtType, + long timeout, + TestCdcConsumer... cnsmrs + ) throws IgniteInterruptedCheckedException { + return waitForCondition( + () -> { + int sum = Arrays.stream(cnsmrs).mapToInt(c -> F.size(c.data(evtType, cacheId(cacheName)))).sum(); + return sum == expSz; + }, + timeout); + } + + /** */ + public ChangeDataCaptureConfiguration cdcConfig(ChangeDataCaptureConsumer cnsmr) { + ChangeDataCaptureConfiguration cdcCfg = new ChangeDataCaptureConfiguration(); + + cdcCfg.setConsumer(cnsmr); + cdcCfg.setKeepBinary(false); + + return cdcCfg; + } + + /** */ + public abstract static class TestCdcConsumer implements ChangeDataCaptureConsumer { + /** Keys */ + final ConcurrentMap, List> data = new ConcurrentHashMap<>(); + + /** */ + private volatile boolean stopped; + + /** {@inheritDoc} */ + @Override public void start() { + stopped = false; + } + + /** {@inheritDoc} */ + @Override public void stop() { + stopped = true; + } + + /** {@inheritDoc} */ + @Override public boolean onEvents(Iterator evts) { + evts.forEachRemaining(evt -> { + if (!evt.primary()) + return; + + data.computeIfAbsent( + F.t(evt.value() == null ? DELETE : UPDATE, evt.cacheId()), + k -> new ArrayList<>()).add(extract(evt)); + + checkEvent(evt); + }); + + return commit(); + } + + /** */ + public abstract void checkEvent(ChangeDataCaptureEvent evt); + + /** */ + public abstract T extract(ChangeDataCaptureEvent evt); + + /** */ + protected boolean commit() { + return true; + } + + /** @return Read keys. */ + public List data(ChangeEventType op, int cacheId) { + return data.get(F.t(op, cacheId)); + } + + /** */ + public boolean stopped() { + return stopped; + } + } + + /** */ + public static class UserCdcConsumer extends TestCdcConsumer { + /** {@inheritDoc} */ + @Override public void checkEvent(ChangeDataCaptureEvent evt) { + if (evt.value() == null) + return; + + User user = (User)evt.value(); + + assertTrue(user.getName().startsWith(JOHN)); + assertTrue(user.getAge() >= 42); + } + + /** {@inheritDoc} */ + @Override public Integer extract(ChangeDataCaptureEvent evt) { + return (Integer)evt.key(); + } + } + + /** */ + protected static User createUser(int i) { + byte[] bytes = new byte[1024]; + + ThreadLocalRandom.current().nextBytes(bytes); + + return new User(JOHN + " " + i, 42 + i, bytes); + } + + /** */ + public static class User { + /** */ + private final String name; + + /** */ + private final int age; + + /** */ + private final byte[] payload; + + /** */ + public User(String name, int age, byte[] payload) { + this.name = name; + this.age = age; + this.payload = payload; + } + + /** */ + public String getName() { + return name; + } + + /** */ + public int getAge() { + return age; + } + + /** */ + public byte[] getPayload() { + return payload; + } + } + + /** */ + public enum ChangeEventType { + /** */ + UPDATE, + + /** */ + DELETE + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureCacheVersionTest.java b/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureCacheVersionTest.java new file mode 100644 index 0000000000000..382a2028a14ab --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureCacheVersionTest.java @@ -0,0 +1,232 @@ +/* + * 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.cdc; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +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.IgniteInternalFuture; +import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.processors.cache.CacheConflictResolutionManager; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheObjectImpl; +import org.apache.ignite.internal.processors.cache.CacheObjectValueContext; +import org.apache.ignite.internal.processors.cache.GridCacheManagerAdapter; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl; +import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo; +import org.apache.ignite.internal.processors.cache.version.CacheVersionConflictResolver; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.plugin.AbstractCachePluginProvider; +import org.apache.ignite.plugin.AbstractTestPluginProvider; +import org.apache.ignite.plugin.CachePluginContext; +import org.apache.ignite.plugin.CachePluginProvider; +import org.jetbrains.annotations.Nullable; +import org.junit.Test; + +import static org.apache.ignite.cluster.ClusterState.ACTIVE; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; + +/** */ +public class ChangeDataCaptureCacheVersionTest extends AbstractChangeDataCaptureTest { + /** */ + public static final String FOR_OTHER_CLUSTER_ID = "for-other-cluster-id"; + + /** */ + public static final byte DFLT_CLUSTER_ID = 1; + + /** */ + public static final byte OTHER_CLUSTER_ID = 2; + + /** */ + public static final int KEY_TO_UPD = 42; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setDataStorageConfiguration(new DataStorageConfiguration() + .setChangeDataCaptureEnabled(true) + .setWalForceArchiveTimeout(WAL_ARCHIVE_TIMEOUT) + .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true))); + + cfg.setPluginProviders(new AbstractTestPluginProvider() { + @Override public String name() { + return "ConflictResolverProvider"; + } + + @Override public CachePluginProvider createCacheProvider(CachePluginContext ctx) { + if (!ctx.igniteCacheConfiguration().getName().equals(FOR_OTHER_CLUSTER_ID)) + return null; + + return new AbstractCachePluginProvider() { + @Override public @Nullable Object createComponent(Class cls) { + if (cls != CacheConflictResolutionManager.class) + return null; + + return new TestCacheConflictResolutionManager(); + } + }; + } + }); + + return cfg; + } + + /** Simplest CDC test with usage of {@link IgniteInternalCache#putAllConflict(Map)}. */ + @Test + public void testReadAllKeysFromOtherCluster() throws Exception { + IgniteConfiguration cfg = getConfiguration("ignite-conflict-resolver"); + + IgniteEx ign = startGrid(cfg); + + ign.context().cache().context().versions().dataCenterId(DFLT_CLUSTER_ID); + ign.cluster().state(ACTIVE); + + UserCdcConsumer cnsmr = new UserCdcConsumer() { + @Override public void checkEvent(ChangeDataCaptureEvent evt) { + assertEquals(DFLT_CLUSTER_ID, evt.version().clusterId()); + assertEquals(OTHER_CLUSTER_ID, evt.version().otherClusterVersion().clusterId()); + + super.checkEvent(evt); + } + }; + + ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcConfig(cnsmr)); + + IgniteCache cache = ign.getOrCreateCache(FOR_OTHER_CLUSTER_ID); + + addAndWaitForConsumption(cnsmr, cdc, cache, null, this::addConflictData, 0, KEYS_CNT, getTestTimeout()); + } + + /** */ + @Test + public void testOrderIncrease() throws Exception { + IgniteConfiguration cfg = getConfiguration("ignite-0"); + + IgniteEx ign = startGrid(cfg); + + ign.cluster().state(ACTIVE); + + AtomicLong updCntr = new AtomicLong(0); + + ChangeDataCaptureConsumer cnsmr = new ChangeDataCaptureConsumer() { + private long order = -1; + + @Override public boolean onEvents(Iterator evts) { + evts.forEachRemaining(evt -> { + assertEquals(KEY_TO_UPD, evt.key()); + + assertTrue(evt.version().order() > order); + + order = evt.version().order(); + + updCntr.incrementAndGet(); + }); + + return true; + } + + @Override public void start() { + // No-op. + } + + @Override public void stop() { + // No-op. + } + }; + + ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcConfig(cnsmr)); + + IgniteCache cache = ign.getOrCreateCache("my-cache"); + + IgniteInternalFuture fut = runAsync(cdc); + + // Update the same key several time. + // Expect {@link CacheEntryVersion#order()} will monotically increase. + for (int i = 0; i < KEYS_CNT; i++) + cache.put(KEY_TO_UPD, createUser(i)); + + assertTrue(waitForCondition(() -> updCntr.get() == KEYS_CNT, getTestTimeout())); + + fut.cancel(); + } + + /** */ + private void addConflictData(IgniteCache cache, int from, int to) { + try { + IgniteEx ign = (IgniteEx)G.allGrids().get(0); + + IgniteInternalCache intCache = ign.cachex(cache.getName()); + + Map drMap = new HashMap<>(); + + for (int i = from; i < to; i++) { + KeyCacheObject key = new KeyCacheObjectImpl(i, null, intCache.affinity().partition(i)); + CacheObject val = + new CacheObjectImpl(createUser(i), null); + + val.prepareMarshal(intCache.context().cacheObjectContext()); + + drMap.put(key, new GridCacheDrInfo(val, new GridCacheVersion(1, i, 1, OTHER_CLUSTER_ID))); + } + + intCache.putAllConflict(drMap); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** */ + public static class TestCacheConflictResolutionManager extends GridCacheManagerAdapter + implements CacheConflictResolutionManager { + + /** {@inheritDoc} */ + @Override public CacheVersionConflictResolver conflictResolver() { + return new CacheVersionConflictResolver() { + @Override public GridCacheVersionConflictContext resolve( + CacheObjectValueContext ctx, + GridCacheVersionedEntryEx oldEntry, + GridCacheVersionedEntryEx newEntry, + boolean atomicVerComparator + ) { + GridCacheVersionConflictContext res = + new GridCacheVersionConflictContext<>(ctx, oldEntry, newEntry); + + res.useNew(); + + return res; + } + }; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureSelfTest.java new file mode 100644 index 0000000000000..2d95acba52782 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureSelfTest.java @@ -0,0 +1,417 @@ +/* + * 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.cdc; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +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.IgniteInternalFuture; +import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.util.typedef.F; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.ChangeEventType.DELETE; +import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.ChangeEventType.UPDATE; +import static org.apache.ignite.cluster.ClusterState.ACTIVE; +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheId; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; + +/** */ +@RunWith(Parameterized.class) +public class ChangeDataCaptureSelfTest extends AbstractChangeDataCaptureTest { + /** */ + public static final String TX_CACHE_NAME = "tx-cache"; + + /** */ + public static final int WAL_ARCHIVE_TIMEOUT = 5_000; + + /** Keys count. */ + public static final int KEYS_CNT = 50; + + /** */ + @Parameterized.Parameter + public boolean specificConsistentId; + + /** */ + @Parameterized.Parameter(1) + public WALMode walMode; + + /** */ + @Parameterized.Parameters(name = "specificConsistentId={0}, walMode={1}") + public static Collection parameters() { + return Arrays.asList(new Object[][] { + {true, WALMode.FSYNC}, + {false, WALMode.FSYNC}, + {true, WALMode.LOG_ONLY}, + {false, WALMode.LOG_ONLY}, + {true, WALMode.BACKGROUND}, + {false, WALMode.BACKGROUND} + }); + } + + /** Consistent id. */ + private UUID consistentId = UUID.randomUUID(); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + if (specificConsistentId) + cfg.setConsistentId(consistentId); + + cfg.setDataStorageConfiguration(new DataStorageConfiguration() + .setChangeDataCaptureEnabled(true) + .setWalMode(walMode) + .setWalForceArchiveTimeout(WAL_ARCHIVE_TIMEOUT) + .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true))); + + cfg.setCacheConfiguration( + new CacheConfiguration<>(TX_CACHE_NAME).setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + ); + + return cfg; + } + + /** Simplest CDC test. */ + @Test + public void testReadAllKeys() throws Exception { + // Read all records from iterator. + readAll(new UserCdcConsumer()); + + // Read one record per call. + readAll(new UserCdcConsumer() { + @Override public boolean onEvents(Iterator evts) { + super.onEvents(Collections.singleton(evts.next()).iterator()); + + return false; + } + }); + + // Read one record per call and commit. + readAll(new UserCdcConsumer() { + @Override public boolean onEvents(Iterator evts) { + super.onEvents(Collections.singleton(evts.next()).iterator()); + + return true; + } + }); + } + + /** */ + private void readAll(UserCdcConsumer cnsmr) throws Exception { + IgniteConfiguration cfg = getConfiguration("ignite-0"); + + Ignite ign = startGrid(cfg); + + ign.cluster().state(ACTIVE); + + ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcConfig(cnsmr)); + + IgniteCache cache = ign.getOrCreateCache(DEFAULT_CACHE_NAME); + IgniteCache txCache = ign.getOrCreateCache(TX_CACHE_NAME); + + addAndWaitForConsumption( + cnsmr, + cdc, + cache, + txCache, + ChangeDataCaptureSelfTest::addData, + 0, + KEYS_CNT + 3, + getTestTimeout() + ); + + removeData(cache, 0, KEYS_CNT); + + IgniteInternalFuture rmvFut = runAsync(cdc); + + assertTrue(waitForSize(KEYS_CNT, DEFAULT_CACHE_NAME, DELETE, getTestTimeout(), cnsmr)); + + rmvFut.cancel(); + + assertTrue(cnsmr.stopped()); + + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** */ + @Test + public void testReadBeforeGracefulShutdown() throws Exception { + IgniteConfiguration cfg = getConfiguration("ignite-0"); + + Ignite ign = startGrid(cfg); + + ign.cluster().state(ACTIVE); + + CountDownLatch cnsmrStarted = new CountDownLatch(1); + CountDownLatch startProcEvts = new CountDownLatch(1); + + UserCdcConsumer cnsmr = new UserCdcConsumer() { + @Override public boolean onEvents(Iterator evts) { + cnsmrStarted.countDown(); + + try { + startProcEvts.await(getTestTimeout(), TimeUnit.MILLISECONDS); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + + return super.onEvents(evts); + } + }; + + ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcConfig(cnsmr)); + + runAsync(cdc); + + IgniteCache cache = ign.getOrCreateCache(DEFAULT_CACHE_NAME); + + addData(cache, 0, KEYS_CNT); + + // Make sure all streamed data will become available for consumption. + Thread.sleep(2 * WAL_ARCHIVE_TIMEOUT); + + cnsmrStarted.await(getTestTimeout(), TimeUnit.MILLISECONDS); + + // Initiate graceful shutdown. + cdc.stop(); + + startProcEvts.countDown(); + + assertTrue(waitForSize(KEYS_CNT, DEFAULT_CACHE_NAME, UPDATE, getTestTimeout(), cnsmr)); + assertTrue(waitForCondition(cnsmr::stopped, getTestTimeout())); + + List keys = cnsmr.data(UPDATE, cacheId(DEFAULT_CACHE_NAME)); + + assertEquals(KEYS_CNT, keys.size()); + + for (int i = 0; i < KEYS_CNT; i++) + assertTrue(keys.contains(i)); + } + + /** */ + @Test + public void testMultiNodeConsumption() throws Exception { + IgniteEx ign1 = startGrid(0); + + if (specificConsistentId) + consistentId = UUID.randomUUID(); + + IgniteEx ign2 = startGrid(1); + + ign1.cluster().state(ACTIVE); + + IgniteCache cache = ign1.getOrCreateCache(DEFAULT_CACHE_NAME); + + // Adds data concurrently with CDC start. + IgniteInternalFuture addDataFut = runAsync(() -> addData(cache, 0, KEYS_CNT)); + + UserCdcConsumer cnsmr1 = new UserCdcConsumer(); + UserCdcConsumer cnsmr2 = new UserCdcConsumer(); + + IgniteConfiguration cfg1 = ign1.configuration(); + IgniteConfiguration cfg2 = ign2.configuration(); + + ChangeDataCapture cdc1 = new ChangeDataCapture(cfg1, null, cdcConfig(cnsmr1)); + ChangeDataCapture cdc2 = new ChangeDataCapture(cfg2, null, cdcConfig(cnsmr2)); + + IgniteInternalFuture fut1 = runAsync(cdc1); + IgniteInternalFuture fut2 = runAsync(cdc2); + + addDataFut.get(getTestTimeout()); + + addDataFut = runAsync(() -> addData(cache, KEYS_CNT, KEYS_CNT * 2)); + + addDataFut.get(getTestTimeout()); + + assertTrue(waitForSize(KEYS_CNT * 2, DEFAULT_CACHE_NAME, UPDATE, getTestTimeout(), cnsmr1, cnsmr2)); + + assertFalse(cnsmr1.stopped()); + assertFalse(cnsmr2.stopped()); + + fut1.cancel(); + fut2.cancel(); + + assertTrue(cnsmr1.stopped()); + assertTrue(cnsmr2.stopped()); + + removeData(cache, 0, KEYS_CNT * 2); + + IgniteInternalFuture rmvFut1 = runAsync(cdc1); + IgniteInternalFuture rmvFut2 = runAsync(cdc2); + + assertTrue(waitForSize(KEYS_CNT * 2, DEFAULT_CACHE_NAME, DELETE, getTestTimeout(), cnsmr1, cnsmr2)); + + rmvFut1.cancel(); + rmvFut2.cancel(); + + assertTrue(cnsmr1.stopped()); + assertTrue(cnsmr2.stopped()); + } + + /** */ + @Test + public void testCdcSingleton() throws Exception { + IgniteEx ign = startGrid(0); + + UserCdcConsumer cnsmr1 = new UserCdcConsumer(); + UserCdcConsumer cnsmr2 = new UserCdcConsumer(); + + IgniteInternalFuture fut1 = runAsync(new ChangeDataCapture(ign.configuration(), null, cdcConfig(cnsmr1))); + IgniteInternalFuture fut2 = runAsync(new ChangeDataCapture(ign.configuration(), null, cdcConfig(cnsmr2))); + + assertTrue(waitForCondition(() -> fut1.isDone() || fut2.isDone(), getTestTimeout())); + + assertEquals(fut1.error() == null, fut2.error() != null); + + if (fut1.isDone()) { + fut2.cancel(); + + assertTrue(cnsmr2.stopped()); + } + else { + fut1.cancel(); + + assertTrue(cnsmr1.stopped()); + } + } + + /** */ + @Test + public void testReReadWhenStateWasNotStored() throws Exception { + IgniteConfiguration cfg = getConfiguration("ignite-0"); + + IgniteEx ign = startGrid(cfg); + + ign.cluster().state(ACTIVE); + + IgniteCache cache = ign.getOrCreateCache(DEFAULT_CACHE_NAME); + + addData(cache, 0, KEYS_CNT); + + for (int i = 0; i < 3; i++) { + UserCdcConsumer cnsmr = new UserCdcConsumer() { + @Override protected boolean commit() { + return false; + } + }; + + ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcConfig(cnsmr)); + + IgniteInternalFuture fut = runAsync(cdc); + + assertTrue(waitForSize(KEYS_CNT, DEFAULT_CACHE_NAME, UPDATE, getTestTimeout(), cnsmr)); + + fut.cancel(); + + assertTrue(cnsmr.stopped()); + } + + AtomicBoolean consumeHalf = new AtomicBoolean(true); + AtomicBoolean halfCommitted = new AtomicBoolean(false); + + int half = KEYS_CNT / 2; + + UserCdcConsumer cnsmr = new UserCdcConsumer() { + @Override public boolean onEvents(Iterator evts) { + if (consumeHalf.get() && F.size(data(UPDATE, cacheId(DEFAULT_CACHE_NAME))) == half) { + // This means that state committed as a result of the previous call. + halfCommitted.set(true); + + return false; + } + + while (evts.hasNext()) { + ChangeDataCaptureEvent evt = evts.next(); + + if (!evt.primary()) + continue; + + data.computeIfAbsent( + F.t(evt.value() == null ? DELETE : UPDATE, evt.cacheId()), + k -> new ArrayList<>()).add((Integer)evt.key() + ); + + if (consumeHalf.get()) + return F.size(data(UPDATE, cacheId(DEFAULT_CACHE_NAME))) == half; + } + + return true; + } + }; + + ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcConfig(cnsmr)); + + IgniteInternalFuture fut = runAsync(cdc); + + waitForSize(half, DEFAULT_CACHE_NAME, UPDATE, getTestTimeout(), cnsmr); + + waitForCondition(halfCommitted::get, getTestTimeout()); + + fut.cancel(); + + assertTrue(cnsmr.stopped()); + + removeData(cache, 0, KEYS_CNT); + + consumeHalf.set(false); + + fut = runAsync(cdc); + + waitForSize(KEYS_CNT, DEFAULT_CACHE_NAME, UPDATE, getTestTimeout(), cnsmr); + waitForSize(KEYS_CNT, DEFAULT_CACHE_NAME, DELETE, getTestTimeout(), cnsmr); + + fut.cancel(); + + assertTrue(cnsmr.stopped()); + } + + /** */ + public static void addData(IgniteCache cache, int from, int to) { + for (int i = from; i < to; i++) + cache.put(i, createUser(i)); + } + + /** */ + private void removeData(IgniteCache cache, int from, int to) { + for (int i = from; i < to; i++) + cache.remove(i); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/plugin/AbstractCachePluginProvider.java b/modules/core/src/test/java/org/apache/ignite/plugin/AbstractCachePluginProvider.java new file mode 100644 index 0000000000000..c301e22f46c23 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/plugin/AbstractCachePluginProvider.java @@ -0,0 +1,70 @@ +/* + * 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.plugin; + +import javax.cache.Cache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.jetbrains.annotations.Nullable; + +/** + * No-op test implementation of {@link CachePluginProvider} which allows to avoid redundant boilerplate code. + */ +public abstract class AbstractCachePluginProvider implements CachePluginProvider { + /** {@inheritDoc} */ + @Override public void start() throws IgniteCheckedException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void stop(boolean cancel) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onIgniteStart() throws IgniteCheckedException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onIgniteStop(boolean cancel) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void validate() throws IgniteCheckedException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void validateRemote(CacheConfiguration locCfg, CacheConfiguration rmtCfg, + ClusterNode rmtNode) throws IgniteCheckedException { + // No-op. + } + + /** {@inheritDoc} */ + @Nullable @Override public Object unwrapCacheEntry(Cache.Entry entry, Class cls) { + return null; + } + + /** {@inheritDoc} */ + @Nullable @Override public Object createComponent(Class cls) { + return null; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java index e7ac51f2c839e..dc7635e1b6231 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java @@ -20,6 +20,8 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import org.apache.ignite.cdc.ChangeDataCaptureCacheVersionTest; +import org.apache.ignite.cdc.ChangeDataCaptureSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.HistoricalRebalanceHeuristicsTest; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.HistoricalRebalanceTwoPartsInDifferentCheckpointsTest; import org.apache.ignite.internal.processors.cache.persistence.IgniteDataStorageMetricsSelfTest; @@ -209,6 +211,10 @@ public static void addRealPageStoreTests(List> suite, Collection GridTestUtils.addTestIfNeeded(suite, IgnitePdsReserveWalSegmentsWithCompactionTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, IgniteWalReplayingAfterRestartTest.class, ignoredTests); + // CDC tests. + GridTestUtils.addTestIfNeeded(suite, ChangeDataCaptureSelfTest.class, ignoredTests); + GridTestUtils.addTestIfNeeded(suite, ChangeDataCaptureCacheVersionTest.class, ignoredTests); + // new style folders with generated consistent ID test GridTestUtils.addTestIfNeeded(suite, IgniteUidAsConsistentIdMigrationTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, IgniteWalSerializerVersionTest.class, ignoredTests); diff --git a/modules/extdata/platform/src/test/java/org/apache/ignite/platform/plugin/cache/PlatformTestCachePluginProvider.java b/modules/extdata/platform/src/test/java/org/apache/ignite/platform/plugin/cache/PlatformTestCachePluginProvider.java index 72cbc2390746e..877903afc5fac 100644 --- a/modules/extdata/platform/src/test/java/org/apache/ignite/platform/plugin/cache/PlatformTestCachePluginProvider.java +++ b/modules/extdata/platform/src/test/java/org/apache/ignite/platform/plugin/cache/PlatformTestCachePluginProvider.java @@ -17,54 +17,11 @@ package org.apache.ignite.platform.plugin.cache; -import javax.cache.Cache; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.plugin.CachePluginProvider; -import org.jetbrains.annotations.Nullable; +import org.apache.ignite.plugin.AbstractCachePluginProvider; /** * Test cache plugin provider. */ -public class PlatformTestCachePluginProvider implements CachePluginProvider { - /** {@inheritDoc} */ - @Override public void start() throws IgniteCheckedException { - // No-op. - } - - /** {@inheritDoc} */ - @Override public void stop(boolean cancel) { - // No-op. - } - - /** {@inheritDoc} */ - @Override public void onIgniteStart() throws IgniteCheckedException { - // No-op. - } - - /** {@inheritDoc} */ - @Override public void onIgniteStop(boolean cancel) { - // No-op. - } - - /** {@inheritDoc} */ - @Override public void validate() throws IgniteCheckedException { - // No-op. - } - - @Override public void validateRemote(CacheConfiguration locCfg, CacheConfiguration rmtCfg, ClusterNode rmtNode) - throws IgniteCheckedException { - // No-op. - } - - /** {@inheritDoc} */ - @Nullable @Override public Object unwrapCacheEntry(Cache.Entry entry, Class cls) { - return null; - } - - /** {@inheritDoc} */ - @Nullable @Override public Object createComponent(Class cls) { - return null; - } +public class PlatformTestCachePluginProvider extends AbstractCachePluginProvider { + // No-op. } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlChangeDataCaptureTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlChangeDataCaptureTest.java new file mode 100644 index 0000000000000..0c46af284f686 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlChangeDataCaptureTest.java @@ -0,0 +1,179 @@ +/* + * 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.cdc; + +import java.util.List; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cdc.AbstractChangeDataCaptureTest; +import org.apache.ignite.cdc.ChangeDataCaptureConfiguration; +import org.apache.ignite.cdc.ChangeDataCaptureEvent; +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.IgniteInternalFuture; +import org.junit.Test; + +import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.ChangeEventType.DELETE; +import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.ChangeEventType.UPDATE; +import static org.apache.ignite.cluster.ClusterState.ACTIVE; +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheId; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; + +/** */ +public class SqlChangeDataCaptureTest extends AbstractChangeDataCaptureTest { + /** */ + private static final String SARAH = "Sarah Connor"; + + /** */ + public static final String USER = "user"; + + /** */ + public static final String CITY = "city"; + + /** */ + public static final String SPB = "Saint-Petersburg"; + + /** */ + public static final String MSK = "Moscow"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setDataStorageConfiguration(new DataStorageConfiguration() + .setChangeDataCaptureEnabled(true) + .setWalForceArchiveTimeout(WAL_ARCHIVE_TIMEOUT) + .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true))); + + return cfg; + } + + /** Simplest CDC test. */ + @Test + public void testReadAllSQLRows() throws Exception { + IgniteConfiguration cfg = getConfiguration("ignite-0"); + + IgniteEx ign = startGrid(cfg); + + ign.cluster().state(ACTIVE); + + BinaryCdcConsumer cnsmr = new BinaryCdcConsumer(); + + ChangeDataCaptureConfiguration cdcCfg = new ChangeDataCaptureConfiguration(); + + cdcCfg.setConsumer(cnsmr); + + ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcCfg); + + IgniteInternalFuture fut = runAsync(cdc); + + executeSql( + ign, + "CREATE TABLE USER(id int, city_id int, name varchar, PRIMARY KEY (id, city_id)) WITH \"CACHE_NAME=user\"" + ); + + executeSql( + ign, + "CREATE TABLE CITY(id int, name varchar, zip_code varchar(6), PRIMARY KEY (id)) WITH \"CACHE_NAME=city\"" + ); + + for (int i = 0; i < KEYS_CNT; i++) { + executeSql( + ign, + "INSERT INTO USER VALUES(?, ?, ?)", + i, + 42 * i, + (i % 2 == 0 ? JOHN : SARAH) + i); + + executeSql( + ign, + "INSERT INTO CITY VALUES(?, ?, ?)", + i, + (i % 2 == 0 ? MSK : SPB) + i, + Integer.toString(127000 + i)); + } + + assertTrue(waitForSize(KEYS_CNT, USER, UPDATE, getTestTimeout(), cnsmr)); + assertTrue(waitForSize(KEYS_CNT, CITY, UPDATE, getTestTimeout(), cnsmr)); + + fut.cancel(); + + assertEquals(KEYS_CNT, cnsmr.data(UPDATE, cacheId(USER)).size()); + assertEquals(KEYS_CNT, cnsmr.data(UPDATE, cacheId(CITY)).size()); + + assertTrue(cnsmr.stopped()); + + for (int i = 0; i < KEYS_CNT; i++) + executeSql(ign, "DELETE FROM USER WHERE id = ?", i); + + IgniteInternalFuture rmvFut = runAsync(cdc); + + assertTrue(waitForSize(KEYS_CNT, USER, DELETE, getTestTimeout(), cnsmr)); + + rmvFut.cancel(); + + assertTrue(cnsmr.stopped()); + } + + /** */ + public static class BinaryCdcConsumer extends TestCdcConsumer { + /** {@inheritDoc} */ + @Override public void checkEvent(ChangeDataCaptureEvent evt) { + if (evt.value() == null) + return; + + if (evt.cacheId() == cacheId(USER)) { + int id = ((BinaryObject)evt.key()).field("ID"); + int cityId = ((BinaryObject)evt.key()).field("CITY_ID"); + + assertEquals(42 * id, cityId); + + String name = ((BinaryObject)evt.value()).field("NAME"); + + if (id % 2 == 0) + assertTrue(name.startsWith(JOHN)); + else + assertTrue(name.startsWith(SARAH)); + } + else { + int id = (Integer)evt.key(); + String name = ((BinaryObject)evt.value()).field("NAME"); + String zipCode = ((BinaryObject)evt.value()).field("ZIP_CODE"); + + assertEquals(Integer.toString(127000 + id), zipCode); + + if (id % 2 == 0) + assertTrue(name.startsWith(MSK)); + else + assertTrue(name.startsWith(SPB)); + } + } + + /** {@inheritDoc} */ + @Override public ChangeDataCaptureEvent extract(ChangeDataCaptureEvent evt) { + return evt; + } + } + + /** */ + private List> executeSql(IgniteEx node, String sqlText, Object... args) { + return node.context().query().querySqlFields(new SqlFieldsQuery(sqlText).setArgs(args), true).getAll(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java index ff0200b683e14..8d2b93360cd41 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java @@ -18,6 +18,7 @@ package org.apache.ignite.testsuites; import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyTypeRegistryTest; +import org.apache.ignite.internal.cdc.SqlChangeDataCaptureTest; import org.apache.ignite.internal.metric.SystemViewSelfTest; import org.apache.ignite.internal.processors.cache.AffinityKeyNameAndValueFieldNameConflictTest; import org.apache.ignite.internal.processors.cache.BigEntryQueryTest; @@ -664,7 +665,10 @@ SqlFieldTypeValidationTypesTest.class, SqlFieldTypeValidationOnKeyValueInsertTest.class, - InlineIndexKeyTypeRegistryTest.class + InlineIndexKeyTypeRegistryTest.class, + + // CDC tests. + SqlChangeDataCaptureTest.class }) public class IgniteBinaryCacheQueryTestSuite { } diff --git a/modules/spring/src/test/config/cdc/cdc-config-without-persistence.xml b/modules/spring/src/test/config/cdc/cdc-config-without-persistence.xml new file mode 100644 index 0000000000000..83b20443dba43 --- /dev/null +++ b/modules/spring/src/test/config/cdc/cdc-config-without-persistence.xml @@ -0,0 +1,30 @@ + + + + + + + + + + diff --git a/modules/spring/src/test/config/cdc/correct-cdc-config.xml b/modules/spring/src/test/config/cdc/correct-cdc-config.xml new file mode 100644 index 0000000000000..3a63298f9d58f --- /dev/null +++ b/modules/spring/src/test/config/cdc/correct-cdc-config.xml @@ -0,0 +1,53 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/modules/spring/src/test/config/cdc/double-cdc-config.xml b/modules/spring/src/test/config/cdc/double-cdc-config.xml new file mode 100644 index 0000000000000..4bd7d5f262759 --- /dev/null +++ b/modules/spring/src/test/config/cdc/double-cdc-config.xml @@ -0,0 +1,30 @@ + + + + + + + + + + diff --git a/modules/spring/src/test/config/cdc/double-ignite-config.xml b/modules/spring/src/test/config/cdc/double-ignite-config.xml new file mode 100644 index 0000000000000..26eb8d52c7873 --- /dev/null +++ b/modules/spring/src/test/config/cdc/double-ignite-config.xml @@ -0,0 +1,30 @@ + + + + + + + + + + diff --git a/modules/spring/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureConfigurationTest.java b/modules/spring/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureConfigurationTest.java new file mode 100644 index 0000000000000..14da7c73c14e7 --- /dev/null +++ b/modules/spring/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureConfigurationTest.java @@ -0,0 +1,140 @@ +/* + * 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.cdc; + +import java.util.Iterator; +import java.util.concurrent.CountDownLatch; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.cdc.WalRecordsConsumer; +import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.resources.SpringApplicationContextResource; +import org.apache.ignite.resources.SpringResource; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Test; +import org.springframework.context.ApplicationContext; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.ignite.cdc.ChangeDataCaptureLoader.loadChangeDataCapture; +import static org.apache.ignite.internal.cdc.ChangeDataCapture.ERR_MSG; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; +import static org.apache.ignite.testframework.GridTestUtils.getFieldValue; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; + +/** */ +public class ChangeDataCaptureConfigurationTest extends GridCommonAbstractTest { + /** */ + @Test + public void testLoadConfig() throws Exception { + assertThrows( + null, + () -> loadChangeDataCapture("modules/spring/src/test/config/cdc/double-ignite-config.xml"), + IgniteCheckedException.class, + "Exact 1 IgniteConfiguration should be defined. Found 2" + ); + + assertThrows( + null, + () -> loadChangeDataCapture("modules/spring/src/test/config/cdc/double-cdc-config.xml"), + IgniteCheckedException.class, + "Exact 1 CaptureDataChangeConfiguration configuration should be defined. Found 2" + ); + + ChangeDataCapture cdc = + loadChangeDataCapture("modules/spring/src/test/config/cdc/cdc-config-without-persistence.xml"); + + assertNotNull(cdc); + + assertThrows(null, cdc::run, IgniteException.class, ERR_MSG); + } + + /** */ + @Test + public void testInjectResources() throws Exception { + ChangeDataCapture cdc = + loadChangeDataCapture("modules/spring/src/test/config/cdc/correct-cdc-config.xml"); + + try (IgniteEx ign = startGrid((IgniteConfiguration)getFieldValue(cdc, "igniteCfg"))) { + TestCdcConsumer cnsmr = + (TestCdcConsumer)((WalRecordsConsumer)getFieldValue(cdc, "consumer")).consumer(); + + assertNotNull(cnsmr); + + CountDownLatch startLatch = cnsmr.startLatch; + + IgniteInternalFuture fut = runAsync(cdc::run); + + startLatch.await(getTestTimeout(), MILLISECONDS); + + assertEquals("someString", cnsmr.springString); + assertEquals("someString2", cnsmr.springString2); + assertNotNull(cnsmr.log); + assertNotNull(cnsmr.ctx); + + fut.cancel(); + } + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + cleanPersistenceDir(); + } + + /** */ + public static class TestCdcConsumer implements ChangeDataCaptureConsumer { + /** */ + @LoggerResource + private IgniteLogger log; + + /** */ + @SpringResource(resourceName = "springString") + private String springString; + + /** */ + private String springString2; + + /** */ + @SpringApplicationContextResource + private ApplicationContext ctx; + + /** */ + public CountDownLatch startLatch = new CountDownLatch(1); + + /** {@inheritDoc} */ + @Override public void start() { + springString2 = ctx.getBean("springString2", String.class); + + startLatch.countDown(); + } + + /** {@inheritDoc} */ + @Override public boolean onEvents(Iterator events) { + return false; + } + + /** {@inheritDoc} */ + @Override public void stop() { + // No-Op. + } + } +} diff --git a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java index 07222d1254024..24df4f7799d44 100644 --- a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java +++ b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java @@ -22,6 +22,7 @@ import org.apache.ignite.cache.store.jdbc.CachePojoStoreXmlSelfTest; import org.apache.ignite.cache.store.jdbc.CachePojoStoreXmlWithSqlEscapeSelfTest; import org.apache.ignite.cache.store.spring.CacheSpringStoreSessionListenerSelfTest; +import org.apache.ignite.cdc.ChangeDataCaptureConfigurationTest; import org.apache.ignite.cluster.ClusterStateXmlPropertiesTest; import org.apache.ignite.encryption.SpringEncryptedCacheRestartClientTest; import org.apache.ignite.encryption.SpringEncryptedCacheRestartTest; @@ -81,7 +82,10 @@ GridCacheDhtMultiBackupTest.class, - ClusterStateXmlPropertiesTest.class + ClusterStateXmlPropertiesTest.class, + + // CDC tests. + ChangeDataCaptureConfigurationTest.class }) public class IgniteSpringTestSuite { } diff --git a/parent/pom.xml b/parent/pom.xml index 6cdfc2e7fb074..c7504159a8248 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -499,6 +499,10 @@ Tracing SPI org.apache.ignite.spi.tracing* + + Capture Data Change API + org.apache.ignite.cdc* + Date: Mon, 7 Jun 2021 17:12:54 +0300 Subject: [PATCH 16/28] IGNITE-13581: Minor fix. --- .../java/org/apache/ignite/cdc/ChangeDataCaptureSelfTest.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureSelfTest.java index 2d95acba52782..4be9e6f73f85c 100644 --- a/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureSelfTest.java @@ -59,9 +59,6 @@ public class ChangeDataCaptureSelfTest extends AbstractChangeDataCaptureTest { /** */ public static final int WAL_ARCHIVE_TIMEOUT = 5_000; - /** Keys count. */ - public static final int KEYS_CNT = 50; - /** */ @Parameterized.Parameter public boolean specificConsistentId; From 975595c57d450101d47b6dc44ff57a7b73146be2 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Tue, 8 Jun 2021 09:47:12 +0300 Subject: [PATCH 17/28] IGNITE-13581: Minor fix. --- .../java/org/apache/ignite/cache/CacheEntryVersion.java | 5 +++-- .../internal/processors/cache/version/GridCacheVersion.java | 2 +- .../processors/cache/version/GridCacheVersionEx.java | 6 ++++++ .../apache/ignite/cdc/AbstractChangeDataCaptureTest.java | 2 ++ .../ignite/cdc/ChangeDataCaptureCacheVersionTest.java | 2 -- 5 files changed, 12 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheEntryVersion.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheEntryVersion.java index f22a4e0f7f5b5..8aa97721b8221 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/CacheEntryVersion.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheEntryVersion.java @@ -56,8 +56,9 @@ public interface CacheEntryVersion extends Comparable, Serial public int topologyVersion(); /** - * If source of the update is "local" cluster then {@code this} will be returned. - * If updated comes from the other cluster using {@link IgniteInternalCache#putAllConflict(Map)} then + * If source of the update is "local" cluster then {@code null} will be returned. + * If updated comes from the other cluster using {@link IgniteInternalCache#putAllConflict(Map)} + * then entry version for other cluster. * @return Replication version. * @see IgniteInternalCache#putAllConflict(Map) * @see IgniteInternalCache#removeAllConflict(Map) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java index 1d5b3456cedb1..7769501aa8164 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java @@ -116,7 +116,7 @@ public int nodeOrderAndDrIdRaw() { /** {@inheritDoc} */ @Override public CacheEntryVersion otherClusterVersion() { - return conflictVersion(); + return null; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionEx.java index 5eb8c7cb15d35..1d5bad049c545 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionEx.java @@ -22,6 +22,7 @@ import java.io.ObjectInput; import java.io.ObjectOutput; import java.nio.ByteBuffer; +import org.apache.ignite.cache.CacheEntryVersion; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; @@ -81,6 +82,11 @@ public GridCacheVersionEx(int topVer, int nodeOrderDrId, long order, GridCacheVe return drVer; } + /** {@inheritDoc} */ + @Override public CacheEntryVersion otherClusterVersion() { + return conflictVersion(); + } + /** {@inheritDoc} */ @Override public short directType() { return 104; diff --git a/modules/core/src/test/java/org/apache/ignite/cdc/AbstractChangeDataCaptureTest.java b/modules/core/src/test/java/org/apache/ignite/cdc/AbstractChangeDataCaptureTest.java index 268e0ac2df8f6..9c8b95d350578 100644 --- a/modules/core/src/test/java/org/apache/ignite/cdc/AbstractChangeDataCaptureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cdc/AbstractChangeDataCaptureTest.java @@ -181,6 +181,8 @@ public boolean stopped() { public static class UserCdcConsumer extends TestCdcConsumer { /** {@inheritDoc} */ @Override public void checkEvent(ChangeDataCaptureEvent evt) { + assertNull(evt.version().otherClusterVersion()); + if (evt.value() == null) return; diff --git a/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureCacheVersionTest.java b/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureCacheVersionTest.java index 382a2028a14ab..640adb97e3c26 100644 --- a/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureCacheVersionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureCacheVersionTest.java @@ -115,8 +115,6 @@ public void testReadAllKeysFromOtherCluster() throws Exception { @Override public void checkEvent(ChangeDataCaptureEvent evt) { assertEquals(DFLT_CLUSTER_ID, evt.version().clusterId()); assertEquals(OTHER_CLUSTER_ID, evt.version().otherClusterVersion().clusterId()); - - super.checkEvent(evt); } }; From 53d207c83da0ea96e337d54f3e7b864e5bbd435d Mon Sep 17 00:00:00 2001 From: Nikolay Date: Thu, 10 Jun 2021 11:39:12 +0300 Subject: [PATCH 18/28] IGNITE-13581: Minor fix. --- .../java/org/apache/ignite/cdc/ChangeDataCaptureLoader.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureLoader.java b/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureLoader.java index 17b95e7be00ac..a02c07cadc55a 100644 --- a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureLoader.java +++ b/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureLoader.java @@ -34,11 +34,10 @@ */ public class ChangeDataCaptureLoader { /** - * Loads {@link ChangeDataCapture} from XML configuration file and possible error message. - * If load fails then error message wouldn't be null. + * Loads {@link ChangeDataCapture} from XML configuration file. * * @param springXmlPath Path to XML configuration file. - * @return Tuple of {@code ChangeDataCapture} and error message. + * @return {@code ChangeDataCapture} instance. * @throws IgniteCheckedException If failed. */ public static ChangeDataCapture loadChangeDataCapture(String springXmlPath) throws IgniteCheckedException { From 8f2b78b917f7c0a987034ca21825b9c13a703b68 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Tue, 15 Jun 2021 17:05:51 +0300 Subject: [PATCH 19/28] IGNITE-13581: Minor fix. --- .../ChangeDataCaptureCommandLineStartup.java | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/startup/cmdline/ChangeDataCaptureCommandLineStartup.java b/modules/core/src/main/java/org/apache/ignite/startup/cmdline/ChangeDataCaptureCommandLineStartup.java index 8eb5b565cb6b3..7669f02d81967 100644 --- a/modules/core/src/main/java/org/apache/ignite/startup/cmdline/ChangeDataCaptureCommandLineStartup.java +++ b/modules/core/src/main/java/org/apache/ignite/startup/cmdline/ChangeDataCaptureCommandLineStartup.java @@ -19,6 +19,7 @@ import java.net.URL; import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cdc.ChangeDataCaptureConfiguration; @@ -79,23 +80,29 @@ public static void main(String[] args) { if (args.length > 0 && args[0].charAt(0) == '-') exit("Invalid arguments: " + args[0], true, -1); + AtomicReference cdc = new AtomicReference<>(); + try { - ChangeDataCapture cdc = ChangeDataCaptureLoader.loadChangeDataCapture(args[0]); + cdc.set(ChangeDataCaptureLoader.loadChangeDataCapture(args[0])); if (!IgniteSystemProperties.getBoolean(IGNITE_NO_SHUTDOWN_HOOK, false)) { Runtime.getRuntime().addShutdownHook(new Thread("cdc-shutdown-hook") { @Override public void run() { - cdc.stop(); + cdc.get().stop(); } }); } - Thread appThread = new Thread(cdc); + Thread appThread = new Thread(cdc.get()); appThread.start(); appThread.join(); } + catch (InterruptedException e) { + if (cdc.get() != null) + cdc.get().stop(); + } catch (Throwable e) { e.printStackTrace(); From 8f568e57f43577dd5ee6dfa059ab01332bc9310b Mon Sep 17 00:00:00 2001 From: Nikolay Date: Fri, 18 Jun 2021 15:49:43 +0300 Subject: [PATCH 20/28] IGNITE-13581: Minor fix. --- .../ignite/internal/cdc/ChangeDataCapture.java | 8 +------- .../ignite/internal/util/IgniteUtils.java | 17 ++++++++++++++++- 2 files changed, 17 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCapture.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCapture.java index 5fb3e183c0e24..9ff6a65f509ad 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCapture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCapture.java @@ -27,7 +27,6 @@ import java.util.HashSet; import java.util.Iterator; import java.util.Set; -import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Stream; import org.apache.ignite.IgniteCheckedException; @@ -169,12 +168,7 @@ public ChangeDataCapture( try { initWorkDir(this.igniteCfg); - log = U.initLogger( - igniteCfg.getGridLogger(), - "ignite-cdc", - igniteCfg.getNodeId() != null ? igniteCfg.getNodeId() : UUID.randomUUID(), - igniteCfg.getWorkDirectory() - ); + log = U.initLogger(igniteCfg, "ignite-cdc"); } catch (IgniteCheckedException e) { throw new IgniteException(e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 3fde13a1d1711..a03cffd490164 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -4639,9 +4639,24 @@ public static void log(@Nullable IgniteLogger log, Object longMsg, Object shortM quiet(false, shortMsg); } + /** + * @param cfg Ignite configuration. + * @param app Application name. + * @return Initialized logger. + * @throws IgniteCheckedException If failed. + */ + public static IgniteLogger initLogger(IgniteConfiguration cfg, String app) throws IgniteCheckedException { + return initLogger( + cfg.getGridLogger(), + app, + cfg.getNodeId() != null ? cfg.getNodeId() : UUID.randomUUID(), + cfg.getWorkDirectory() + ); + } + /** * @param cfgLog Configured logger. - * @param nodeId Local node ID. + * @param app Application name. * @param workDir Work directory. * @return Initialized logger. * @throws IgniteCheckedException If failed. From 374f31e54bac98ca0ef91ee4160bc53f3c128f1e Mon Sep 17 00:00:00 2001 From: Nikolay Date: Wed, 23 Jun 2021 14:56:23 +0300 Subject: [PATCH 21/28] ChangeDataCapture -> Cdc (#9190) --- ...nfiguration.java => CdcConfiguration.java} | 18 +++--- ...aCaptureConsumer.java => CdcConsumer.java} | 14 ++--- ...ngeDataCaptureEvent.java => CdcEvent.java} | 10 ++-- ...eDataCaptureLoader.java => CdcLoader.java} | 16 ++--- .../DataStorageConfiguration.java | 24 ++++---- ...nsumerState.java => CdcConsumerState.java} | 16 ++--- ...aptureEventImpl.java => CdcEventImpl.java} | 14 ++--- ...LockHolder.java => CdcFileLockHolder.java} | 8 +-- .../{ChangeDataCapture.java => CdcMain.java} | 58 +++++++++---------- .../internal/cdc/WalRecordsConsumer.java | 26 ++++----- .../wal/FileWriteAheadLogManager.java | 6 +- ...tartup.java => CdcCommandLineStartup.java} | 20 +++---- ...aCaptureTest.java => AbstractCdcTest.java} | 34 +++++------ ...sionTest.java => CdcCacheVersionTest.java} | 16 ++--- ...aCaptureSelfTest.java => CdcSelfTest.java} | 38 ++++++------ .../db/wal/reader/IgniteWalReaderTest.java | 1 + .../testsuites/IgnitePdsTestSuite2.java | 8 +-- ...geDataCaptureTest.java => SqlCdcTest.java} | 24 ++++---- .../IgniteBinaryCacheQueryTestSuite.java | 4 +- .../cdc/cdc-config-without-persistence.xml | 2 +- .../test/config/cdc/correct-cdc-config.xml | 6 +- .../src/test/config/cdc/double-cdc-config.xml | 2 +- ...ionTest.java => CdcConfigurationTest.java} | 24 ++++---- .../testsuites/IgniteSpringTestSuite.java | 4 +- 24 files changed, 197 insertions(+), 196 deletions(-) rename modules/core/src/main/java/org/apache/ignite/cdc/{ChangeDataCaptureConfiguration.java => CdcConfiguration.java} (83%) rename modules/core/src/main/java/org/apache/ignite/cdc/{ChangeDataCaptureConsumer.java => CdcConsumer.java} (87%) rename modules/core/src/main/java/org/apache/ignite/cdc/{ChangeDataCaptureEvent.java => CdcEvent.java} (89%) rename modules/core/src/main/java/org/apache/ignite/cdc/{ChangeDataCaptureLoader.java => CdcLoader.java} (80%) rename modules/core/src/main/java/org/apache/ignite/internal/cdc/{ChangeDataCaptureConsumerState.java => CdcConsumerState.java} (87%) rename modules/core/src/main/java/org/apache/ignite/internal/cdc/{ChangeDataCaptureEventImpl.java => CdcEventImpl.java} (86%) rename modules/core/src/main/java/org/apache/ignite/internal/cdc/{ChangeDataCaptureFileLockHolder.java => CdcFileLockHolder.java} (88%) rename modules/core/src/main/java/org/apache/ignite/internal/cdc/{ChangeDataCapture.java => CdcMain.java} (89%) rename modules/core/src/main/java/org/apache/ignite/startup/cmdline/{ChangeDataCaptureCommandLineStartup.java => CdcCommandLineStartup.java} (91%) rename modules/core/src/test/java/org/apache/ignite/cdc/{AbstractChangeDataCaptureTest.java => AbstractCdcTest.java} (83%) rename modules/core/src/test/java/org/apache/ignite/cdc/{ChangeDataCaptureCacheVersionTest.java => CdcCacheVersionTest.java} (93%) rename modules/core/src/test/java/org/apache/ignite/cdc/{ChangeDataCaptureSelfTest.java => CdcSelfTest.java} (88%) rename modules/indexing/src/test/java/org/apache/ignite/internal/cdc/{SqlChangeDataCaptureTest.java => SqlCdcTest.java} (87%) rename modules/spring/src/test/java/org/apache/ignite/cdc/{ChangeDataCaptureConfigurationTest.java => CdcConfigurationTest.java} (81%) diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConfiguration.java b/modules/core/src/main/java/org/apache/ignite/cdc/CdcConfiguration.java similarity index 83% rename from modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConfiguration.java rename to modules/core/src/main/java/org/apache/ignite/cdc/CdcConfiguration.java index 302d70f9adb09..ca2d89cba7f23 100644 --- a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/cdc/CdcConfiguration.java @@ -18,15 +18,15 @@ package org.apache.ignite.cdc; import org.apache.ignite.configuration.DataStorageConfiguration; -import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.cdc.CdcMain; import org.apache.ignite.lang.IgniteExperimental; /** - * This class defines {@link ChangeDataCapture} runtime configuration. - * Configuration is passed to {@link ChangeDataCapture} constructor. + * This class defines {@link CdcMain} runtime configuration. + * Configuration is passed to {@link CdcMain} constructor. */ @IgniteExperimental -public class ChangeDataCaptureConfiguration { +public class CdcConfiguration { /** */ private static final int DFLT_LOCK_TIMEOUT = 1000; @@ -37,32 +37,32 @@ public class ChangeDataCaptureConfiguration { private static final boolean DFLT_KEEP_BINARY = true; /** Change Data Capture consumer. */ - private ChangeDataCaptureConsumer consumer; + private CdcConsumer consumer; /** Keep binary flag.
Default value {@code true}. */ private boolean keepBinary = DFLT_KEEP_BINARY; /** - * {@link ChangeDataCapture} acquire file lock on startup to ensure exclusive consumption. + * {@link CdcMain} acquire file lock on startup to ensure exclusive consumption. * This property specifies amount of time to wait for lock acquisition.
* Default is {@code 1000 ms}. */ private long lockTimeout = DFLT_LOCK_TIMEOUT; /** - * CDC application periodically scans {@link DataStorageConfiguration#getChangeDataCaptureWalPath()} folder to find new WAL segments. + * CDC application periodically scans {@link DataStorageConfiguration#getCdcWalPath()} folder to find new WAL segments. * This frequency specify amount of time application sleeps between subsequent checks when no new files available. * Default is {@code 1000 ms}. */ private long checkFreq = DFLT_CHECK_FREQ; /** @return CDC consumer. */ - public ChangeDataCaptureConsumer getConsumer() { + public CdcConsumer getConsumer() { return consumer; } /** @param consumer CDC consumer. */ - public void setConsumer(ChangeDataCaptureConsumer consumer) { + public void setConsumer(CdcConsumer consumer) { this.consumer = consumer; } diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConsumer.java b/modules/core/src/main/java/org/apache/ignite/cdc/CdcConsumer.java similarity index 87% rename from modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConsumer.java rename to modules/core/src/main/java/org/apache/ignite/cdc/CdcConsumer.java index aa58002046e29..67c7f3c0a1c45 100644 --- a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureConsumer.java +++ b/modules/core/src/main/java/org/apache/ignite/cdc/CdcConsumer.java @@ -20,13 +20,13 @@ import java.util.Iterator; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.CacheEntryVersion; -import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.cdc.CdcMain; import org.apache.ignite.lang.IgniteExperimental; import org.apache.ignite.resources.LoggerResource; /** * Consumer of WAL data change events. - * This consumer will receive data change events during {@link ChangeDataCapture} application invocation. + * This consumer will receive data change events during {@link CdcMain} application invocation. * The lifecycle of the consumer is the following: *

    *
  • Start of the consumer {@link #start()}.
  • @@ -44,15 +44,15 @@ * } * } * - * Note, consumption of the {@link ChangeDataCaptureEvent} will be started from the last saved offset. + * Note, consumption of the {@link CdcEvent} will be started from the last saved offset. * The offset of consumptions is saved on the disk every time {@link #onEvents(Iterator)} returns {@code true}. * - * @see ChangeDataCapture - * @see ChangeDataCaptureEvent + * @see CdcMain + * @see CdcEvent * @see CacheEntryVersion */ @IgniteExperimental -public interface ChangeDataCaptureConsumer { +public interface CdcConsumer { /** * Starts the consumer. */ @@ -67,7 +67,7 @@ public interface ChangeDataCaptureConsumer { * @return {@code True} if current offset should be saved on the disk * to continue from it in case any failures or restart. */ - public boolean onEvents(Iterator events); + public boolean onEvents(Iterator events); /** * Stops the consumer. diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureEvent.java b/modules/core/src/main/java/org/apache/ignite/cdc/CdcEvent.java similarity index 89% rename from modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureEvent.java rename to modules/core/src/main/java/org/apache/ignite/cdc/CdcEvent.java index 90540c160aa5c..2f8f7a5a6654d 100644 --- a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureEvent.java +++ b/modules/core/src/main/java/org/apache/ignite/cdc/CdcEvent.java @@ -20,7 +20,7 @@ import java.io.Serializable; import org.apache.ignite.cache.CacheEntryVersion; import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.cdc.CdcMain; import org.apache.ignite.lang.IgniteExperimental; import org.apache.ignite.spi.systemview.view.CacheView; import org.jetbrains.annotations.Nullable; @@ -29,11 +29,11 @@ * Event of single entry change. * Instance presents new value of modified entry. * - * @see ChangeDataCapture - * @see ChangeDataCaptureConsumer + * @see CdcMain + * @see CdcConsumer */ @IgniteExperimental -public interface ChangeDataCaptureEvent extends Serializable { +public interface CdcEvent extends Serializable { /** * @return Key for the changed entry. */ @@ -54,7 +54,7 @@ public interface ChangeDataCaptureEvent extends Serializable { /** * Ignite split dataset into smaller chunks to distribute them across the cluster. - * {@link ChangeDataCaptureConsumer} implementations can use {@link #partition()} to split changes processing + * {@link CdcConsumer} implementations can use {@link #partition()} to split changes processing * in the same way as it done for the cache. * * @return Partition number. diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureLoader.java b/modules/core/src/main/java/org/apache/ignite/cdc/CdcLoader.java similarity index 80% rename from modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureLoader.java rename to modules/core/src/main/java/org/apache/ignite/cdc/CdcLoader.java index a02c07cadc55a..8a907f7d434ec 100644 --- a/modules/core/src/main/java/org/apache/ignite/cdc/ChangeDataCaptureLoader.java +++ b/modules/core/src/main/java/org/apache/ignite/cdc/CdcLoader.java @@ -21,7 +21,7 @@ import java.util.Collection; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.cdc.CdcMain; import org.apache.ignite.internal.processors.resource.GridSpringResourceContext; import org.apache.ignite.internal.util.spring.IgniteSpringHelper; import org.apache.ignite.internal.util.typedef.internal.U; @@ -30,17 +30,17 @@ import static org.apache.ignite.internal.IgniteComponentType.SPRING; /** - * Utility class to load {@link ChangeDataCapture} from Spring XML configuration. + * Utility class to load {@link CdcMain} from Spring XML configuration. */ -public class ChangeDataCaptureLoader { +public class CdcLoader { /** - * Loads {@link ChangeDataCapture} from XML configuration file. + * Loads {@link CdcMain} from XML configuration file. * * @param springXmlPath Path to XML configuration file. * @return {@code ChangeDataCapture} instance. * @throws IgniteCheckedException If failed. */ - public static ChangeDataCapture loadChangeDataCapture(String springXmlPath) throws IgniteCheckedException { + public static CdcMain loadCdc(String springXmlPath) throws IgniteCheckedException { URL cfgUrl = U.resolveSpringUrl(springXmlPath); IgniteSpringHelper spring = SPRING.create(false); @@ -54,8 +54,8 @@ public static ChangeDataCapture loadChangeDataCapture(String springXmlPath) thro ); } - IgniteBiTuple, ? extends GridSpringResourceContext> cdcCfgs = - spring.loadConfigurations(cfgUrl, ChangeDataCaptureConfiguration.class); + IgniteBiTuple, ? extends GridSpringResourceContext> cdcCfgs = + spring.loadConfigurations(cfgUrl, CdcConfiguration.class); if (cdcCfgs.get1().size() > 1) { throw new IgniteCheckedException( @@ -64,7 +64,7 @@ public static ChangeDataCapture loadChangeDataCapture(String springXmlPath) thro ); } - return new ChangeDataCapture( + return new CdcMain( cfgTuple.get1().iterator().next(), cfgTuple.get2(), cdcCfgs.get1().iterator().next() diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java index 984f98f9c2f99..e62707e01d96e 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java @@ -250,11 +250,11 @@ public class DataStorageConfiguration implements Serializable { /** Change Data Capture path. */ @IgniteExperimental - private String changeDataCaptureWalPath = DFLT_WAL_CDC_PATH; + private String cdcWalPath = DFLT_WAL_CDC_PATH; /** Change Data Capture enabled flag. */ @IgniteExperimental - private boolean changeDataCaptureEnabled; + private boolean cdcEnabled; /** Metrics enabled flag. */ private boolean metricsEnabled = DFLT_METRICS_ENABLED; @@ -751,20 +751,20 @@ public DataStorageConfiguration setWalArchivePath(String walArchivePath) { * @return CDC directory. */ @IgniteExperimental - public String getChangeDataCaptureWalPath() { - return changeDataCaptureWalPath; + public String getCdcWalPath() { + return cdcWalPath; } /** * Sets a path for the CDC directory. * Hard link to every WAL Archive segment will be created in it for CDC processing purpose. * - * @param changeDataCaptureWalPath CDC directory. + * @param cdcWalPath CDC directory. * @return {@code this} for chaining. */ @IgniteExperimental - public DataStorageConfiguration setChangeDataCaptureWalPath(String changeDataCaptureWalPath) { - this.changeDataCaptureWalPath = changeDataCaptureWalPath; + public DataStorageConfiguration setCdcWalPath(String cdcWalPath) { + this.cdcWalPath = cdcWalPath; return this; } @@ -772,11 +772,11 @@ public DataStorageConfiguration setChangeDataCaptureWalPath(String changeDataCap /** * Sets flag indicating whether CDC enabled. * - * @param changeDataCaptureEnabled CDC enabled flag. + * @param cdcEnabled CDC enabled flag. */ @IgniteExperimental - public DataStorageConfiguration setChangeDataCaptureEnabled(boolean changeDataCaptureEnabled) { - this.changeDataCaptureEnabled = changeDataCaptureEnabled; + public DataStorageConfiguration setCdcEnabled(boolean cdcEnabled) { + this.cdcEnabled = cdcEnabled; return this; } @@ -788,8 +788,8 @@ public DataStorageConfiguration setChangeDataCaptureEnabled(boolean changeDataCa * @return Metrics enabled flag. */ @IgniteExperimental - public boolean isChangeDataCaptureEnabled() { - return changeDataCaptureEnabled; + public boolean isCdcEnabled() { + return cdcEnabled; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureConsumerState.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcConsumerState.java similarity index 87% rename from modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureConsumerState.java rename to modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcConsumerState.java index cc6f67cfac5e1..0ac8b5781bbe8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureConsumerState.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcConsumerState.java @@ -25,8 +25,8 @@ import java.nio.file.StandardOpenOption; import java.util.Iterator; import org.apache.ignite.IgniteException; -import org.apache.ignite.cdc.ChangeDataCaptureConsumer; -import org.apache.ignite.cdc.ChangeDataCaptureEvent; +import org.apache.ignite.cdc.CdcConsumer; +import org.apache.ignite.cdc.CdcEvent; import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer; import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; @@ -38,15 +38,15 @@ /** * Change Data Capture Consumer state. * - * Each time {@link ChangeDataCaptureConsumer#onEvents(Iterator)} returns {@code true} + * Each time {@link CdcConsumer#onEvents(Iterator)} returns {@code true} * current offset in WAL segment saved to file. - * This allows to the {@link ChangeDataCaptureConsumer} to continue consumption of the {@link ChangeDataCaptureEvent} + * This allows to the {@link CdcConsumer} to continue consumption of the {@link CdcEvent} * from the last saved offset in case of fail or restart. * - * @see ChangeDataCaptureConsumer#onEvents(Iterator) - * @see ChangeDataCapture + * @see CdcConsumer#onEvents(Iterator) + * @see CdcMain */ -public class ChangeDataCaptureConsumerState { +public class CdcConsumerState { /** */ public static final String STATE_FILE_NAME = "cdc-state" + FILE_SUFFIX; @@ -59,7 +59,7 @@ public class ChangeDataCaptureConsumerState { /** * @param stateDir State directory. */ - public ChangeDataCaptureConsumerState(Path stateDir) { + public CdcConsumerState(Path stateDir) { state = stateDir.resolve(STATE_FILE_NAME); tmp = stateDir.resolve(STATE_FILE_NAME + TMP_SUFFIX); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureEventImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcEventImpl.java similarity index 86% rename from modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureEventImpl.java rename to modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcEventImpl.java index ff4ceec323da6..a12aa0e388a29 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureEventImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcEventImpl.java @@ -18,18 +18,18 @@ package org.apache.ignite.internal.cdc; import org.apache.ignite.cache.CacheEntryVersion; -import org.apache.ignite.cdc.ChangeDataCaptureConsumer; -import org.apache.ignite.cdc.ChangeDataCaptureEvent; +import org.apache.ignite.cdc.CdcConsumer; +import org.apache.ignite.cdc.CdcEvent; import org.apache.ignite.internal.util.typedef.internal.S; /** * Event of single entry change. * Instance presents new value of modified entry. * - * @see ChangeDataCapture - * @see ChangeDataCaptureConsumer + * @see CdcMain + * @see CdcConsumer */ -public class ChangeDataCaptureEventImpl implements ChangeDataCaptureEvent { +public class CdcEventImpl implements CdcEvent { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -59,7 +59,7 @@ public class ChangeDataCaptureEventImpl implements ChangeDataCaptureEvent { * @param ord Order of the entry change. * @param cacheId Cache id. */ - public ChangeDataCaptureEventImpl(Object key, Object val, boolean primary, int part, + public CdcEventImpl(Object key, Object val, boolean primary, int part, CacheEntryVersion ord, int cacheId) { this.key = key; this.val = val; @@ -101,6 +101,6 @@ public ChangeDataCaptureEventImpl(Object key, Object val, boolean primary, int p /** {@inheritDoc} */ @Override public String toString() { - return S.toString(ChangeDataCaptureEventImpl.class, this); + return S.toString(CdcEventImpl.class, this); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureFileLockHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcFileLockHolder.java similarity index 88% rename from modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureFileLockHolder.java rename to modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcFileLockHolder.java index 5d91b145def66..093e23513ed17 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCaptureFileLockHolder.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcFileLockHolder.java @@ -24,10 +24,10 @@ /** * Lock file holder for Change Data Capture application. * - * @see ChangeDataCapture - * @see ChangeDataCaptureConsumerState + * @see CdcMain + * @see CdcConsumerState */ -public class ChangeDataCaptureFileLockHolder extends FileLockHolder { +public class CdcFileLockHolder extends FileLockHolder { /** Consumer ID. */ private final String consumerId; @@ -35,7 +35,7 @@ public class ChangeDataCaptureFileLockHolder extends FileLockHolder { * @param rootDir Root directory for lock file. * @param log Log. */ - public ChangeDataCaptureFileLockHolder(String rootDir, String consumerId, IgniteLogger log) { + public CdcFileLockHolder(String rootDir, String consumerId, IgniteLogger log) { super(rootDir, log); this.consumerId = consumerId; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCapture.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java similarity index 89% rename from modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCapture.java rename to modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java index 9ff6a65f509ad..2ec1ecccf4239 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/ChangeDataCapture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java @@ -32,9 +32,9 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.cdc.ChangeDataCaptureConfiguration; -import org.apache.ignite.cdc.ChangeDataCaptureConsumer; -import org.apache.ignite.cdc.ChangeDataCaptureEvent; +import org.apache.ignite.cdc.CdcConfiguration; +import org.apache.ignite.cdc.CdcConsumer; +import org.apache.ignite.cdc.CdcEvent; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInterruptedCheckedException; @@ -55,7 +55,7 @@ import org.apache.ignite.resources.LoggerResource; import org.apache.ignite.resources.SpringApplicationContextResource; import org.apache.ignite.resources.SpringResource; -import org.apache.ignite.startup.cmdline.ChangeDataCaptureCommandLineStartup; +import org.apache.ignite.startup.cmdline.CdcCommandLineStartup; import static org.apache.ignite.internal.IgniteKernal.NL; import static org.apache.ignite.internal.IgniteKernal.SITE; @@ -67,23 +67,23 @@ /** * Change Data Capture (CDC) application. * The application runs independently of Ignite node process and provides the ability - * for the {@link ChangeDataCaptureConsumer} to consume events({@link ChangeDataCaptureEvent}) from WAL segments. - * The user should provide {@link ChangeDataCaptureConsumer} implementation with custom consumption logic. + * for the {@link CdcConsumer} to consume events({@link CdcEvent}) from WAL segments. + * The user should provide {@link CdcConsumer} implementation with custom consumption logic. * - * Ignite node should be explicitly configured for using {@link ChangeDataCapture}. + * Ignite node should be explicitly configured for using {@link CdcMain}. *
      - *
    1. Set {@link DataStorageConfiguration#setChangeDataCaptureEnabled(boolean)} to true.
    2. - *
    3. Optional: Set {@link DataStorageConfiguration#setChangeDataCaptureWalPath(String)} to path to the directory + *
    4. Set {@link DataStorageConfiguration#setCdcEnabled(boolean)} to true.
    5. + *
    6. Optional: Set {@link DataStorageConfiguration#setCdcWalPath(String)} to path to the directory * to store WAL segments for CDC.
    7. *
    8. Optional: Set {@link DataStorageConfiguration#setWalForceArchiveTimeout(long)} to configure timeout for * force WAL rollover, so new events will be available for consumptions with the predicted time.
    9. *
    * - * When {@link DataStorageConfiguration#getChangeDataCaptureWalPath()} is true then Ignite node on each WAL segment + * When {@link DataStorageConfiguration#getCdcWalPath()} is true then Ignite node on each WAL segment * rollover creates hard link to archive WAL segment in - * {@link DataStorageConfiguration#getChangeDataCaptureWalPath()} directory. {@link ChangeDataCapture} application takes + * {@link DataStorageConfiguration#getCdcWalPath()} directory. {@link CdcMain} application takes * segment file and consumes events from it. - * After successful consumption (see {@link ChangeDataCaptureConsumer#onEvents(Iterator)}) WAL segment will be deleted + * After successful consumption (see {@link CdcConsumer#onEvents(Iterator)}) WAL segment will be deleted * from directory. * * Several Ignite nodes can be started on the same host. @@ -99,14 +99,14 @@ *
  • Infinitely waits for new available segment and processes it.
  • * * - * @see DataStorageConfiguration#setChangeDataCaptureEnabled(boolean) - * @see DataStorageConfiguration#setChangeDataCaptureWalPath(String) + * @see DataStorageConfiguration#setCdcEnabled(boolean) + * @see DataStorageConfiguration#setCdcWalPath(String) * @see DataStorageConfiguration#setWalForceArchiveTimeout(long) - * @see ChangeDataCaptureCommandLineStartup - * @see ChangeDataCaptureConsumer + * @see CdcCommandLineStartup + * @see CdcConsumer * @see DataStorageConfiguration#DFLT_WAL_CDC_PATH */ -public class ChangeDataCapture implements Runnable { +public class CdcMain implements Runnable { /** */ public static final String ERR_MSG = "Persistence disabled. Capture Data Change can't run!"; @@ -120,7 +120,7 @@ public class ChangeDataCapture implements Runnable { private final GridSpringResourceContext ctx; /** Change Data Capture configuration. */ - private final ChangeDataCaptureConfiguration cdcCfg; + private final CdcConfiguration cdcCfg; /** WAL iterator factory. */ private final IgniteWalIteratorFactory factory; @@ -141,7 +141,7 @@ public class ChangeDataCapture implements Runnable { private File marshaller; /** Change Data Capture state. */ - private ChangeDataCaptureConsumerState state; + private CdcConsumerState state; /** Save state to start from. */ private WALPointer initState; @@ -157,10 +157,10 @@ public class ChangeDataCapture implements Runnable { * @param ctx Spring resource context. * @param cdcCfg Change Data Capture configuration. */ - public ChangeDataCapture( + public CdcMain( IgniteConfiguration igniteCfg, GridSpringResourceContext ctx, - ChangeDataCaptureConfiguration cdcCfg) { + CdcConfiguration cdcCfg) { this.igniteCfg = new IgniteConfiguration(igniteCfg); this.ctx = ctx; this.cdcCfg = cdcCfg; @@ -204,7 +204,7 @@ public void runX() throws Exception { throw new IllegalArgumentException(ERR_MSG); } - PdsFolderSettings settings = + PdsFolderSettings settings = new PdsFolderResolver<>(igniteCfg, log, null, this::tryLock).resolve(); if (settings == null) { @@ -212,7 +212,7 @@ public void runX() throws Exception { "[workDir=" + igniteCfg.getWorkDirectory() + ", consistentId=" + igniteCfg.getConsistentId() + ']'); } - ChangeDataCaptureFileLockHolder lock = settings.getLockedFileLockHolder(); + CdcFileLockHolder lock = settings.getLockedFileLockHolder(); if (lock == null) { File consIdDir = new File(settings.persistentStoreRootPath(), settings.folderName()); @@ -239,7 +239,7 @@ public void runX() throws Exception { ackAsciiLogo(); - state = new ChangeDataCaptureConsumerState(cdcDir.resolve(STATE_DIR)); + state = new CdcConsumerState(cdcDir.resolve(STATE_DIR)); initState = state.load(); @@ -393,19 +393,19 @@ private void consumeSegment(Path segment) { * @param dbStoreDirWithSubdirectory Root PDS directory. * @return Lock or null if lock failed. */ - private ChangeDataCaptureFileLockHolder tryLock(File dbStoreDirWithSubdirectory) { + private CdcFileLockHolder tryLock(File dbStoreDirWithSubdirectory) { if (!dbStoreDirWithSubdirectory.exists()) { log.warning("DB store directory not exists [dir=" + dbStoreDirWithSubdirectory + ']'); return null; } - File cdcRoot = new File(igniteCfg.getDataStorageConfiguration().getChangeDataCaptureWalPath()); + File cdcRoot = new File(igniteCfg.getDataStorageConfiguration().getCdcWalPath()); if (!cdcRoot.isAbsolute()) { cdcRoot = new File( igniteCfg.getWorkDirectory(), - igniteCfg.getDataStorageConfiguration().getChangeDataCaptureWalPath() + igniteCfg.getDataStorageConfiguration().getCdcWalPath() ); } @@ -427,7 +427,7 @@ private ChangeDataCaptureFileLockHolder tryLock(File dbStoreDirWithSubdirectory) this.cdcDir = cdcDir; - ChangeDataCaptureFileLockHolder lock = new ChangeDataCaptureFileLockHolder(cdcDir.toString(), "cdc.lock", log); + CdcFileLockHolder lock = new CdcFileLockHolder(cdcDir.toString(), "cdc.lock", log); try { lock.tryLock(cdcCfg.getLockTimeout()); @@ -481,7 +481,7 @@ public void stop() { } /** */ - private void injectResources(ChangeDataCaptureConsumer dataConsumer) throws IgniteCheckedException { + private void injectResources(CdcConsumer dataConsumer) throws IgniteCheckedException { GridResourceIoc ioc = new GridResourceIoc(); ioc.inject( diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java index 1b28f2b1b0fbb..38c2938399647 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java @@ -22,8 +22,8 @@ import java.util.NoSuchElementException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.cdc.ChangeDataCaptureConsumer; -import org.apache.ignite.cdc.ChangeDataCaptureEvent; +import org.apache.ignite.cdc.CdcConsumer; +import org.apache.ignite.cdc.CdcEvent; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; import org.apache.ignite.internal.pagemem.wal.record.DataRecord; import org.apache.ignite.internal.pagemem.wal.record.UnwrappedDataEntry; @@ -38,17 +38,17 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE; /** - * Transform {@link DataEntry} to {@link ChangeDataCaptureEvent} and sends it to {@link ChangeDataCaptureConsumer}. + * Transform {@link DataEntry} to {@link CdcEvent} and sends it to {@link CdcConsumer}. * - * @see ChangeDataCapture - * @see ChangeDataCaptureConsumer + * @see CdcMain + * @see CdcConsumer */ public class WalRecordsConsumer { /** Ignite logger. */ private final IgniteLogger log; /** Data change events consumer. */ - private final ChangeDataCaptureConsumer consumer; + private final CdcConsumer consumer; /** Operations types we interested in. */ private static final EnumSet OPERATIONS_TYPES = EnumSet.of(CREATE, UPDATE, DELETE, TRANSFORM); @@ -69,7 +69,7 @@ public class WalRecordsConsumer { * @param consumer User provided CDC consumer. * @param log Logger. */ - public WalRecordsConsumer(ChangeDataCaptureConsumer consumer, IgniteLogger log) { + public WalRecordsConsumer(CdcConsumer consumer, IgniteLogger log) { this.consumer = consumer; this.log = log; } @@ -83,9 +83,9 @@ public WalRecordsConsumer(ChangeDataCaptureConsumer consumer, IgniteLogger log) * @return {@code True} if current offset in WAL should be commited. */ public boolean onRecords(Iterator recs) { - Iterator evts = new Iterator() { + Iterator evts = new Iterator() { /** */ - private Iterator entries; + private Iterator entries; @Override public boolean hasNext() { advance(); @@ -93,7 +93,7 @@ public boolean onRecords(Iterator recs) { return hasCurrent(); } - @Override public ChangeDataCaptureEvent next() { + @Override public CdcEvent next() { advance(); if (!hasCurrent()) @@ -122,10 +122,10 @@ private boolean hasCurrent() { } /** */ - private ChangeDataCaptureEvent transform(DataEntry e) { + private CdcEvent transform(DataEntry e) { UnwrappedDataEntry ue = (UnwrappedDataEntry)e; - return new ChangeDataCaptureEventImpl( + return new CdcEventImpl( ue.unwrappedKey(), ue.unwrappedValue(), (e.flags() & DataEntry.PRIMARY_FLAG) != 0, @@ -163,7 +163,7 @@ public void stop() { } /** @return Change Data Capture Consumer. */ - public ChangeDataCaptureConsumer consumer() { + public CdcConsumer consumer() { return consumer; } 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 f0761fa6c3297..673accd058e99 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 @@ -469,9 +469,9 @@ public void setFileIOFactory(FileIOFactory ioFactory) { "write ahead log archive directory" ); - if (dsCfg.isChangeDataCaptureEnabled()) { + if (dsCfg.isCdcEnabled()) { walCdcDir = initDirectory( - dsCfg.getChangeDataCaptureWalPath(), + dsCfg.getCdcWalPath(), DataStorageConfiguration.DFLT_WAL_CDC_PATH, resolveFolders.folderName(), "change data capture directory" @@ -2036,7 +2036,7 @@ public SegmentArchiveResult archiveSegment(long absIdx) throws StorageException Files.move(dstTmpFile.toPath(), dstFile.toPath()); - if (dsCfg.isChangeDataCaptureEnabled()) + if (dsCfg.isCdcEnabled()) Files.createLink(walCdcDir.toPath().resolve(dstFile.getName()), dstFile.toPath()); if (mode != WALMode.NONE) { diff --git a/modules/core/src/main/java/org/apache/ignite/startup/cmdline/ChangeDataCaptureCommandLineStartup.java b/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CdcCommandLineStartup.java similarity index 91% rename from modules/core/src/main/java/org/apache/ignite/startup/cmdline/ChangeDataCaptureCommandLineStartup.java rename to modules/core/src/main/java/org/apache/ignite/startup/cmdline/CdcCommandLineStartup.java index 7669f02d81967..da5c84f7729d7 100644 --- a/modules/core/src/main/java/org/apache/ignite/startup/cmdline/ChangeDataCaptureCommandLineStartup.java +++ b/modules/core/src/main/java/org/apache/ignite/startup/cmdline/CdcCommandLineStartup.java @@ -22,9 +22,9 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteSystemProperties; -import org.apache.ignite.cdc.ChangeDataCaptureConfiguration; -import org.apache.ignite.cdc.ChangeDataCaptureLoader; -import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.cdc.CdcConfiguration; +import org.apache.ignite.cdc.CdcLoader; +import org.apache.ignite.internal.cdc.CdcMain; import org.apache.ignite.internal.util.spring.IgniteSpringHelper; import org.apache.ignite.internal.util.typedef.X; import org.jetbrains.annotations.Nullable; @@ -46,9 +46,9 @@ * this startup and you can use them as an example. *

    * - * @see ChangeDataCapture + * @see CdcMain */ -public class ChangeDataCaptureCommandLineStartup { +public class CdcCommandLineStartup { /** Quite log flag. */ private static final boolean QUITE = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_QUIET); @@ -80,10 +80,10 @@ public static void main(String[] args) { if (args.length > 0 && args[0].charAt(0) == '-') exit("Invalid arguments: " + args[0], true, -1); - AtomicReference cdc = new AtomicReference<>(); + AtomicReference cdc = new AtomicReference<>(); try { - cdc.set(ChangeDataCaptureLoader.loadChangeDataCapture(args[0])); + cdc.set(CdcLoader.loadCdc(args[0])); if (!IgniteSystemProperties.getBoolean(IGNITE_NO_SHUTDOWN_HOOK, false)) { Runtime.getRuntime().addShutdownHook(new Thread("cdc-shutdown-hook") { @@ -121,16 +121,16 @@ public static void main(String[] args) { * @return CDC consumer defined in spring configuration. * @throws IgniteCheckedException in case of load error. */ - private static ChangeDataCaptureConfiguration consumerConfig( + private static CdcConfiguration consumerConfig( URL cfgUrl, IgniteSpringHelper spring ) throws IgniteCheckedException { - Map, Object> cdcCfgs = spring.loadBeans(cfgUrl, ChangeDataCaptureConfiguration.class); + Map, Object> cdcCfgs = spring.loadBeans(cfgUrl, CdcConfiguration.class); if (cdcCfgs == null || cdcCfgs.size() != 1) exit("Exact 1 CaptureDataChangeConfiguration configuration should be defined", false, 1); - return (ChangeDataCaptureConfiguration)cdcCfgs.values().iterator().next(); + return (CdcConfiguration)cdcCfgs.values().iterator().next(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/cdc/AbstractChangeDataCaptureTest.java b/modules/core/src/test/java/org/apache/ignite/cdc/AbstractCdcTest.java similarity index 83% rename from modules/core/src/test/java/org/apache/ignite/cdc/AbstractChangeDataCaptureTest.java rename to modules/core/src/test/java/org/apache/ignite/cdc/AbstractCdcTest.java index 9c8b95d350578..e250f68b1be7f 100644 --- a/modules/core/src/test/java/org/apache/ignite/cdc/AbstractChangeDataCaptureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cdc/AbstractCdcTest.java @@ -28,20 +28,20 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.cdc.CdcMain; import org.apache.ignite.internal.util.typedef.CI3; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.ChangeEventType.DELETE; -import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.ChangeEventType.UPDATE; +import static org.apache.ignite.cdc.AbstractCdcTest.ChangeEventType.DELETE; +import static org.apache.ignite.cdc.AbstractCdcTest.ChangeEventType.UPDATE; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheId; import static org.apache.ignite.testframework.GridTestUtils.runAsync; import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** */ -public abstract class AbstractChangeDataCaptureTest extends GridCommonAbstractTest { +public abstract class AbstractCdcTest extends GridCommonAbstractTest { /** */ public static final String JOHN = "John Connor"; @@ -63,10 +63,10 @@ public abstract class AbstractChangeDataCaptureTest extends GridCommonAbstractTe /** */ public void addAndWaitForConsumption( UserCdcConsumer cnsmr, - ChangeDataCapture cdc, - IgniteCache cache, - IgniteCache txCache, - CI3, Integer, Integer> addData, + CdcMain cdc, + IgniteCache cache, + IgniteCache txCache, + CI3, Integer, Integer> addData, int from, int to, long timeout @@ -99,7 +99,7 @@ public void addAndWaitForConsumption( public boolean waitForSize( int expSz, String cacheName, - ChangeDataCaptureSelfTest.ChangeEventType evtType, + CdcSelfTest.ChangeEventType evtType, long timeout, TestCdcConsumer... cnsmrs ) throws IgniteInterruptedCheckedException { @@ -112,8 +112,8 @@ public boolean waitForSize( } /** */ - public ChangeDataCaptureConfiguration cdcConfig(ChangeDataCaptureConsumer cnsmr) { - ChangeDataCaptureConfiguration cdcCfg = new ChangeDataCaptureConfiguration(); + public CdcConfiguration cdcConfig(CdcConsumer cnsmr) { + CdcConfiguration cdcCfg = new CdcConfiguration(); cdcCfg.setConsumer(cnsmr); cdcCfg.setKeepBinary(false); @@ -122,7 +122,7 @@ public ChangeDataCaptureConfiguration cdcConfig(ChangeDataCaptureConsumer cnsmr) } /** */ - public abstract static class TestCdcConsumer implements ChangeDataCaptureConsumer { + public abstract static class TestCdcConsumer implements CdcConsumer { /** Keys */ final ConcurrentMap, List> data = new ConcurrentHashMap<>(); @@ -140,7 +140,7 @@ public abstract static class TestCdcConsumer implements ChangeDataCaptureCons } /** {@inheritDoc} */ - @Override public boolean onEvents(Iterator evts) { + @Override public boolean onEvents(Iterator evts) { evts.forEachRemaining(evt -> { if (!evt.primary()) return; @@ -156,10 +156,10 @@ public abstract static class TestCdcConsumer implements ChangeDataCaptureCons } /** */ - public abstract void checkEvent(ChangeDataCaptureEvent evt); + public abstract void checkEvent(CdcEvent evt); /** */ - public abstract T extract(ChangeDataCaptureEvent evt); + public abstract T extract(CdcEvent evt); /** */ protected boolean commit() { @@ -180,7 +180,7 @@ public boolean stopped() { /** */ public static class UserCdcConsumer extends TestCdcConsumer { /** {@inheritDoc} */ - @Override public void checkEvent(ChangeDataCaptureEvent evt) { + @Override public void checkEvent(CdcEvent evt) { assertNull(evt.version().otherClusterVersion()); if (evt.value() == null) @@ -193,7 +193,7 @@ public static class UserCdcConsumer extends TestCdcConsumer { } /** {@inheritDoc} */ - @Override public Integer extract(ChangeDataCaptureEvent evt) { + @Override public Integer extract(CdcEvent evt) { return (Integer)evt.key(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureCacheVersionTest.java b/modules/core/src/test/java/org/apache/ignite/cdc/CdcCacheVersionTest.java similarity index 93% rename from modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureCacheVersionTest.java rename to modules/core/src/test/java/org/apache/ignite/cdc/CdcCacheVersionTest.java index 640adb97e3c26..30fd1e9332e13 100644 --- a/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureCacheVersionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cdc/CdcCacheVersionTest.java @@ -29,7 +29,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.cdc.CdcMain; import org.apache.ignite.internal.processors.cache.CacheConflictResolutionManager; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.CacheObjectImpl; @@ -56,7 +56,7 @@ import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** */ -public class ChangeDataCaptureCacheVersionTest extends AbstractChangeDataCaptureTest { +public class CdcCacheVersionTest extends AbstractCdcTest { /** */ public static final String FOR_OTHER_CLUSTER_ID = "for-other-cluster-id"; @@ -74,7 +74,7 @@ public class ChangeDataCaptureCacheVersionTest extends AbstractChangeDataCapture IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); cfg.setDataStorageConfiguration(new DataStorageConfiguration() - .setChangeDataCaptureEnabled(true) + .setCdcEnabled(true) .setWalForceArchiveTimeout(WAL_ARCHIVE_TIMEOUT) .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true))); @@ -112,13 +112,13 @@ public void testReadAllKeysFromOtherCluster() throws Exception { ign.cluster().state(ACTIVE); UserCdcConsumer cnsmr = new UserCdcConsumer() { - @Override public void checkEvent(ChangeDataCaptureEvent evt) { + @Override public void checkEvent(CdcEvent evt) { assertEquals(DFLT_CLUSTER_ID, evt.version().clusterId()); assertEquals(OTHER_CLUSTER_ID, evt.version().otherClusterVersion().clusterId()); } }; - ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcConfig(cnsmr)); + CdcMain cdc = new CdcMain(cfg, null, cdcConfig(cnsmr)); IgniteCache cache = ign.getOrCreateCache(FOR_OTHER_CLUSTER_ID); @@ -136,10 +136,10 @@ public void testOrderIncrease() throws Exception { AtomicLong updCntr = new AtomicLong(0); - ChangeDataCaptureConsumer cnsmr = new ChangeDataCaptureConsumer() { + CdcConsumer cnsmr = new CdcConsumer() { private long order = -1; - @Override public boolean onEvents(Iterator evts) { + @Override public boolean onEvents(Iterator evts) { evts.forEachRemaining(evt -> { assertEquals(KEY_TO_UPD, evt.key()); @@ -162,7 +162,7 @@ public void testOrderIncrease() throws Exception { } }; - ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcConfig(cnsmr)); + CdcMain cdc = new CdcMain(cfg, null, cdcConfig(cnsmr)); IgniteCache cache = ign.getOrCreateCache("my-cache"); diff --git a/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cdc/CdcSelfTest.java similarity index 88% rename from modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureSelfTest.java rename to modules/core/src/test/java/org/apache/ignite/cdc/CdcSelfTest.java index 4be9e6f73f85c..9f1e55f6b2797 100644 --- a/modules/core/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cdc/CdcSelfTest.java @@ -37,14 +37,14 @@ import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.cdc.CdcMain; import org.apache.ignite.internal.util.typedef.F; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.ChangeEventType.DELETE; -import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.ChangeEventType.UPDATE; +import static org.apache.ignite.cdc.AbstractCdcTest.ChangeEventType.DELETE; +import static org.apache.ignite.cdc.AbstractCdcTest.ChangeEventType.UPDATE; import static org.apache.ignite.cluster.ClusterState.ACTIVE; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheId; import static org.apache.ignite.testframework.GridTestUtils.runAsync; @@ -52,7 +52,7 @@ /** */ @RunWith(Parameterized.class) -public class ChangeDataCaptureSelfTest extends AbstractChangeDataCaptureTest { +public class CdcSelfTest extends AbstractCdcTest { /** */ public static final String TX_CACHE_NAME = "tx-cache"; @@ -91,7 +91,7 @@ public static Collection parameters() { cfg.setConsistentId(consistentId); cfg.setDataStorageConfiguration(new DataStorageConfiguration() - .setChangeDataCaptureEnabled(true) + .setCdcEnabled(true) .setWalMode(walMode) .setWalForceArchiveTimeout(WAL_ARCHIVE_TIMEOUT) .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true))); @@ -111,7 +111,7 @@ public void testReadAllKeys() throws Exception { // Read one record per call. readAll(new UserCdcConsumer() { - @Override public boolean onEvents(Iterator evts) { + @Override public boolean onEvents(Iterator evts) { super.onEvents(Collections.singleton(evts.next()).iterator()); return false; @@ -120,7 +120,7 @@ public void testReadAllKeys() throws Exception { // Read one record per call and commit. readAll(new UserCdcConsumer() { - @Override public boolean onEvents(Iterator evts) { + @Override public boolean onEvents(Iterator evts) { super.onEvents(Collections.singleton(evts.next()).iterator()); return true; @@ -136,7 +136,7 @@ private void readAll(UserCdcConsumer cnsmr) throws Exception { ign.cluster().state(ACTIVE); - ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcConfig(cnsmr)); + CdcMain cdc = new CdcMain(cfg, null, cdcConfig(cnsmr)); IgniteCache cache = ign.getOrCreateCache(DEFAULT_CACHE_NAME); IgniteCache txCache = ign.getOrCreateCache(TX_CACHE_NAME); @@ -146,7 +146,7 @@ private void readAll(UserCdcConsumer cnsmr) throws Exception { cdc, cache, txCache, - ChangeDataCaptureSelfTest::addData, + CdcSelfTest::addData, 0, KEYS_CNT + 3, getTestTimeout() @@ -180,7 +180,7 @@ public void testReadBeforeGracefulShutdown() throws Exception { CountDownLatch startProcEvts = new CountDownLatch(1); UserCdcConsumer cnsmr = new UserCdcConsumer() { - @Override public boolean onEvents(Iterator evts) { + @Override public boolean onEvents(Iterator evts) { cnsmrStarted.countDown(); try { @@ -194,7 +194,7 @@ public void testReadBeforeGracefulShutdown() throws Exception { } }; - ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcConfig(cnsmr)); + CdcMain cdc = new CdcMain(cfg, null, cdcConfig(cnsmr)); runAsync(cdc); @@ -246,8 +246,8 @@ public void testMultiNodeConsumption() throws Exception { IgniteConfiguration cfg1 = ign1.configuration(); IgniteConfiguration cfg2 = ign2.configuration(); - ChangeDataCapture cdc1 = new ChangeDataCapture(cfg1, null, cdcConfig(cnsmr1)); - ChangeDataCapture cdc2 = new ChangeDataCapture(cfg2, null, cdcConfig(cnsmr2)); + CdcMain cdc1 = new CdcMain(cfg1, null, cdcConfig(cnsmr1)); + CdcMain cdc2 = new CdcMain(cfg2, null, cdcConfig(cnsmr2)); IgniteInternalFuture fut1 = runAsync(cdc1); IgniteInternalFuture fut2 = runAsync(cdc2); @@ -291,8 +291,8 @@ public void testCdcSingleton() throws Exception { UserCdcConsumer cnsmr1 = new UserCdcConsumer(); UserCdcConsumer cnsmr2 = new UserCdcConsumer(); - IgniteInternalFuture fut1 = runAsync(new ChangeDataCapture(ign.configuration(), null, cdcConfig(cnsmr1))); - IgniteInternalFuture fut2 = runAsync(new ChangeDataCapture(ign.configuration(), null, cdcConfig(cnsmr2))); + IgniteInternalFuture fut1 = runAsync(new CdcMain(ign.configuration(), null, cdcConfig(cnsmr1))); + IgniteInternalFuture fut2 = runAsync(new CdcMain(ign.configuration(), null, cdcConfig(cnsmr2))); assertTrue(waitForCondition(() -> fut1.isDone() || fut2.isDone(), getTestTimeout())); @@ -330,7 +330,7 @@ public void testReReadWhenStateWasNotStored() throws Exception { } }; - ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcConfig(cnsmr)); + CdcMain cdc = new CdcMain(cfg, null, cdcConfig(cnsmr)); IgniteInternalFuture fut = runAsync(cdc); @@ -347,7 +347,7 @@ public void testReReadWhenStateWasNotStored() throws Exception { int half = KEYS_CNT / 2; UserCdcConsumer cnsmr = new UserCdcConsumer() { - @Override public boolean onEvents(Iterator evts) { + @Override public boolean onEvents(Iterator evts) { if (consumeHalf.get() && F.size(data(UPDATE, cacheId(DEFAULT_CACHE_NAME))) == half) { // This means that state committed as a result of the previous call. halfCommitted.set(true); @@ -356,7 +356,7 @@ public void testReReadWhenStateWasNotStored() throws Exception { } while (evts.hasNext()) { - ChangeDataCaptureEvent evt = evts.next(); + CdcEvent evt = evts.next(); if (!evt.primary()) continue; @@ -374,7 +374,7 @@ public void testReReadWhenStateWasNotStored() throws Exception { } }; - ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcConfig(cnsmr)); + CdcMain cdc = new CdcMain(cfg, null, cdcConfig(cnsmr)); IgniteInternalFuture fut = runAsync(cdc); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java index cb928c42eb8aa..afd756dd5a84d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java @@ -45,6 +45,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.IgniteEvents; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheRebalanceMode; diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java index dc7635e1b6231..f2e1cfff6e451 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java @@ -20,8 +20,8 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -import org.apache.ignite.cdc.ChangeDataCaptureCacheVersionTest; -import org.apache.ignite.cdc.ChangeDataCaptureSelfTest; +import org.apache.ignite.cdc.CdcCacheVersionTest; +import org.apache.ignite.cdc.CdcSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.HistoricalRebalanceHeuristicsTest; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.HistoricalRebalanceTwoPartsInDifferentCheckpointsTest; import org.apache.ignite.internal.processors.cache.persistence.IgniteDataStorageMetricsSelfTest; @@ -212,8 +212,8 @@ public static void addRealPageStoreTests(List> suite, Collection GridTestUtils.addTestIfNeeded(suite, IgniteWalReplayingAfterRestartTest.class, ignoredTests); // CDC tests. - GridTestUtils.addTestIfNeeded(suite, ChangeDataCaptureSelfTest.class, ignoredTests); - GridTestUtils.addTestIfNeeded(suite, ChangeDataCaptureCacheVersionTest.class, ignoredTests); + GridTestUtils.addTestIfNeeded(suite, CdcSelfTest.class, ignoredTests); + GridTestUtils.addTestIfNeeded(suite, CdcCacheVersionTest.class, ignoredTests); // new style folders with generated consistent ID test GridTestUtils.addTestIfNeeded(suite, IgniteUidAsConsistentIdMigrationTest.class, ignoredTests); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlChangeDataCaptureTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlCdcTest.java similarity index 87% rename from modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlChangeDataCaptureTest.java rename to modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlCdcTest.java index 0c46af284f686..1e24d3b96bf00 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlChangeDataCaptureTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlCdcTest.java @@ -20,9 +20,9 @@ import java.util.List; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.cache.query.SqlFieldsQuery; -import org.apache.ignite.cdc.AbstractChangeDataCaptureTest; -import org.apache.ignite.cdc.ChangeDataCaptureConfiguration; -import org.apache.ignite.cdc.ChangeDataCaptureEvent; +import org.apache.ignite.cdc.AbstractCdcTest; +import org.apache.ignite.cdc.CdcConfiguration; +import org.apache.ignite.cdc.CdcEvent; import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; @@ -30,14 +30,14 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.junit.Test; -import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.ChangeEventType.DELETE; -import static org.apache.ignite.cdc.AbstractChangeDataCaptureTest.ChangeEventType.UPDATE; +import static org.apache.ignite.cdc.AbstractCdcTest.ChangeEventType.DELETE; +import static org.apache.ignite.cdc.AbstractCdcTest.ChangeEventType.UPDATE; import static org.apache.ignite.cluster.ClusterState.ACTIVE; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheId; import static org.apache.ignite.testframework.GridTestUtils.runAsync; /** */ -public class SqlChangeDataCaptureTest extends AbstractChangeDataCaptureTest { +public class SqlCdcTest extends AbstractCdcTest { /** */ private static final String SARAH = "Sarah Connor"; @@ -58,7 +58,7 @@ public class SqlChangeDataCaptureTest extends AbstractChangeDataCaptureTest { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); cfg.setDataStorageConfiguration(new DataStorageConfiguration() - .setChangeDataCaptureEnabled(true) + .setCdcEnabled(true) .setWalForceArchiveTimeout(WAL_ARCHIVE_TIMEOUT) .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true))); @@ -76,11 +76,11 @@ public void testReadAllSQLRows() throws Exception { BinaryCdcConsumer cnsmr = new BinaryCdcConsumer(); - ChangeDataCaptureConfiguration cdcCfg = new ChangeDataCaptureConfiguration(); + CdcConfiguration cdcCfg = new CdcConfiguration(); cdcCfg.setConsumer(cnsmr); - ChangeDataCapture cdc = new ChangeDataCapture(cfg, null, cdcCfg); + CdcMain cdc = new CdcMain(cfg, null, cdcCfg); IgniteInternalFuture fut = runAsync(cdc); @@ -133,9 +133,9 @@ public void testReadAllSQLRows() throws Exception { } /** */ - public static class BinaryCdcConsumer extends TestCdcConsumer { + public static class BinaryCdcConsumer extends TestCdcConsumer { /** {@inheritDoc} */ - @Override public void checkEvent(ChangeDataCaptureEvent evt) { + @Override public void checkEvent(CdcEvent evt) { if (evt.value() == null) return; @@ -167,7 +167,7 @@ public static class BinaryCdcConsumer extends TestCdcConsumer - + diff --git a/modules/spring/src/test/config/cdc/correct-cdc-config.xml b/modules/spring/src/test/config/cdc/correct-cdc-config.xml index 3a63298f9d58f..7ee68cbff81cf 100644 --- a/modules/spring/src/test/config/cdc/correct-cdc-config.xml +++ b/modules/spring/src/test/config/cdc/correct-cdc-config.xml @@ -32,14 +32,14 @@ - + - + - + diff --git a/modules/spring/src/test/config/cdc/double-cdc-config.xml b/modules/spring/src/test/config/cdc/double-cdc-config.xml index 4bd7d5f262759..c99a24187957f 100644 --- a/modules/spring/src/test/config/cdc/double-cdc-config.xml +++ b/modules/spring/src/test/config/cdc/double-cdc-config.xml @@ -26,5 +26,5 @@ http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd"> - + diff --git a/modules/spring/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureConfigurationTest.java b/modules/spring/src/test/java/org/apache/ignite/cdc/CdcConfigurationTest.java similarity index 81% rename from modules/spring/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureConfigurationTest.java rename to modules/spring/src/test/java/org/apache/ignite/cdc/CdcConfigurationTest.java index 14da7c73c14e7..1ed624441b6bd 100644 --- a/modules/spring/src/test/java/org/apache/ignite/cdc/ChangeDataCaptureConfigurationTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/cdc/CdcConfigurationTest.java @@ -25,7 +25,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.cdc.ChangeDataCapture; +import org.apache.ignite.internal.cdc.CdcMain; import org.apache.ignite.internal.cdc.WalRecordsConsumer; import org.apache.ignite.resources.LoggerResource; import org.apache.ignite.resources.SpringApplicationContextResource; @@ -35,33 +35,33 @@ import org.springframework.context.ApplicationContext; import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static org.apache.ignite.cdc.ChangeDataCaptureLoader.loadChangeDataCapture; -import static org.apache.ignite.internal.cdc.ChangeDataCapture.ERR_MSG; +import static org.apache.ignite.cdc.CdcLoader.loadCdc; +import static org.apache.ignite.internal.cdc.CdcMain.ERR_MSG; import static org.apache.ignite.testframework.GridTestUtils.assertThrows; import static org.apache.ignite.testframework.GridTestUtils.getFieldValue; import static org.apache.ignite.testframework.GridTestUtils.runAsync; /** */ -public class ChangeDataCaptureConfigurationTest extends GridCommonAbstractTest { +public class CdcConfigurationTest extends GridCommonAbstractTest { /** */ @Test public void testLoadConfig() throws Exception { assertThrows( null, - () -> loadChangeDataCapture("modules/spring/src/test/config/cdc/double-ignite-config.xml"), + () -> loadCdc("modules/spring/src/test/config/cdc/double-ignite-config.xml"), IgniteCheckedException.class, "Exact 1 IgniteConfiguration should be defined. Found 2" ); assertThrows( null, - () -> loadChangeDataCapture("modules/spring/src/test/config/cdc/double-cdc-config.xml"), + () -> loadCdc("modules/spring/src/test/config/cdc/double-cdc-config.xml"), IgniteCheckedException.class, "Exact 1 CaptureDataChangeConfiguration configuration should be defined. Found 2" ); - ChangeDataCapture cdc = - loadChangeDataCapture("modules/spring/src/test/config/cdc/cdc-config-without-persistence.xml"); + CdcMain cdc = + loadCdc("modules/spring/src/test/config/cdc/cdc-config-without-persistence.xml"); assertNotNull(cdc); @@ -71,8 +71,8 @@ public void testLoadConfig() throws Exception { /** */ @Test public void testInjectResources() throws Exception { - ChangeDataCapture cdc = - loadChangeDataCapture("modules/spring/src/test/config/cdc/correct-cdc-config.xml"); + CdcMain cdc = + loadCdc("modules/spring/src/test/config/cdc/correct-cdc-config.xml"); try (IgniteEx ign = startGrid((IgniteConfiguration)getFieldValue(cdc, "igniteCfg"))) { TestCdcConsumer cnsmr = @@ -101,7 +101,7 @@ public void testInjectResources() throws Exception { } /** */ - public static class TestCdcConsumer implements ChangeDataCaptureConsumer { + public static class TestCdcConsumer implements CdcConsumer { /** */ @LoggerResource private IgniteLogger log; @@ -128,7 +128,7 @@ public static class TestCdcConsumer implements ChangeDataCaptureConsumer { } /** {@inheritDoc} */ - @Override public boolean onEvents(Iterator events) { + @Override public boolean onEvents(Iterator events) { return false; } diff --git a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java index 24df4f7799d44..a50d0d756f995 100644 --- a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java +++ b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java @@ -22,7 +22,7 @@ import org.apache.ignite.cache.store.jdbc.CachePojoStoreXmlSelfTest; import org.apache.ignite.cache.store.jdbc.CachePojoStoreXmlWithSqlEscapeSelfTest; import org.apache.ignite.cache.store.spring.CacheSpringStoreSessionListenerSelfTest; -import org.apache.ignite.cdc.ChangeDataCaptureConfigurationTest; +import org.apache.ignite.cdc.CdcConfigurationTest; import org.apache.ignite.cluster.ClusterStateXmlPropertiesTest; import org.apache.ignite.encryption.SpringEncryptedCacheRestartClientTest; import org.apache.ignite.encryption.SpringEncryptedCacheRestartTest; @@ -85,7 +85,7 @@ ClusterStateXmlPropertiesTest.class, // CDC tests. - ChangeDataCaptureConfigurationTest.class + CdcConfigurationTest.class }) public class IgniteSpringTestSuite { } From 2ad5e396df20a605d884d3f6a359927cf2e83840 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Wed, 23 Jun 2021 15:21:42 +0300 Subject: [PATCH 22/28] ChangeDataCapture -> Cdc --- bin/ignite-cdc.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/ignite-cdc.sh b/bin/ignite-cdc.sh index 8f806a4eacfde..b04345d3991dd 100755 --- a/bin/ignite-cdc.sh +++ b/bin/ignite-cdc.sh @@ -17,7 +17,7 @@ # limitations under the License. # -export MAIN_CLASS="org.apache.ignite.startup.cmdline.ChangeDataCaptureCommandLineStartup" +export MAIN_CLASS="org.apache.ignite.startup.cmdline.CdcCommandLineStartup" if [ "${IGNITE_HOME:-}" = "" ]; then IGNITE_HOME_TMP="$(dirname "$(cd "$(dirname "$0")"; "pwd")")"; From fe923c585aafbf22c9d61b6e6e972a12704e8f43 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Fri, 25 Jun 2021 16:21:16 +0300 Subject: [PATCH 23/28] Improve logging --- .../java/org/apache/ignite/cdc/CdcLoader.java | 2 + .../apache/ignite/internal/cdc/CdcMain.java | 82 +++++++++++++------ 2 files changed, 59 insertions(+), 25 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/CdcLoader.java b/modules/core/src/main/java/org/apache/ignite/cdc/CdcLoader.java index 8a907f7d434ec..b3bc583f1fafc 100644 --- a/modules/core/src/main/java/org/apache/ignite/cdc/CdcLoader.java +++ b/modules/core/src/main/java/org/apache/ignite/cdc/CdcLoader.java @@ -26,12 +26,14 @@ import org.apache.ignite.internal.util.spring.IgniteSpringHelper; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteExperimental; import static org.apache.ignite.internal.IgniteComponentType.SPRING; /** * Utility class to load {@link CdcMain} from Spring XML configuration. */ +@IgniteExperimental public class CdcLoader { /** * Loads {@link CdcMain} from XML configuration file. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java index 2ec1ecccf4239..9b2a6d777fd6a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java @@ -37,6 +37,7 @@ import org.apache.ignite.cdc.CdcEvent; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.GridLoggerProxy; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.MarshallerContextImpl; import org.apache.ignite.internal.pagemem.wal.WALIterator; @@ -153,20 +154,20 @@ public class CdcMain implements Runnable { private final Set processedSegments = new HashSet<>(); /** - * @param igniteCfg Ignite configuration. + * @param cfg Ignite configuration. * @param ctx Spring resource context. * @param cdcCfg Change Data Capture configuration. */ public CdcMain( - IgniteConfiguration igniteCfg, + IgniteConfiguration cfg, GridSpringResourceContext ctx, CdcConfiguration cdcCfg) { - this.igniteCfg = new IgniteConfiguration(igniteCfg); + igniteCfg = new IgniteConfiguration(cfg); this.ctx = ctx; this.cdcCfg = cdcCfg; try { - initWorkDir(this.igniteCfg); + initWorkDir(igniteCfg); log = U.initLogger(igniteCfg, "ignite-cdc"); } @@ -198,6 +199,8 @@ public CdcMain( /** Runs Change Data Capture application with possible exception. */ public void runX() throws Exception { + ackAsciiLogo(); + if (!CU.isPersistenceEnabled(igniteCfg)) { log.error(ERR_MSG); @@ -235,9 +238,13 @@ public void runX() throws Exception { marshaller = MarshallerContextImpl.mappingFileStoreWorkDir(igniteCfg.getWorkDirectory()); - injectResources(consumer.consumer()); + if (log.isInfoEnabled()) { + log.info("Change Data Capture [dir=" + cdcDir + ']'); + log.info("Ignite node Binary meta [dir=" + binaryMeta + ']'); + log.info("Ignite node Marshaller [dir=" + marshaller + ']'); + } - ackAsciiLogo(); + injectResources(consumer.consumer()); state = new CdcConsumerState(cdcDir.resolve(STATE_DIR)); @@ -513,26 +520,51 @@ private void injectResources(CdcConsumer dataConsumer) throws IgniteCheckedExcep /** */ private void ackAsciiLogo() { - if (!log.isInfoEnabled()) - return; - String ver = "ver. " + ACK_VER_STR; - log.info(NL + NL + - ">>> __________ ________________ ________ _____" + NL + - ">>> / _/ ___/ |/ / _/_ __/ __/ / ___/ _ \\/ ___/" + NL + - ">>> _/ // (7 7 // / / / / _/ / /__/ // / /__ " + NL + - ">>> /___/\\___/_/|_/___/ /_/ /___/ \\___/____/\\___/ " + NL + - ">>> " + NL + - ">>> " + ver + NL + - ">>> " + COPYRIGHT + NL + - ">>> " + NL + - ">>> Ignite documentation: " + "http://" + SITE + NL + - ">>> Consumer: " + consumer.consumer().toString() + NL + - ">>> ConsistentId: " + igniteCfg.getConsistentId() + NL + - ">>> Change Data Capture: " + cdcDir + NL + - ">>> Ignite node Binary meta: " + binaryMeta + NL + - ">>> Ignite node Marshaller: " + marshaller + NL - ); + if (!log.isInfoEnabled()) { + log.info(NL + NL + + ">>> __________ ________________ ________ _____" + NL + + ">>> / _/ ___/ |/ / _/_ __/ __/ / ___/ _ \\/ ___/" + NL + + ">>> _/ // (7 7 // / / / / _/ / /__/ // / /__ " + NL + + ">>> /___/\\___/_/|_/___/ /_/ /___/ \\___/____/\\___/ " + NL + + ">>> " + NL + + ">>> " + ver + NL + + ">>> " + COPYRIGHT + NL + + ">>> " + NL + + ">>> Ignite documentation: " + "http://" + SITE + NL + + ">>> Consumer: " + consumer.consumer().toString() + NL + + ">>> ConsistentId: " + igniteCfg.getConsistentId() + NL + ); + } + + if (log.isQuiet()) { + U.quiet(false, + " __________ ________________ ________ _____", + " / _/ ___/ |/ / _/_ __/ __/ / ___/ _ \\/ ___/", + " _/ // (7 7 // / / / / _/ / /__/ // / /__ ", + "/___/\\___/_/|_/___/ /_/ /___/ \\___/____/\\___/ ", + "", + ver, + COPYRIGHT, + "", + "Ignite documentation: " + "http://" + SITE, + "Consumer: " + consumer.consumer().toString(), + "ConsistentId: " + igniteCfg.getConsistentId(), + "", + "Quiet mode."); + + String fileName = log.fileName(); + + if (fileName != null) + U.quiet(false, " ^-- Logging to file '" + fileName + '\''); + + if (log instanceof GridLoggerProxy) + U.quiet(false, " ^-- Logging by '" + ((GridLoggerProxy)log).getLoggerInfo() + '\''); + + U.quiet(false, + " ^-- To see **FULL** console log here add -DIGNITE_QUIET=false or \"-v\" to ignite-cdc.{sh|bat}", + ""); + } } } From ce2a1bd10b3b596a3be02c8ec3ec01074d32af6e Mon Sep 17 00:00:00 2001 From: Nikolay Date: Fri, 25 Jun 2021 17:15:24 +0300 Subject: [PATCH 24/28] Improve logging --- .../src/main/java/org/apache/ignite/internal/cdc/CdcMain.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java index 9b2a6d777fd6a..62b78a124b210 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java @@ -522,7 +522,7 @@ private void injectResources(CdcConsumer dataConsumer) throws IgniteCheckedExcep private void ackAsciiLogo() { String ver = "ver. " + ACK_VER_STR; - if (!log.isInfoEnabled()) { + if (log.isInfoEnabled()) { log.info(NL + NL + ">>> __________ ________________ ________ _____" + NL + ">>> / _/ ___/ |/ / _/_ __/ __/ / ___/ _ \\/ ___/" + NL + From f7176d0222df8dc2796e115c4bebffbdad7cb2e7 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Mon, 28 Jun 2021 23:53:42 +0300 Subject: [PATCH 25/28] Improve logging --- .../org/apache/ignite/internal/cdc/CdcMain.java | 16 +--------------- .../ignite/internal/util/IgniteUtils.java | 17 +++++++++++++++++ 2 files changed, 18 insertions(+), 15 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java index 62b78a124b210..fdb21952faee4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java @@ -167,7 +167,7 @@ public CdcMain( this.cdcCfg = cdcCfg; try { - initWorkDir(igniteCfg); + U.initWorkDir(igniteCfg); log = U.initLogger(igniteCfg, "ignite-cdc"); } @@ -453,20 +453,6 @@ private CdcFileLockHolder tryLock(File dbStoreDirWithSubdirectory) { } } - /** Resolves work directory. */ - private static void initWorkDir(IgniteConfiguration cfg) throws IgniteCheckedException { - String igniteHome = cfg.getIgniteHome(); - - // Set Ignite home. - if (igniteHome == null) - igniteHome = U.getIgniteHome(); - - String userProvidedWorkDir = cfg.getWorkDirectory(); - - // Correctly resolve work directory and set it back to configuration. - cfg.setWorkDirectory(U.workDirectory(userProvidedWorkDir, igniteHome)); - } - /** * @param segment WAL segment file. * @return Segment index. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index a03cffd490164..069ca3709bf10 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -4639,6 +4639,23 @@ public static void log(@Nullable IgniteLogger log, Object longMsg, Object shortM quiet(false, shortMsg); } + /** + * Resolves work directory. + * @param cfg Ignite configuration. + */ + public static void initWorkDir(IgniteConfiguration cfg) throws IgniteCheckedException { + String igniteHome = cfg.getIgniteHome(); + + // Set Ignite home. + if (igniteHome == null) + igniteHome = U.getIgniteHome(); + + String userProvidedWorkDir = cfg.getWorkDirectory(); + + // Correctly resolve work directory and set it back to configuration. + cfg.setWorkDirectory(U.workDirectory(userProvidedWorkDir, igniteHome)); + } + /** * @param cfg Ignite configuration. * @param app Application name. From a61013cb5b94f68e1e5ea31c8bbaf4617b17e40b Mon Sep 17 00:00:00 2001 From: Nikolay Date: Tue, 29 Jun 2021 14:26:52 +0300 Subject: [PATCH 26/28] Improve logging --- .../src/main/java/org/apache/ignite/internal/cdc/CdcMain.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java index fdb21952faee4..8596aa5524e33 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java @@ -191,7 +191,7 @@ public CdcMain( runX(); } catch (Throwable e) { - e.printStackTrace(); + log.error("Cdc error", e); throw new IgniteException(e); } From 653624820ca06929a16a0a08548f81f79bd9efa5 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Wed, 30 Jun 2021 10:40:57 +0300 Subject: [PATCH 27/28] fix javadoc --- .../src/main/java/org/apache/ignite/cdc/CdcConsumer.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cdc/CdcConsumer.java b/modules/core/src/main/java/org/apache/ignite/cdc/CdcConsumer.java index 67c7f3c0a1c45..af119e9e21d55 100644 --- a/modules/core/src/main/java/org/apache/ignite/cdc/CdcConsumer.java +++ b/modules/core/src/main/java/org/apache/ignite/cdc/CdcConsumer.java @@ -35,14 +35,14 @@ *

* * In case consumer implementation wants to user {@link IgniteLogger}, please, use, {@link LoggerResource} annotation: - *
 {@code
+ * 
  * public class ChangeDataCaptureConsumer implements ChangeDataCaptureConsumer {
- *     @LoggerReource
+ *     @LoggerResource
  *     private IgniteLogger log;
  *
  *     ...
  * }
- * }
+ *
* * Note, consumption of the {@link CdcEvent} will be started from the last saved offset. * The offset of consumptions is saved on the disk every time {@link #onEvents(Iterator)} returns {@code true}. From cead6e64289379b07c28fa096b5b429171e3071e Mon Sep 17 00:00:00 2001 From: Nikolay Date: Wed, 30 Jun 2021 12:00:27 +0300 Subject: [PATCH 28/28] NPE fix --- .../src/main/java/org/apache/ignite/internal/cdc/CdcMain.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java index 8596aa5524e33..d2b7bd334bae3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java @@ -519,7 +519,7 @@ private void ackAsciiLogo() { ">>> " + COPYRIGHT + NL + ">>> " + NL + ">>> Ignite documentation: " + "http://" + SITE + NL + - ">>> Consumer: " + consumer.consumer().toString() + NL + + ">>> Consumer: " + U.toStringSafe(consumer.consumer()) + NL + ">>> ConsistentId: " + igniteCfg.getConsistentId() + NL ); } @@ -535,7 +535,7 @@ private void ackAsciiLogo() { COPYRIGHT, "", "Ignite documentation: " + "http://" + SITE, - "Consumer: " + consumer.consumer().toString(), + "Consumer: " + U.toStringSafe(consumer.consumer()), "ConsistentId: " + igniteCfg.getConsistentId(), "", "Quiet mode.");