From 9efa6cc9c85aab66b325e9f37f7bad13826aea39 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Wed, 8 Sep 2021 13:31:52 -0700 Subject: [PATCH] Make persists concurrent with adding rows in batch ingestion (#11536) * Make persists concurrent with ingestion * Remove semaphore but keep concurrent persists (with add) and add push in the backround as well * Go back to documented default persists (zero) * Move to debug * Remove unnecessary Atomics * Comments on synchronization (or not) for sinks & sinkMetadata * Some cleanup for unit tests but they still need further work * Shutdown & wait for persists and push on close * Provide support for three existing batch appenderators using batchProcessingMode flag * Fix reference to wrong appenderator * Fix doc typos * Add BatchAppenderators class test coverage * Add log message to batchProcessingMode final value, fix typo in enum name * Another typo and minor fix to log message * LEGACY->OPEN_SEGMENTS, Edit docs * Minor update legacy->open segments log message * More code comments, mostly small adjustments to naming etc * fix spelling * Exclude BtachAppenderators from Jacoco since it is fully tested but Jacoco still refuses to ack coverage * Coverage for Appenderators & BatchAppenderators, name change of a method that was still using "legacy" rather than "openSegments" Co-authored-by: Clint Wylie --- docs/configuration/index.md | 2 +- .../indexing/kafka/KafkaIndexTaskTest.java | 3 +- .../kinesis/KinesisIndexTaskTest.java | 3 +- .../indexing/common/config/TaskConfig.java | 54 +- .../common/task/BatchAppenderators.java | 58 +- .../indexing/common/TaskToolboxTest.java | 3 +- ...penderatorDriverRealtimeIndexTaskTest.java | 3 +- .../common/task/AppenderatorsTest.java | 534 ++++++ .../common/task/BatchAppenderatorsTest.java | 627 +++++++ .../common/task/CompactionTaskRunTest.java | 15 +- .../common/task/CompactionTaskTest.java | 15 +- .../indexing/common/task/HadoopTaskTest.java | 3 +- .../common/task/IngestionTestBase.java | 3 +- .../common/task/RealtimeIndexTaskTest.java | 3 +- .../common/task/TestAppenderatorsManager.java | 62 +- ...stractParallelIndexSupervisorTaskTest.java | 33 +- .../overlord/ForkingTaskRunnerTest.java | 9 +- .../SingleTaskBackgroundRunnerTest.java | 3 +- .../indexing/overlord/TaskLifecycleTest.java | 3 +- .../overlord/ThreadingTaskRunnerTest.java | 3 +- .../worker/WorkerTaskManagerTest.java | 3 +- .../worker/WorkerTaskMonitorTest.java | 3 +- ...ntermediaryDataManagerAutoCleanupTest.java | 3 +- ...iaryDataManagerManualAddAndDeleteTest.java | 3 +- .../shuffle/ShuffleDataSegmentPusherTest.java | 3 +- .../worker/shuffle/ShuffleResourceTest.java | 3 +- .../realtime/appenderator/Appenderator.java | 2 +- .../appenderator/AppenderatorImpl.java | 1596 +++++++++++++++++ .../realtime/appenderator/Appenderators.java | 91 +- .../appenderator/AppenderatorsManager.java | 29 +- .../appenderator/BatchAppenderator.java | 450 +++-- .../DefaultOfflineAppenderatorFactory.java | 5 +- ...DummyForInjectionAppenderatorsManager.java | 37 +- .../PeonAppenderatorsManager.java | 73 +- .../UnifiedIndexerAppenderatorsManager.java | 80 +- ...dSegmensSinksBatchAppenderatorTester.java} | 25 +- ...entsSinksBatchAppenderatorDriverTest.java} | 8 +- ...edSegmentsSinksBatchAppenderatorTest.java} | 250 ++- ...DefaultOfflineAppenderatorFactoryTest.java | 8 +- ...enAndClosedSegmentsAppenderatorTester.java | 293 +++ ...edSegmentsBatchAppenderatorDriverTest.java | 205 +++ ...ndClosedSegmentsBatchAppenderatorTest.java | 229 +++ ...nifiedIndexerAppenderatorsManagerTest.java | 53 +- 43 files changed, 4469 insertions(+), 422 deletions(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java create mode 100644 server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java rename server/src/test/java/org/apache/druid/segment/realtime/appenderator/{BatchAppenderatorTester.java => ClosedSegmensSinksBatchAppenderatorTester.java} (96%) rename server/src/test/java/org/apache/druid/segment/realtime/appenderator/{BatchAppenderatorDriverTest.java => ClosedSegmentsSinksBatchAppenderatorDriverTest.java} (96%) rename server/src/test/java/org/apache/druid/segment/realtime/appenderator/{BatchAppenderatorTest.java => ClosedSegmentsSinksBatchAppenderatorTest.java} (86%) create mode 100644 server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java create mode 100644 server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java create mode 100644 server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorTest.java diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 1f1bbd1f4aa..53fd2bec216 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1343,7 +1343,7 @@ Additional peon configs include: |`druid.peon.mode`|Choices are "local" and "remote". Setting this to local means you intend to run the peon as a standalone process (Not recommended).|remote| |`druid.indexer.task.baseDir`|Base temporary working directory.|`System.getProperty("java.io.tmpdir")`| |`druid.indexer.task.baseTaskDir`|Base temporary working directory for tasks.|`${druid.indexer.task.baseDir}/persistent/task`| -|`druid.indexer.task.useLegacyBatchProcessing`|If false, native batch ingestion will use a new, recommended, code path with memory optimized code for the segment creation phase. If true it will use the previous code path for the create segments phase of batch ingestion. This does not apply to streaming ingestion, just to batch. This setting should only be used when a bug is suspected or found in the new optimized batch ingestion code. If a bug is suspected or found, you can set this flag to `true` to fall back to previous, working but more memory intensive, code path.|`false`| +|`druid.indexer.task.batchProcessingMode`| Batch ingestion tasks have three operating modes that control how intermediary segments are constructed and tracked: `OPEN_SEGMENTS`, `CLOSED_SEGMENTS`, and `CLOSED_SEGMENT_SINKS`. `OPEN_SEGMENTS` will use code based on the original batch ingestion path and performs a `mmap` on intermediary segments to build a timeline so that these segments can be queryable by realtime queries. This is not needed at all for batch, so the default mode, `CLOSED_SEGMENTS`, eliminates `mmap` of intermediary segments, but still tracks the entire set of segments in heap. The `CLOSED_SEGMENTS_SINKS` mode is the most aggressive and should have the smallest memory footprint, and works by eliminating in memory tracking and `mmap` of intermediary segments produced during segment creation. This mode isn't as well tested as other modes so is currently considered experimental. `OPEN_SEGMENTS` mode can be selected if any problems occur with the 2 newer modes. |`CLOSED_SEGMENTS`| |`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.8.5| |`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|75000| |`druid.indexer.task.directoryLockTimeout`|Wait this long for zombie peons to exit before giving up on their replacements.|PT10M| diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 54242883c01..8732ed7f1bf 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -2783,7 +2783,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase null, null, false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ); final TestDerbyConnector derbyConnector = derby.getConnector(); derbyConnector.createDataSourceTable(); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 3d0f86d67f5..5a781a86f43 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -2869,7 +2869,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase null, null, false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ); final TestDerbyConnector derbyConnector = derby.getConnector(); derbyConnector.createDataSourceTable(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java index 27e2c552a3b..a41488642c4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java @@ -22,6 +22,8 @@ package org.apache.druid.indexing.common.config; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; +import org.apache.commons.lang3.EnumUtils; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.loading.StorageLocationConfig; import org.joda.time.Period; @@ -39,10 +41,22 @@ import java.util.List; */ public class TaskConfig { + private static final Logger log = new Logger(TaskConfig.class); + public static final List DEFAULT_DEFAULT_HADOOP_COORDINATES = ImmutableList.of( "org.apache.hadoop:hadoop-client:2.8.5" ); + // This enum controls processing mode of batch ingestion "segment creation" phase (i.e. appenderator logic) + public enum BatchProcessingMode + { + OPEN_SEGMENTS, /* mmap segments, legacy code */ + CLOSED_SEGMENTS, /* Do not mmap segments but keep most other legacy code */ + CLOSED_SEGMENTS_SINKS /* Most aggressive memory optimization, do not mmap segments and eliminate sinks, etc. */ + } + + public static final BatchProcessingMode BATCH_PROCESSING_MODE_DEFAULT = BatchProcessingMode.CLOSED_SEGMENTS; + private static final Period DEFAULT_DIRECTORY_LOCK_TIMEOUT = new Period("PT10M"); private static final Period DEFAULT_GRACEFUL_SHUTDOWN_TIMEOUT = new Period("PT5M"); @@ -77,7 +91,10 @@ public class TaskConfig private final boolean ignoreTimestampSpecForDruidInputSource; @JsonProperty - private final boolean useLegacyBatchProcessing; + private final boolean batchMemoryMappedIndex; + + @JsonProperty + private final BatchProcessingMode batchProcessingMode; @JsonCreator public TaskConfig( @@ -91,7 +108,8 @@ public class TaskConfig @JsonProperty("directoryLockTimeout") Period directoryLockTimeout, @JsonProperty("shuffleDataLocations") List shuffleDataLocations, @JsonProperty("ignoreTimestampSpecForDruidInputSource") boolean ignoreTimestampSpecForDruidInputSource, - @JsonProperty("useLegacyBatchProcessing") boolean useLegacyBatchProcessing // only set to true to fall back to older behavior + @JsonProperty("batchMemoryMappedIndex") boolean batchMemoryMappedIndex, // deprecated, only set to true to fall back to older behavior + @JsonProperty("batchProcessingMode") String batchProcessingMode ) { this.baseDir = baseDir == null ? System.getProperty("java.io.tmpdir") : baseDir; @@ -117,7 +135,23 @@ public class TaskConfig this.shuffleDataLocations = shuffleDataLocations; } this.ignoreTimestampSpecForDruidInputSource = ignoreTimestampSpecForDruidInputSource; - this.useLegacyBatchProcessing = useLegacyBatchProcessing; + + this.batchMemoryMappedIndex = batchMemoryMappedIndex; + // Conflict resolution. Assume that if batchMemoryMappedIndex is set (since false is the default) that + // the user changed it intentionally to use legacy, in this case oveeride batchProcessingMode and also + // set it to legacy else just use batchProcessingMode and don't pay attention to batchMemoryMappedIndexMode: + if (batchMemoryMappedIndex) { + this.batchProcessingMode = BatchProcessingMode.OPEN_SEGMENTS; + } else if (EnumUtils.isValidEnum(BatchProcessingMode.class, batchProcessingMode)) { + this.batchProcessingMode = BatchProcessingMode.valueOf(batchProcessingMode); + } else { + // batchProcessingMode input string is invalid, log & use the default. + this.batchProcessingMode = BatchProcessingMode.CLOSED_SEGMENTS; // Default + log.warn("Batch processing mode argument value is null or not valid:[%s], defaulting to[%s] ", + batchProcessingMode, this.batchProcessingMode + ); + } + log.info("Batch processing mode:[%s]", this.batchProcessingMode); } @JsonProperty @@ -201,9 +235,19 @@ public class TaskConfig } @JsonProperty - public boolean getuseLegacyBatchProcessing() + public BatchProcessingMode getBatchProcessingMode() { - return useLegacyBatchProcessing; + return batchProcessingMode; + } + + /** + * Do not use in code! use {@link TaskConfig#getBatchProcessingMode() instead} + */ + @Deprecated + @JsonProperty + public boolean getbatchMemoryMappedIndex() + { + return batchMemoryMappedIndex; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java index 1a437063060..8d15dc815e1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java @@ -21,6 +21,8 @@ package org.apache.druid.indexing.common.task; import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; @@ -70,21 +72,51 @@ public final class BatchAppenderators ParseExceptionHandler parseExceptionHandler ) { - return appenderatorsManager.createOfflineAppenderatorForTask( - taskId, - dataSchema, - appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()), - metrics, - segmentPusher, - toolbox.getJsonMapper(), - toolbox.getIndexIO(), - toolbox.getIndexMergerV9(), - rowIngestionMeters, - parseExceptionHandler, - toolbox.getConfig().getuseLegacyBatchProcessing() - ); + if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.OPEN_SEGMENTS) { + return appenderatorsManager.createOpenSegmentsOfflineAppenderatorForTask( + taskId, + dataSchema, + appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()), + metrics, + segmentPusher, + toolbox.getJsonMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9(), + rowIngestionMeters, + parseExceptionHandler + ); + } else if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS) { + return appenderatorsManager.createClosedSegmentsOfflineAppenderatorForTask( + taskId, + dataSchema, + appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()), + metrics, + segmentPusher, + toolbox.getJsonMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9(), + rowIngestionMeters, + parseExceptionHandler + ); + } else if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS_SINKS) { + return appenderatorsManager.createOfflineAppenderatorForTask( + taskId, + dataSchema, + appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()), + metrics, + segmentPusher, + toolbox.getJsonMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9(), + rowIngestionMeters, + parseExceptionHandler + ); + } else { + throw new IAE("Invalid batchProcesingMode[%s]", toolbox.getConfig().getBatchProcessingMode()); + } } + public static BatchAppenderatorDriver newDriver( final Appenderator appenderator, final TaskToolbox toolbox, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index 3f75ce8dd97..fabb1271933 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -113,7 +113,8 @@ public class TaskToolboxTest null, null, false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ), new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false), mockTaskActionClientFactory, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 0c6b1c482a2..48bfab5df75 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1518,7 +1518,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand null, null, false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ); final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java new file mode 100644 index 00000000000..b90b95df0d0 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java @@ -0,0 +1,534 @@ +/* + * 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.druid.indexing.common.task; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.core.NoopEmitter; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; +import org.apache.druid.segment.incremental.ParseExceptionHandler; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; +import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; +import org.apache.druid.segment.realtime.appenderator.Appenderators; +import org.apache.druid.segment.realtime.appenderator.BatchAppenderator; +import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CopyOnWriteArrayList; + + +public class AppenderatorsTest +{ + @Test + public void testOpenSegmentsOfflineAppenderator() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester("OPEN_SEGMENTS")) { + Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl); + AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator; + Assert.assertTrue(appenderator.isOpenSegments()); + } + } + + @Test + public void testClosedSegmentsOfflineAppenderator() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS")) { + Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl); + AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator; + Assert.assertFalse(appenderator.isOpenSegments()); + } + } + + @Test + public void testClosedSegmentsSinksOfflineAppenderator() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS_SINKS")) { + Assert.assertTrue(tester.appenderator instanceof BatchAppenderator); + } + } + + + private static class AppenderatorTester implements AutoCloseable + { + public static final String DATASOURCE = "foo"; + + private final DataSchema schema; + private final AppenderatorConfig tuningConfig; + private final FireDepartmentMetrics metrics; + private final ObjectMapper objectMapper; + private final Appenderator appenderator; + private final ServiceEmitter emitter; + + private final List pushedSegments = new CopyOnWriteArrayList<>(); + + + public AppenderatorTester( + final String batchMode + ) + { + this(100, 100, null, false, new SimpleRowIngestionMeters(), + false, batchMode + ); + } + + public AppenderatorTester( + final int maxRowsInMemory, + final long maxSizeInBytes, + @Nullable final File basePersistDirectory, + final boolean enablePushFailure, + final RowIngestionMeters rowIngestionMeters, + final boolean skipBytesInMemoryOverheadCheck, + String batchMode + ) + { + objectMapper = new DefaultObjectMapper(); + objectMapper.registerSubtypes(LinearShardSpec.class); + + final Map parserMap = objectMapper.convertValue( + new MapInputRowParser( + new JSONParseSpec( + new TimestampSpec("ts", "auto", null), + new DimensionsSpec(null, null, null), + null, + null, + null + ) + ), + Map.class + ); + + schema = new DataSchema( + DATASOURCE, + null, + null, + new AggregatorFactory[]{ + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("met", "met") + }, + new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), + null, + parserMap, + objectMapper + ); + + tuningConfig = new TestIndexTuningConfig( + TuningConfig.DEFAULT_APPENDABLE_INDEX, + maxRowsInMemory, + maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, + skipBytesInMemoryOverheadCheck, + new IndexSpec(), + 0, + false, + 0L, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, + basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory + ); + metrics = new FireDepartmentMetrics(); + + IndexIO indexIO = new IndexIO( + objectMapper, + () -> 0 + ); + IndexMerger indexMerger = new IndexMergerV9( + objectMapper, + indexIO, + OffHeapMemorySegmentWriteOutMediumFactory.instance() + ); + + emitter = new ServiceEmitter( + "test", + "test", + new NoopEmitter() + ); + emitter.start(); + EmittingLogger.registerEmitter(emitter); + DataSegmentPusher dataSegmentPusher = new DataSegmentPusher() + { + private boolean mustFail = true; + + @Deprecated + @Override + public String getPathForHadoop(String dataSource) + { + return getPathForHadoop(); + } + + @Override + public String getPathForHadoop() + { + throw new UnsupportedOperationException(); + } + + @Override + public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException + { + if (enablePushFailure && mustFail) { + mustFail = false; + throw new IOException("Push failure test"); + } else if (enablePushFailure) { + mustFail = true; + } + pushedSegments.add(segment); + return segment; + } + + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException(); + } + }; + + switch (batchMode) { + case "OPEN_SEGMENTS": + appenderator = Appenderators.createOpenSegmentsOffline( + schema.getDataSource(), + schema, + tuningConfig, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0) + ); + break; + case "CLOSED_SEGMENTS": + appenderator = Appenderators.createClosedSegmentsOffline( + schema.getDataSource(), + schema, + tuningConfig, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0) + ); + + break; + case "CLOSED_SEGMENTS_SINKS": + appenderator = Appenderators.createOffline( + schema.getDataSource(), + schema, + tuningConfig, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0) + ); + break; + default: + throw new IllegalArgumentException("Unrecognized batchMode: " + batchMode); + } + } + + private long getDefaultMaxBytesInMemory() + { + return (Runtime.getRuntime().totalMemory()) / 3; + } + + public DataSchema getSchema() + { + return schema; + } + + public AppenderatorConfig getTuningConfig() + { + return tuningConfig; + } + + public FireDepartmentMetrics getMetrics() + { + return metrics; + } + + public ObjectMapper getObjectMapper() + { + return objectMapper; + } + + public Appenderator getAppenderator() + { + return appenderator; + } + + public List getPushedSegments() + { + return pushedSegments; + } + + @Override + public void close() throws Exception + { + appenderator.close(); + emitter.close(); + FileUtils.deleteDirectory(tuningConfig.getBasePersistDirectory()); + } + + private static File createNewBasePersistDirectory() + { + return FileUtils.createTempDir("druid-batch-persist"); + } + + + static class TestIndexTuningConfig implements AppenderatorConfig + { + private final AppendableIndexSpec appendableIndexSpec; + private final int maxRowsInMemory; + private final long maxBytesInMemory; + private final boolean skipBytesInMemoryOverheadCheck; + private final int maxColumnsToMerge; + private final PartitionsSpec partitionsSpec; + private final IndexSpec indexSpec; + private final File basePersistDirectory; + private final int maxPendingPersists; + private final boolean reportParseExceptions; + private final long pushTimeout; + private final IndexSpec indexSpecForIntermediatePersists; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + + public TestIndexTuningConfig( + AppendableIndexSpec appendableIndexSpec, + Integer maxRowsInMemory, + Long maxBytesInMemory, + Boolean skipBytesInMemoryOverheadCheck, + IndexSpec indexSpec, + Integer maxPendingPersists, + Boolean reportParseExceptions, + Long pushTimeout, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + Integer maxColumnsToMerge, + File basePersistDirectory + ) + { + this.appendableIndexSpec = appendableIndexSpec; + this.maxRowsInMemory = maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory; + this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck; + this.indexSpec = indexSpec; + this.maxPendingPersists = maxPendingPersists; + this.reportParseExceptions = reportParseExceptions; + this.pushTimeout = pushTimeout; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + this.maxColumnsToMerge = maxColumnsToMerge; + this.basePersistDirectory = basePersistDirectory; + + this.partitionsSpec = null; + this.indexSpecForIntermediatePersists = this.indexSpec; + } + + @Override + public TestIndexTuningConfig withBasePersistDirectory(File dir) + { + throw new UnsupportedOperationException(); + } + + @Override + public AppendableIndexSpec getAppendableIndexSpec() + { + return appendableIndexSpec; + } + + @Override + public int getMaxRowsInMemory() + { + return maxRowsInMemory; + } + + @Override + public long getMaxBytesInMemory() + { + return maxBytesInMemory; + } + + @Override + public boolean isSkipBytesInMemoryOverheadCheck() + { + return skipBytesInMemoryOverheadCheck; + } + + @Nullable + @Override + public PartitionsSpec getPartitionsSpec() + { + return partitionsSpec; + } + + @Override + public IndexSpec getIndexSpec() + { + return indexSpec; + } + + @Override + public IndexSpec getIndexSpecForIntermediatePersists() + { + return indexSpecForIntermediatePersists; + } + + @Override + public int getMaxPendingPersists() + { + return maxPendingPersists; + } + + @Override + public boolean isReportParseExceptions() + { + return reportParseExceptions; + } + + @Nullable + @Override + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + + @Override + public int getMaxColumnsToMerge() + { + return maxColumnsToMerge; + } + + @Override + public File getBasePersistDirectory() + { + return basePersistDirectory; + } + + @Override + public Period getIntermediatePersistPeriod() + { + return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestIndexTuningConfig that = (TestIndexTuningConfig) o; + return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) && + maxRowsInMemory == that.maxRowsInMemory && + maxBytesInMemory == that.maxBytesInMemory && + skipBytesInMemoryOverheadCheck == that.skipBytesInMemoryOverheadCheck && + maxColumnsToMerge == that.maxColumnsToMerge && + maxPendingPersists == that.maxPendingPersists && + reportParseExceptions == that.reportParseExceptions && + pushTimeout == that.pushTimeout && + Objects.equals(partitionsSpec, that.partitionsSpec) && + Objects.equals(indexSpec, that.indexSpec) && + Objects.equals(indexSpecForIntermediatePersists, that.indexSpecForIntermediatePersists) && + Objects.equals(basePersistDirectory, that.basePersistDirectory) && + Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory); + } + + @Override + public int hashCode() + { + return Objects.hash( + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + maxColumnsToMerge, + partitionsSpec, + indexSpec, + indexSpecForIntermediatePersists, + basePersistDirectory, + maxPendingPersists, + reportParseExceptions, + pushTimeout, + segmentWriteOutMediumFactory + ); + } + + @Override + public String toString() + { + return "IndexTuningConfig{" + + "maxRowsInMemory=" + maxRowsInMemory + + ", maxBytesInMemory=" + maxBytesInMemory + + ", skipBytesInMemoryOverheadCheck=" + skipBytesInMemoryOverheadCheck + + ", maxColumnsToMerge=" + maxColumnsToMerge + + ", partitionsSpec=" + partitionsSpec + + ", indexSpec=" + indexSpec + + ", indexSpecForIntermediatePersists=" + indexSpecForIntermediatePersists + + ", basePersistDirectory=" + basePersistDirectory + + ", maxPendingPersists=" + maxPendingPersists + + ", reportParseExceptions=" + reportParseExceptions + + ", pushTimeout=" + pushTimeout + + ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + + '}'; + } + } + + } + +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java new file mode 100644 index 00000000000..6813dcc97c4 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java @@ -0,0 +1,627 @@ +/* + * 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.druid.indexing.common.task; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.core.NoopEmitter; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; +import org.apache.druid.segment.incremental.ParseExceptionHandler; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.join.NoopJoinableFactory; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; +import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; +import org.apache.druid.segment.realtime.appenderator.BatchAppenderator; +import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.server.security.AuthTestUtils; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CopyOnWriteArrayList; + + +public class BatchAppenderatorsTest +{ + @Test + public void testLegacyOfflineAppenderator() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester("OPEN_SEGMENTS")) { + Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl); + AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator; + Assert.assertTrue(appenderator.isOpenSegments()); + } + } + + @Test + public void testClosedSegmentsOfflineAppenderator() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS")) { + Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl); + AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator; + Assert.assertFalse(appenderator.isOpenSegments()); + } + } + + @Test + public void testClosedSegmentsSinksOfflineAppenderator() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS_SINKS")) { + Assert.assertTrue(tester.appenderator instanceof BatchAppenderator); + } + } + + + private static class AppenderatorTester implements AutoCloseable + { + public static final String DATASOURCE = "foo"; + + private final DataSchema schema; + private final AppenderatorConfig tuningConfig; + private final FireDepartmentMetrics metrics; + private final ObjectMapper objectMapper; + private final Appenderator appenderator; + private final ServiceEmitter emitter; + + private final List pushedSegments = new CopyOnWriteArrayList<>(); + + + public AppenderatorTester( + final String batchMode + ) + { + this(100, 100, null, false, new SimpleRowIngestionMeters(), + false, batchMode + ); + } + + public AppenderatorTester( + final int maxRowsInMemory, + final long maxSizeInBytes, + @Nullable final File basePersistDirectory, + final boolean enablePushFailure, + final RowIngestionMeters rowIngestionMeters, + final boolean skipBytesInMemoryOverheadCheck, + String batchMode + ) + { + objectMapper = new DefaultObjectMapper(); + objectMapper.registerSubtypes(LinearShardSpec.class); + + final Map parserMap = objectMapper.convertValue( + new MapInputRowParser( + new JSONParseSpec( + new TimestampSpec("ts", "auto", null), + new DimensionsSpec(null, null, null), + null, + null, + null + ) + ), + Map.class + ); + + schema = new DataSchema( + DATASOURCE, + null, + null, + new AggregatorFactory[]{ + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("met", "met") + }, + new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), + null, + parserMap, + objectMapper + ); + + tuningConfig = new TestIndexTuningConfig( + TuningConfig.DEFAULT_APPENDABLE_INDEX, + maxRowsInMemory, + maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, + skipBytesInMemoryOverheadCheck, + new IndexSpec(), + 0, + false, + 0L, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, + basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory + ); + metrics = new FireDepartmentMetrics(); + + IndexIO indexIO = new IndexIO( + objectMapper, + () -> 0 + ); + IndexMergerV9 indexMerger = new IndexMergerV9( + objectMapper, + indexIO, + OffHeapMemorySegmentWriteOutMediumFactory.instance() + ); + + emitter = new ServiceEmitter( + "test", + "test", + new NoopEmitter() + ); + emitter.start(); + EmittingLogger.registerEmitter(emitter); + DataSegmentPusher dataSegmentPusher = new DataSegmentPusher() + { + private boolean mustFail = true; + + @Deprecated + @Override + public String getPathForHadoop(String dataSource) + { + return getPathForHadoop(); + } + + @Override + public String getPathForHadoop() + { + throw new UnsupportedOperationException(); + } + + @Override + public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException + { + if (enablePushFailure && mustFail) { + mustFail = false; + throw new IOException("Push failure test"); + } else if (enablePushFailure) { + mustFail = true; + } + pushedSegments.add(segment); + return segment; + } + + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException(); + } + }; + + switch (batchMode) { + case "OPEN_SEGMENTS": + appenderator = BatchAppenderators.newAppenderator( + "foo", + new TestAppenderatorsManager(), + metrics, + new TestTaskToolbox( + objectMapper, + indexMerger, + TaskConfig.BatchProcessingMode.OPEN_SEGMENTS + ), + schema, + tuningConfig, + dataSegmentPusher, + rowIngestionMeters, + new ParseExceptionHandler( + rowIngestionMeters, + false, + Integer.MAX_VALUE, + 0 + ) + ); + break; + case "CLOSED_SEGMENTS": + appenderator = BatchAppenderators.newAppenderator( + "foo", + new TestAppenderatorsManager(), + metrics, + new TestTaskToolbox( + objectMapper, + indexMerger, + TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS + ), + schema, + tuningConfig, + dataSegmentPusher, + rowIngestionMeters, + new ParseExceptionHandler( + rowIngestionMeters, + false, + Integer.MAX_VALUE, + 0 + ) + ); + + break; + case "CLOSED_SEGMENTS_SINKS": + appenderator = BatchAppenderators.newAppenderator( + "foo", + new TestAppenderatorsManager(), + metrics, + new TestTaskToolbox( + objectMapper, + indexMerger, + TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS_SINKS + ), + schema, + tuningConfig, + dataSegmentPusher, + rowIngestionMeters, + new ParseExceptionHandler( + rowIngestionMeters, + false, + Integer.MAX_VALUE, + 0 + ) + ); + break; + default: + throw new IllegalArgumentException("Unrecognized batchMode: " + batchMode); + } + } + + private long getDefaultMaxBytesInMemory() + { + return (Runtime.getRuntime().totalMemory()) / 3; + } + + public DataSchema getSchema() + { + return schema; + } + + public AppenderatorConfig getTuningConfig() + { + return tuningConfig; + } + + public FireDepartmentMetrics getMetrics() + { + return metrics; + } + + public ObjectMapper getObjectMapper() + { + return objectMapper; + } + + public Appenderator getAppenderator() + { + return appenderator; + } + + public List getPushedSegments() + { + return pushedSegments; + } + + @Override + public void close() throws Exception + { + appenderator.close(); + emitter.close(); + FileUtils.deleteDirectory(tuningConfig.getBasePersistDirectory()); + } + + private static File createNewBasePersistDirectory() + { + return FileUtils.createTempDir("druid-batch-persist"); + } + + + static class TestIndexTuningConfig implements AppenderatorConfig + { + private final AppendableIndexSpec appendableIndexSpec; + private final int maxRowsInMemory; + private final long maxBytesInMemory; + private final boolean skipBytesInMemoryOverheadCheck; + private final int maxColumnsToMerge; + private final PartitionsSpec partitionsSpec; + private final IndexSpec indexSpec; + private final File basePersistDirectory; + private final int maxPendingPersists; + private final boolean reportParseExceptions; + private final long pushTimeout; + private final IndexSpec indexSpecForIntermediatePersists; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + + public TestIndexTuningConfig( + AppendableIndexSpec appendableIndexSpec, + Integer maxRowsInMemory, + Long maxBytesInMemory, + Boolean skipBytesInMemoryOverheadCheck, + IndexSpec indexSpec, + Integer maxPendingPersists, + Boolean reportParseExceptions, + Long pushTimeout, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + Integer maxColumnsToMerge, + File basePersistDirectory + ) + { + this.appendableIndexSpec = appendableIndexSpec; + this.maxRowsInMemory = maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory; + this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck; + this.indexSpec = indexSpec; + this.maxPendingPersists = maxPendingPersists; + this.reportParseExceptions = reportParseExceptions; + this.pushTimeout = pushTimeout; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + this.maxColumnsToMerge = maxColumnsToMerge; + this.basePersistDirectory = basePersistDirectory; + + this.partitionsSpec = null; + this.indexSpecForIntermediatePersists = this.indexSpec; + } + + @Override + public TestIndexTuningConfig withBasePersistDirectory(File dir) + { + return this; + } + + @Override + public AppendableIndexSpec getAppendableIndexSpec() + { + return appendableIndexSpec; + } + + @Override + public int getMaxRowsInMemory() + { + return maxRowsInMemory; + } + + @Override + public long getMaxBytesInMemory() + { + return maxBytesInMemory; + } + + @Override + public boolean isSkipBytesInMemoryOverheadCheck() + { + return skipBytesInMemoryOverheadCheck; + } + + @Nullable + @Override + public PartitionsSpec getPartitionsSpec() + { + return partitionsSpec; + } + + @Override + public IndexSpec getIndexSpec() + { + return indexSpec; + } + + @Override + public IndexSpec getIndexSpecForIntermediatePersists() + { + return indexSpecForIntermediatePersists; + } + + @Override + public int getMaxPendingPersists() + { + return maxPendingPersists; + } + + @Override + public boolean isReportParseExceptions() + { + return reportParseExceptions; + } + + @Nullable + @Override + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + + @Override + public int getMaxColumnsToMerge() + { + return maxColumnsToMerge; + } + + @Override + public File getBasePersistDirectory() + { + return basePersistDirectory; + } + + @Override + public Period getIntermediatePersistPeriod() + { + return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestIndexTuningConfig that = (TestIndexTuningConfig) o; + return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) && + maxRowsInMemory == that.maxRowsInMemory && + maxBytesInMemory == that.maxBytesInMemory && + skipBytesInMemoryOverheadCheck == that.skipBytesInMemoryOverheadCheck && + maxColumnsToMerge == that.maxColumnsToMerge && + maxPendingPersists == that.maxPendingPersists && + reportParseExceptions == that.reportParseExceptions && + pushTimeout == that.pushTimeout && + Objects.equals(partitionsSpec, that.partitionsSpec) && + Objects.equals(indexSpec, that.indexSpec) && + Objects.equals(indexSpecForIntermediatePersists, that.indexSpecForIntermediatePersists) && + Objects.equals(basePersistDirectory, that.basePersistDirectory) && + Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory); + } + + @Override + public int hashCode() + { + return Objects.hash( + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + maxColumnsToMerge, + partitionsSpec, + indexSpec, + indexSpecForIntermediatePersists, + basePersistDirectory, + maxPendingPersists, + reportParseExceptions, + pushTimeout, + segmentWriteOutMediumFactory + ); + } + + @Override + public String toString() + { + return "IndexTuningConfig{" + + "maxRowsInMemory=" + maxRowsInMemory + + ", maxBytesInMemory=" + maxBytesInMemory + + ", skipBytesInMemoryOverheadCheck=" + skipBytesInMemoryOverheadCheck + + ", maxColumnsToMerge=" + maxColumnsToMerge + + ", partitionsSpec=" + partitionsSpec + + ", indexSpec=" + indexSpec + + ", indexSpecForIntermediatePersists=" + indexSpecForIntermediatePersists + + ", basePersistDirectory=" + basePersistDirectory + + ", maxPendingPersists=" + maxPendingPersists + + ", reportParseExceptions=" + reportParseExceptions + + ", pushTimeout=" + pushTimeout + + ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + + '}'; + } + } + + + private static class TestTaskToolbox extends TaskToolbox + { + private final Map segmentFileMap; + + TestTaskToolbox(ObjectMapper mapper, IndexMergerV9 indexMergerV9, TaskConfig.BatchProcessingMode mode) + { + super( + new TaskConfig( + null, + null, + null, + null, + null, + false, + null, + null, + null, + false, + false, + mode.name() + ), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + NoopJoinableFactory.INSTANCE, + null, + null, + mapper, + null, + new IndexIO( + new ObjectMapper(), + () -> 0 + ), + null, + null, + null, + indexMergerV9, + null, + null, + null, + null, + new NoopTestTaskReportFileWriter(), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + new NoopChatHandlerProvider(), + null, + new TestAppenderatorsManager(), + null, + null, + null, + null + ); + this.segmentFileMap = null; + } + } + } +} + diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index b6098ea9854..19c6575600a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -1297,7 +1297,20 @@ public class CompactionTaskRunTest extends IngestionTestBase ); return new TaskToolbox( - new TaskConfig(null, null, null, null, null, false, null, null, null, false, false), + new TaskConfig( + null, + null, + null, + null, + null, + false, + null, + null, + null, + false, + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() + ), null, createActionClient(task), null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index ac9e3d011ae..9adb4c231f0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1747,7 +1747,20 @@ public class CompactionTaskTest ) { super( - new TaskConfig(null, null, null, null, null, false, null, null, null, false, false), + new TaskConfig( + null, + null, + null, + null, + null, + false, + null, + null, + null, + false, + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() + ), null, taskActionClient, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java index f08e3f73aa2..47091696aa1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java @@ -118,7 +118,8 @@ public class HadoopTaskTest null, null, false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() )).once(); EasyMock.replay(toolbox); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index ab9b0d72b69..81a6bcc6e9a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -314,7 +314,8 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest ); final TaskToolbox box = new TaskToolbox( - new TaskConfig(null, null, null, null, null, false, null, null, null, false, false), + new TaskConfig(null, null, null, null, null, false, null, null, null, false, false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()), new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false), taskActionClient, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 94bf84d0e32..bfd96787811 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -899,7 +899,8 @@ public class RealtimeIndexTaskTest extends InitializedNullHandlingTest null, null, false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ); final TaskLockbox taskLockbox = new TaskLockbox(taskStorage, mdc); try { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java index 225769e23be..bcdbb66226c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java @@ -93,6 +93,62 @@ public class TestAppenderatorsManager implements AppenderatorsManager return realtimeAppenderator; } + @Override + public Appenderator createOpenSegmentsOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler + ) + { + return Appenderators.createOpenSegmentsOffline( + taskId, + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + parseExceptionHandler + ); + } + + @Override + public Appenderator createClosedSegmentsOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler + ) + { + return Appenderators.createClosedSegmentsOffline( + taskId, + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + parseExceptionHandler + ); + } + @Override public Appenderator createOfflineAppenderatorForTask( String taskId, @@ -104,8 +160,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useLegacyBatchProcessing + ParseExceptionHandler parseExceptionHandler ) { return Appenderators.createOffline( @@ -118,8 +173,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager indexIO, indexMerger, rowIngestionMeters, - parseExceptionHandler, - useLegacyBatchProcessing + parseExceptionHandler ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 82bc1f38632..fc1328b5523 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -244,7 +244,8 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase null, ImmutableList.of(new StorageLocationConfig(temporaryFolder.newFolder(), null, null)), false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ), null ); @@ -600,7 +601,20 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase public void prepareObjectMapper(ObjectMapper objectMapper, IndexIO indexIO) { - final TaskConfig taskConfig = new TaskConfig(null, null, null, null, null, false, null, null, null, false, false); + final TaskConfig taskConfig = new TaskConfig( + null, + null, + null, + null, + null, + false, + null, + null, + null, + false, + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() + ); objectMapper.setInjectableValues( new InjectableValues.Std() @@ -635,7 +649,20 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase protected TaskToolbox createTaskToolbox(Task task, TaskActionClient actionClient) throws IOException { return new TaskToolbox( - new TaskConfig(null, null, null, null, null, false, null, null, null, false, false), + new TaskConfig( + null, + null, + null, + null, + null, + false, + null, + null, + null, + false, + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() + ), new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false), actionClient, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ForkingTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ForkingTaskRunnerTest.java index 06629b10862..6e453d8a2bc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ForkingTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ForkingTaskRunnerTest.java @@ -202,7 +202,8 @@ public class ForkingTaskRunnerTest new Period("PT10S"), ImmutableList.of(), false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ), new WorkerConfig(), new Properties(), @@ -255,7 +256,8 @@ public class ForkingTaskRunnerTest new Period("PT10S"), ImmutableList.of(), false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ), new WorkerConfig(), new Properties(), @@ -312,7 +314,8 @@ public class ForkingTaskRunnerTest new Period("PT10S"), ImmutableList.of(), false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ), new WorkerConfig(), new Properties(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index 7b12c0dc381..06a4068cdd6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -96,7 +96,8 @@ public class SingleTaskBackgroundRunnerTest null, null, false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ); final ServiceEmitter emitter = new NoopServiceEmitter(); EmittingLogger.registerEmitter(emitter); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 80e31acb0c3..88cebef7f1b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -602,7 +602,8 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest new TaskAuditLogConfig(true) ); File tmpDir = temporaryFolder.newFolder(); - taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, false, null, null, null, false, false); + taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, false, null, null, null, false, false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()); return new TaskToolboxFactory( taskConfig, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ThreadingTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ThreadingTaskRunnerTest.java index d8ddd5145e5..c5c1eb367eb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ThreadingTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/ThreadingTaskRunnerTest.java @@ -61,7 +61,8 @@ public class ThreadingTaskRunnerTest new Period("PT10S"), ImmutableList.of(), false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ), new WorkerConfig(), new NoopTaskLogs(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index 389d3800da8..1fe3acb482e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -93,7 +93,8 @@ public class WorkerTaskManagerTest null, null, false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ); TaskActionClientFactory taskActionClientFactory = EasyMock.createNiceMock(TaskActionClientFactory.class); TaskActionClient taskActionClient = EasyMock.createNiceMock(TaskActionClient.class); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index 5a0270098fa..27ca68cd8cc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -164,7 +164,8 @@ public class WorkerTaskMonitorTest null, null, false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ); TaskActionClientFactory taskActionClientFactory = EasyMock.createNiceMock(TaskActionClientFactory.class); TaskActionClient taskActionClient = EasyMock.createNiceMock(TaskActionClient.class); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java index 2031b190b53..19ecb259d3a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerAutoCleanupTest.java @@ -94,7 +94,8 @@ public class LocalIntermediaryDataManagerAutoCleanupTest null, ImmutableList.of(new StorageLocationConfig(tempDir.newFolder(), null, null)), false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ); final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java index 9a5dc8fc8d3..5155024be8f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/LocalIntermediaryDataManagerManualAddAndDeleteTest.java @@ -79,7 +79,8 @@ public class LocalIntermediaryDataManagerManualAddAndDeleteTest null, ImmutableList.of(new StorageLocationConfig(intermediarySegmentsLocation, 1200L, null)), false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ); final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient(); intermediaryDataManager = new LocalIntermediaryDataManager(workerConfig, taskConfig, indexingServiceClient); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java index f05a1dc57f4..a1b80ca283c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleDataSegmentPusherTest.java @@ -109,7 +109,8 @@ public class ShuffleDataSegmentPusherTest null, ImmutableList.of(new StorageLocationConfig(temporaryFolder.newFolder(), null, null)), false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ); final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient(); if (LOCAL.equals(intermediateDataStore)) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java index 03340309528..db96fe1f2f4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/shuffle/ShuffleResourceTest.java @@ -99,7 +99,8 @@ public class ShuffleResourceTest null, ImmutableList.of(new StorageLocationConfig(tempDir.newFolder(), null, null)), false, - false + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name() ); final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient() { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java index 26d6f5804fa..8f2a4a29654 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java @@ -46,7 +46,7 @@ import java.util.List; *

* Important note: For historical reasons there was a single implementation for this interface ({@code AppenderatorImpl}) * but that since has been split into two classes: {@link StreamAppenderator} and {@link BatchAppenderator}. With this change - * all the query support & concurrency has been removed from the {@code BatchAppenderator} therefore this class no longer + * all the query support & concurrency has been removed/changed in {@code BatchAppenderator} therefore this class no longer * makes sense to have as an {@code Appenderator}. In the future we may want to refactor away the {@code Appenderator} * interface from {@code BatchAppenderator}. */ diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java new file mode 100644 index 00000000000..c79b57ebb3f --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -0,0 +1,1596 @@ +/* + * 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.druid.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Stopwatch; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.commons.lang.mutable.MutableLong; +import org.apache.druid.client.cache.Cache; +import org.apache.druid.data.input.Committer; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.incremental.IncrementalIndexAddResult; +import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.ParseExceptionHandler; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.FireHydrant; +import org.apache.druid.segment.realtime.plumber.Sink; +import org.apache.druid.server.coordination.DataSegmentAnnouncer; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.IdentityHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; + +/** + * This class is to support OPEN_SEGMENTS and CLOSED_SEGMENTS appenderators. It is mostly taken + * from 0.21 and it is meant to keep for backward compatibility. For now though this class + * with isLegacy constructor argument set to false is the default. When {@link BatchAppenderator} + * proves stable then the plan is to remove this class + */ +public class AppenderatorImpl implements Appenderator +{ + // Rough estimate of memory footprint of a ColumnHolder based on actual heap dumps + public static final int ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER = 1000; + public static final int ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER = 700; + public static final int ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER = 600; + // Rough estimate of memory footprint of empty Sink based on actual heap dumps + public static final int ROUGH_OVERHEAD_PER_SINK = 5000; + // Rough estimate of memory footprint of empty FireHydrant based on actual heap dumps + public static final int ROUGH_OVERHEAD_PER_HYDRANT = 1000; + + private static final EmittingLogger log = new EmittingLogger(AppenderatorImpl.class); + private static final int WARN_DELAY = 1000; + private static final String IDENTIFIER_FILE_NAME = "identifier.json"; + + private final String myId; + private final DataSchema schema; + private final AppenderatorConfig tuningConfig; + private final FireDepartmentMetrics metrics; + private final DataSegmentPusher dataSegmentPusher; + private final ObjectMapper objectMapper; + private final DataSegmentAnnouncer segmentAnnouncer; + private final IndexIO indexIO; + private final IndexMerger indexMerger; + private final Cache cache; + /** + * This map needs to be concurrent because it's accessed and mutated from multiple threads: both the thread from where + * this Appenderator is used (and methods like {@link #add(SegmentIdWithShardSpec, InputRow, Supplier, boolean)} are + * called) and from {@link #persistExecutor}. It could also be accessed (but not mutated) potentially in the context + * of any thread from {@link #drop}. + */ + private final ConcurrentMap sinks = new ConcurrentHashMap<>(); + private final Set droppingSinks = Sets.newConcurrentHashSet(); + private final VersionedIntervalTimeline sinkTimeline; + private final long maxBytesTuningConfig; + private final boolean skipBytesInMemoryOverheadCheck; + + private final QuerySegmentWalker texasRanger; + // This variable updated in add(), persist(), and drop() + private final AtomicInteger rowsCurrentlyInMemory = new AtomicInteger(); + private final AtomicInteger totalRows = new AtomicInteger(); + private final AtomicLong bytesCurrentlyInMemory = new AtomicLong(); + private final RowIngestionMeters rowIngestionMeters; + private final ParseExceptionHandler parseExceptionHandler; + // Synchronize persisting commitMetadata so that multiple persist threads (if present) + // and abandon threads do not step over each other + private final Lock commitLock = new ReentrantLock(); + + private final AtomicBoolean closed = new AtomicBoolean(false); + + private volatile ListeningExecutorService persistExecutor = null; + private volatile ListeningExecutorService pushExecutor = null; + // use intermediate executor so that deadlock conditions can be prevented + // where persist and push Executor try to put tasks in each other queues + // thus creating circular dependency + private volatile ListeningExecutorService intermediateTempExecutor = null; + private volatile long nextFlush; + private volatile FileLock basePersistDirLock = null; + private volatile FileChannel basePersistDirLockChannel = null; + + private volatile Throwable persistError; + + private final boolean isOpenSegments; + /** + * Use next Map to store metadata (File, SegmentId) for a hydrant for batch appenderator + * in order to facilitate the mapping of the QueryableIndex associated with a given hydrant + * at merge time. This is necessary since batch appenderator will not map the QueryableIndex + * at persist time in order to minimize its memory footprint. This has to be synchronized since the + * map may be accessed from multiple threads. + * Use {@link IdentityHashMap} to better reflect the fact that the key needs to be interpreted + * with reference semantics. + */ + private final Map> persistedHydrantMetadata = + Collections.synchronizedMap(new IdentityHashMap<>()); + + /** + * This constructor allows the caller to provide its own SinkQuerySegmentWalker. + * + * The sinkTimeline is set to the sink timeline of the provided SinkQuerySegmentWalker. + * If the SinkQuerySegmentWalker is null, a new sink timeline is initialized. + * + * It is used by UnifiedIndexerAppenderatorsManager which allows queries on data associated with multiple + * Appenderators. + */ + AppenderatorImpl( + String id, + DataSchema schema, + AppenderatorConfig tuningConfig, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + DataSegmentAnnouncer segmentAnnouncer, + @Nullable SinkQuerySegmentWalker sinkQuerySegmentWalker, + IndexIO indexIO, + IndexMerger indexMerger, + Cache cache, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler, + boolean isOpenSegments + ) + { + this.myId = id; + this.schema = Preconditions.checkNotNull(schema, "schema"); + this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); + this.metrics = Preconditions.checkNotNull(metrics, "metrics"); + this.dataSegmentPusher = Preconditions.checkNotNull(dataSegmentPusher, "dataSegmentPusher"); + this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper"); + this.segmentAnnouncer = Preconditions.checkNotNull(segmentAnnouncer, "segmentAnnouncer"); + this.indexIO = Preconditions.checkNotNull(indexIO, "indexIO"); + this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger"); + this.cache = cache; + this.texasRanger = sinkQuerySegmentWalker; + this.rowIngestionMeters = Preconditions.checkNotNull(rowIngestionMeters, "rowIngestionMeters"); + this.parseExceptionHandler = Preconditions.checkNotNull(parseExceptionHandler, "parseExceptionHandler"); + this.isOpenSegments = isOpenSegments; + + if (sinkQuerySegmentWalker == null) { + this.sinkTimeline = new VersionedIntervalTimeline<>( + String.CASE_INSENSITIVE_ORDER + ); + } else { + this.sinkTimeline = sinkQuerySegmentWalker.getSinkTimeline(); + } + + maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault(); + skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck(); + + if (isOpenSegments) { + log.info("Running open segments appenderator"); + } else { + log.info("Running closed segments appenderator"); + } + + } + + @Override + public String getId() + { + return myId; + } + + @Override + public String getDataSource() + { + return schema.getDataSource(); + } + + @Override + public Object startJob() + { + tuningConfig.getBasePersistDirectory().mkdirs(); + lockBasePersistDirectory(); + final Object retVal = bootstrapSinksFromDisk(); + initializeExecutors(); + resetNextFlush(); + return retVal; + } + + private void throwPersistErrorIfExists() + { + if (persistError != null) { + throw new RE(persistError, "Error while persisting"); + } + } + + @Override + public AppenderatorAddResult add( + final SegmentIdWithShardSpec identifier, + final InputRow row, + @Nullable final Supplier committerSupplier, + final boolean allowIncrementalPersists + ) throws IndexSizeExceededException, SegmentNotWritableException + { + throwPersistErrorIfExists(); + + if (!identifier.getDataSource().equals(schema.getDataSource())) { + throw new IAE( + "Expected dataSource[%s] but was asked to insert row for dataSource[%s]?!", + schema.getDataSource(), + identifier.getDataSource() + ); + } + + final Sink sink = getOrCreateSink(identifier); + metrics.reportMessageMaxTimestamp(row.getTimestampFromEpoch()); + final int sinkRowsInMemoryBeforeAdd = sink.getNumRowsInMemory(); + final int sinkRowsInMemoryAfterAdd; + final long bytesInMemoryBeforeAdd = sink.getBytesInMemory(); + final long bytesInMemoryAfterAdd; + final IncrementalIndexAddResult addResult; + + try { + addResult = sink.add(row, !allowIncrementalPersists); + sinkRowsInMemoryAfterAdd = addResult.getRowCount(); + bytesInMemoryAfterAdd = addResult.getBytesInMemory(); + } + catch (IndexSizeExceededException e) { + // Uh oh, we can't do anything about this! We can't persist (commit metadata would be out of sync) and we + // can't add the row (it just failed). This should never actually happen, though, because we check + // sink.canAddRow after returning from add. + log.error(e, "Sink for segment[%s] was unexpectedly full!", identifier); + throw e; + } + + if (sinkRowsInMemoryAfterAdd < 0) { + throw new SegmentNotWritableException("Attempt to add row to swapped-out sink for segment[%s].", identifier); + } + + if (addResult.isRowAdded()) { + rowIngestionMeters.incrementProcessed(); + } else if (addResult.hasParseException()) { + parseExceptionHandler.handle(addResult.getParseException()); + } + + final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd; + rowsCurrentlyInMemory.addAndGet(numAddedRows); + bytesCurrentlyInMemory.addAndGet(bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd); + totalRows.addAndGet(numAddedRows); + + boolean isPersistRequired = false; + boolean persist = false; + List persistReasons = new ArrayList<>(); + + if (!sink.canAppendRow()) { + persist = true; + persistReasons.add("No more rows can be appended to sink"); + } + if (System.currentTimeMillis() > nextFlush) { + persist = true; + persistReasons.add(StringUtils.format( + "current time[%d] is greater than nextFlush[%d]", + System.currentTimeMillis(), + nextFlush + )); + } + if (rowsCurrentlyInMemory.get() >= tuningConfig.getMaxRowsInMemory()) { + persist = true; + persistReasons.add(StringUtils.format( + "rowsCurrentlyInMemory[%d] is greater than maxRowsInMemory[%d]", + rowsCurrentlyInMemory.get(), + tuningConfig.getMaxRowsInMemory() + )); + } + if (bytesCurrentlyInMemory.get() >= maxBytesTuningConfig) { + persist = true; + persistReasons.add(StringUtils.format( + "(estimated) bytesCurrentlyInMemory[%d] is greater than maxBytesInMemory[%d]", + bytesCurrentlyInMemory.get(), + maxBytesTuningConfig + )); + } + if (persist) { + if (allowIncrementalPersists) { + // persistAll clears rowsCurrentlyInMemory, no need to update it. + log.info("Flushing in-memory data to disk because %s.", String.join(",", persistReasons)); + + long bytesToBePersisted = 0L; + for (Map.Entry entry : sinks.entrySet()) { + final Sink sinkEntry = entry.getValue(); + if (sinkEntry != null) { + bytesToBePersisted += sinkEntry.getBytesInMemory(); + if (sinkEntry.swappable()) { + // After swapping the sink, we use memory mapped segment instead (but only for real time appenderators!). + // However, the memory mapped segment still consumes memory. + // These memory mapped segments are held in memory throughout the ingestion phase and permanently add to the bytesCurrentlyInMemory + int memoryStillInUse = calculateMMappedHydrantMemoryInUsed(sink.getCurrHydrant()); + bytesCurrentlyInMemory.addAndGet(memoryStillInUse); + } + } + } + + if (!skipBytesInMemoryOverheadCheck && bytesCurrentlyInMemory.get() - bytesToBePersisted > maxBytesTuningConfig) { + // We are still over maxBytesTuningConfig even after persisting. + // This means that we ran out of all available memory to ingest (due to overheads created as part of ingestion) + final String alertMessage = StringUtils.format( + "Task has exceeded safe estimated heap usage limits, failing " + + "(numSinks: [%d] numHydrantsAcrossAllSinks: [%d] totalRows: [%d])" + + "(bytesCurrentlyInMemory: [%d] - bytesToBePersisted: [%d] > maxBytesTuningConfig: [%d])", + sinks.size(), + sinks.values().stream().mapToInt(Iterables::size).sum(), + getTotalRowCount(), + bytesCurrentlyInMemory.get(), + bytesToBePersisted, + maxBytesTuningConfig + ); + final String errorMessage = StringUtils.format( + "%s.\nThis can occur when the overhead from too many intermediary segment persists becomes to " + + "great to have enough space to process additional input rows. This check, along with metering the overhead " + + "of these objects to factor into the 'maxBytesInMemory' computation, can be disabled by setting " + + "'skipBytesInMemoryOverheadCheck' to 'true' (note that doing so might allow the task to naturally encounter " + + "a 'java.lang.OutOfMemoryError'). Alternatively, 'maxBytesInMemory' can be increased which will cause an " + + "increase in heap footprint, but will allow for more intermediary segment persists to occur before " + + "reaching this condition.", + alertMessage + ); + log.makeAlert(alertMessage) + .addData("dataSource", schema.getDataSource()) + .emit(); + throw new RuntimeException(errorMessage); + } + + Futures.addCallback( + persistAll(committerSupplier == null ? null : committerSupplier.get()), + new FutureCallback() + { + @Override + public void onSuccess(@Nullable Object result) + { + // do nothing + } + + @Override + public void onFailure(Throwable t) + { + persistError = t; + } + } + ); + } else { + isPersistRequired = true; + } + } + return new AppenderatorAddResult(identifier, sink.getNumRows(), isPersistRequired); + } + + @Override + public List getSegments() + { + return ImmutableList.copyOf(sinks.keySet()); + } + + @Override + public int getRowCount(final SegmentIdWithShardSpec identifier) + { + final Sink sink = sinks.get(identifier); + + if (sink == null) { + throw new ISE("No such sink: %s", identifier); + } else { + return sink.getNumRows(); + } + } + + @Override + public int getTotalRowCount() + { + return totalRows.get(); + } + + @VisibleForTesting + int getRowsInMemory() + { + return rowsCurrentlyInMemory.get(); + } + + @VisibleForTesting + long getBytesCurrentlyInMemory() + { + return bytesCurrentlyInMemory.get(); + } + + @VisibleForTesting + long getBytesInMemory(SegmentIdWithShardSpec identifier) + { + final Sink sink = sinks.get(identifier); + + if (sink == null) { + throw new ISE("No such sink: %s", identifier); + } else { + return sink.getBytesInMemory(); + } + } + + private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) + { + Sink retVal = sinks.get(identifier); + + if (retVal == null) { + retVal = new Sink( + identifier.getInterval(), + schema, + identifier.getShardSpec(), + identifier.getVersion(), + tuningConfig.getAppendableIndexSpec(), + tuningConfig.getMaxRowsInMemory(), + maxBytesTuningConfig, + null + ); + bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed(retVal)); + + try { + segmentAnnouncer.announceSegment(retVal.getSegment()); + } + catch (IOException e) { + log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource()) + .addData("interval", retVal.getInterval()) + .emit(); + } + + sinks.put(identifier, retVal); + metrics.setSinkCount(sinks.size()); + sinkTimeline.add(retVal.getInterval(), retVal.getVersion(), identifier.getShardSpec().createChunk(retVal)); + } + + return retVal; + } + + @Override + public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) + { + if (texasRanger == null) { + throw new IllegalStateException("Don't query me, bro."); + } + + return texasRanger.getQueryRunnerForIntervals(query, intervals); + } + + @Override + public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) + { + if (texasRanger == null) { + throw new IllegalStateException("Don't query me, bro."); + } + + return texasRanger.getQueryRunnerForSegments(query, specs); + } + + @Override + public void clear() throws InterruptedException + { + // Drop commit metadata, then abandon all segments. + + try { + throwPersistErrorIfExists(); + + if (persistExecutor != null) { + final ListenableFuture uncommitFuture = persistExecutor.submit( + () -> { + try { + commitLock.lock(); + objectMapper.writeValue(computeCommitFile(), Committed.nil()); + } + finally { + commitLock.unlock(); + } + return null; + } + ); + + // Await uncommit. + uncommitFuture.get(); + + // Drop everything. + final List> futures = new ArrayList<>(); + for (Map.Entry entry : sinks.entrySet()) { + futures.add(abandonSegment(entry.getKey(), entry.getValue(), true)); + } + + // Re-initialize hydrant map: + persistedHydrantMetadata.clear(); + + // Await dropping. + Futures.allAsList(futures).get(); + } + } + catch (ExecutionException e) { + throw new RuntimeException(e); + } + } + + @Override + public ListenableFuture drop(final SegmentIdWithShardSpec identifier) + { + final Sink sink = sinks.get(identifier); + if (sink != null) { + return abandonSegment(identifier, sink, true); + } else { + return Futures.immediateFuture(null); + } + } + + @Override + public ListenableFuture persistAll(@Nullable final Committer committer) + { + throwPersistErrorIfExists(); + final Map currentHydrants = new HashMap<>(); + final List> indexesToPersist = new ArrayList<>(); + int numPersistedRows = 0; + long bytesPersisted = 0L; + MutableLong totalHydrantsCount = new MutableLong(); + MutableLong totalHydrantsPersisted = new MutableLong(); + final long totalSinks = sinks.size(); + for (Map.Entry entry : sinks.entrySet()) { + final SegmentIdWithShardSpec identifier = entry.getKey(); + final Sink sink = entry.getValue(); + if (sink == null) { + throw new ISE("No sink for identifier: %s", identifier); + } + final List hydrants = Lists.newArrayList(sink); + totalHydrantsCount.add(hydrants.size()); + currentHydrants.put(identifier.toString(), hydrants.size()); + numPersistedRows += sink.getNumRowsInMemory(); + bytesPersisted += sink.getBytesInMemory(); + + final int limit = sink.isWritable() ? hydrants.size() - 1 : hydrants.size(); + + // gather hydrants that have not been persisted: + for (FireHydrant hydrant : hydrants.subList(0, limit)) { + if (!hydrant.hasSwapped()) { + log.debug("Hydrant[%s] hasn't persisted yet, persisting. Segment[%s]", hydrant, identifier); + indexesToPersist.add(Pair.of(hydrant, identifier)); + totalHydrantsPersisted.add(1); + } + } + + if (sink.swappable()) { + // It is swappable. Get the old one to persist it and create a new one: + indexesToPersist.add(Pair.of(sink.swap(), identifier)); + totalHydrantsPersisted.add(1); + } + } + log.debug("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); + + final Object commitMetadata = committer == null ? null : committer.getMetadata(); + final Stopwatch runExecStopwatch = Stopwatch.createStarted(); + final Stopwatch persistStopwatch = Stopwatch.createStarted(); + AtomicLong totalPersistedRows = new AtomicLong(numPersistedRows); + final ListenableFuture future = persistExecutor.submit( + new Callable() + { + @Override + public Object call() throws IOException + { + try { + for (Pair pair : indexesToPersist) { + metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs)); + } + + if (committer != null) { + log.debug( + "Committing metadata[%s] for sinks[%s].", + commitMetadata, + Joiner.on(", ").join( + currentHydrants.entrySet() + .stream() + .map(entry -> StringUtils.format( + "%s:%d", + entry.getKey(), + entry.getValue() + )) + .collect(Collectors.toList()) + ) + ); + + committer.run(); + + try { + commitLock.lock(); + final Map commitHydrants = new HashMap<>(); + final Committed oldCommit = readCommit(); + if (oldCommit != null) { + // merge current hydrants with existing hydrants + commitHydrants.putAll(oldCommit.getHydrants()); + } + commitHydrants.putAll(currentHydrants); + writeCommit(new Committed(commitHydrants, commitMetadata)); + } + finally { + commitLock.unlock(); + } + } + + log.info( + "Flushed in-memory data with commit metadata [%s] for segments: %s", + commitMetadata, + indexesToPersist.stream() + .map(itp -> itp.rhs.asSegmentId().toString()) + .distinct() + .collect(Collectors.joining(", ")) + ); + log.info( + "Persisted stats: processed rows: [%d], persisted rows[%d], sinks: [%d], total fireHydrants (across sinks): [%d], persisted fireHydrants (across sinks): [%d]", + rowIngestionMeters.getProcessed(), + totalPersistedRows.get(), + totalSinks, + totalHydrantsCount.longValue(), + totalHydrantsPersisted.longValue() + ); + + // return null if committer is null + return commitMetadata; + } + catch (IOException e) { + metrics.incrementFailedPersists(); + throw e; + } + finally { + metrics.incrementNumPersists(); + metrics.incrementPersistTimeMillis(persistStopwatch.elapsed(TimeUnit.MILLISECONDS)); + persistStopwatch.stop(); + } + } + } + ); + + final long startDelay = runExecStopwatch.elapsed(TimeUnit.MILLISECONDS); + metrics.incrementPersistBackPressureMillis(startDelay); + if (startDelay > WARN_DELAY) { + log.warn("Ingestion was throttled for [%,d] millis because persists were pending.", startDelay); + } + runExecStopwatch.stop(); + resetNextFlush(); + + // NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes. + rowsCurrentlyInMemory.addAndGet(-numPersistedRows); + bytesCurrentlyInMemory.addAndGet(-bytesPersisted); + + log.info("Persisted rows[%,d] and (estimated) bytes[%,d]", numPersistedRows, bytesPersisted); + + return future; + } + + @Override + public ListenableFuture push( + final Collection identifiers, + @Nullable final Committer committer, + final boolean useUniquePath + ) + { + final Map theSinks = new HashMap<>(); + AtomicLong pushedHydrantsCount = new AtomicLong(); + for (final SegmentIdWithShardSpec identifier : identifiers) { + final Sink sink = sinks.get(identifier); + if (sink == null) { + throw new ISE("No sink for identifier: %s", identifier); + } + theSinks.put(identifier, sink); + if (sink.finishWriting()) { + totalRows.addAndGet(-sink.getNumRows()); + } + // count hydrants for stats: + pushedHydrantsCount.addAndGet(Iterables.size(sink)); + } + + return Futures.transform( + // We should always persist all segments regardless of the input because metadata should be committed for all + // segments. + persistAll(committer), + (Function) commitMetadata -> { + final List dataSegments = new ArrayList<>(); + + log.info("Preparing to push (stats): processed rows: [%d], sinks: [%d], fireHydrants (across sinks): [%d]", + rowIngestionMeters.getProcessed(), theSinks.size(), pushedHydrantsCount.get() + ); + + log.debug( + "Building and pushing segments: %s", + theSinks.keySet().stream().map(SegmentIdWithShardSpec::toString).collect(Collectors.joining(", ")) + ); + + for (Map.Entry entry : theSinks.entrySet()) { + if (droppingSinks.contains(entry.getKey())) { + log.warn("Skipping push of currently-dropping sink[%s]", entry.getKey()); + continue; + } + + final DataSegment dataSegment = mergeAndPush( + entry.getKey(), + entry.getValue(), + useUniquePath + ); + if (dataSegment != null) { + dataSegments.add(dataSegment); + } else { + log.warn("mergeAndPush[%s] returned null, skipping.", entry.getKey()); + } + } + + log.info("Push complete..."); + + return new SegmentsAndCommitMetadata(dataSegments, commitMetadata); + }, + pushExecutor + ); + } + + /** + * Insert a barrier into the merge-and-push queue. When this future resolves, all pending pushes will have finished. + * This is useful if we're going to do something that would otherwise potentially break currently in-progress + * pushes. + */ + private ListenableFuture pushBarrier() + { + return intermediateTempExecutor.submit( + (Runnable) () -> pushExecutor.submit(() -> {}) + ); + } + + /** + * Merge segment, push to deep storage. Should only be used on segments that have been fully persisted. Must only + * be run in the single-threaded pushExecutor. + * + * @param identifier sink identifier + * @param sink sink to push + * @param useUniquePath true if the segment should be written to a path with a unique identifier + * + * @return segment descriptor, or null if the sink is no longer valid + */ + @Nullable + private DataSegment mergeAndPush( + final SegmentIdWithShardSpec identifier, + final Sink sink, + final boolean useUniquePath + ) + { + // Bail out if this sink is null or otherwise not what we expect. + //noinspection ObjectEquality + if (sinks.get(identifier) != sink) { + log.warn("Sink for segment[%s] no longer valid, bailing out of mergeAndPush.", identifier); + return null; + } + + // Use a descriptor file to indicate that pushing has completed. + final File persistDir = computePersistDir(identifier); + final File mergedTarget = new File(persistDir, "merged"); + final File descriptorFile = computeDescriptorFile(identifier); + + // Sanity checks + for (FireHydrant hydrant : sink) { + if (sink.isWritable()) { + throw new ISE("Expected sink to be no longer writable before mergeAndPush for segment[%s].", identifier); + } + + synchronized (hydrant) { + if (!hydrant.hasSwapped()) { + throw new ISE("Expected sink to be fully persisted before mergeAndPush for segment[%s].", identifier); + } + } + } + + try { + if (descriptorFile.exists()) { + // Already pushed. + + if (useUniquePath) { + // Don't reuse the descriptor, because the caller asked for a unique path. Leave the old one as-is, since + // it might serve some unknown purpose. + log.debug( + "Segment[%s] already pushed, but we want a unique path, so will push again with a new path.", + identifier + ); + } else { + log.info("Segment[%s] already pushed, skipping.", identifier); + return objectMapper.readValue(descriptorFile, DataSegment.class); + } + } + + removeDirectory(mergedTarget); + + if (mergedTarget.exists()) { + throw new ISE("Merged target[%s] exists after removing?!", mergedTarget); + } + + final File mergedFile; + final long mergeFinishTime; + final long startTime = System.nanoTime(); + List indexes = new ArrayList<>(); + Closer closer = Closer.create(); + try { + for (FireHydrant fireHydrant : sink) { + + // if batch, swap/persist did not memory map the incremental index, we need it mapped now: + if (!isOpenSegments()) { + + // sanity + Pair persistedMetadata = persistedHydrantMetadata.get(fireHydrant); + if (persistedMetadata == null) { + throw new ISE("Persisted metadata for batch hydrant [%s] is null!", fireHydrant); + } + + File persistedFile = persistedMetadata.lhs; + SegmentId persistedSegmentId = persistedMetadata.rhs; + + // sanity: + if (persistedFile == null) { + throw new ISE("Persisted file for batch hydrant [%s] is null!", fireHydrant); + } else if (persistedSegmentId == null) { + throw new ISE( + "Persisted segmentId for batch hydrant in file [%s] is null!", + persistedFile.getPath() + ); + } + fireHydrant.swapSegment(new QueryableIndexSegment( + indexIO.loadIndex(persistedFile), + persistedSegmentId + )); + } + + Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); + final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex(); + log.debug("Segment[%s] adding hydrant[%s]", identifier, fireHydrant); + indexes.add(queryableIndex); + closer.register(segmentAndCloseable.rhs); + } + + mergedFile = indexMerger.mergeQueryableIndex( + indexes, + schema.getGranularitySpec().isRollup(), + schema.getAggregators(), + schema.getDimensionsSpec(), + mergedTarget, + tuningConfig.getIndexSpec(), + tuningConfig.getSegmentWriteOutMediumFactory(), + tuningConfig.getMaxColumnsToMerge() + ); + + mergeFinishTime = System.nanoTime(); + + log.debug("Segment[%s] built in %,dms.", identifier, (mergeFinishTime - startTime) / 1000000); + } + catch (Throwable t) { + throw closer.rethrow(t); + } + finally { + closer.close(); + } + + final DataSegment segmentToPush = sink.getSegment().withDimensions( + IndexMerger.getMergedDimensionsFromQueryableIndexes(indexes, schema.getDimensionsSpec()) + ); + + // Retry pushing segments because uploading to deep storage might fail especially for cloud storage types + final DataSegment segment = RetryUtils.retry( + // The appenderator is currently being used for the local indexing task and the Kafka indexing task. For the + // Kafka indexing task, pushers must use unique file paths in deep storage in order to maintain exactly-once + // semantics. + () -> dataSegmentPusher.push( + mergedFile, + segmentToPush, + useUniquePath + ), + exception -> exception instanceof Exception, + 5 + ); + + if (!isOpenSegments()) { + // Drop the queryable indexes behind the hydrants... they are not needed anymore and their + // mapped file references + // can generate OOMs during merge if enough of them are held back... + for (FireHydrant fireHydrant : sink) { + fireHydrant.swapSegment(null); + } + } + + final long pushFinishTime = System.nanoTime(); + + objectMapper.writeValue(descriptorFile, segment); + + log.info( + "Segment[%s] of %,d bytes " + + "built from %d incremental persist(s) in %,dms; " + + "pushed to deep storage in %,dms. " + + "Load spec is: %s", + identifier, + segment.getSize(), + indexes.size(), + (mergeFinishTime - startTime) / 1000000, + (pushFinishTime - mergeFinishTime) / 1000000, + objectMapper.writeValueAsString(segment.getLoadSpec()) + ); + + return segment; + } + catch (Exception e) { + metrics.incrementFailedHandoffs(); + log.warn(e, "Failed to push merged index for segment[%s].", identifier); + throw new RuntimeException(e); + } + } + + @Override + public void close() + { + if (!closed.compareAndSet(false, true)) { + log.debug("Appenderator already closed, skipping close() call."); + return; + } + + log.debug("Shutting down..."); + + final List> futures = new ArrayList<>(); + for (Map.Entry entry : sinks.entrySet()) { + futures.add(abandonSegment(entry.getKey(), entry.getValue(), false)); + } + + try { + Futures.allAsList(futures).get(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + log.warn(e, "Interrupted during close()"); + } + catch (ExecutionException e) { + log.warn(e, "Unable to abandon existing segments during close()"); + } + + try { + shutdownExecutors(); + Preconditions.checkState( + persistExecutor == null || persistExecutor.awaitTermination(365, TimeUnit.DAYS), + "persistExecutor not terminated" + ); + Preconditions.checkState( + pushExecutor == null || pushExecutor.awaitTermination(365, TimeUnit.DAYS), + "pushExecutor not terminated" + ); + Preconditions.checkState( + intermediateTempExecutor == null || intermediateTempExecutor.awaitTermination(365, TimeUnit.DAYS), + "intermediateTempExecutor not terminated" + ); + persistExecutor = null; + pushExecutor = null; + intermediateTempExecutor = null; + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ISE("Failed to shutdown executors during close()"); + } + + // Only unlock if executors actually shut down. + unlockBasePersistDirectory(); + } + + /** + * Unannounce the segments and wait for outstanding persists to finish. + * Do not unlock base persist dir as we are not waiting for push executor to shut down + * relying on current JVM to shutdown to not cause any locking problem if the task is restored. + * In case when task is restored and current task is still active because of push executor (which it shouldn't be + * since push executor starts daemon threads) then the locking should fail and new task should fail to start. + * This also means that this method should only be called when task is shutting down. + */ + @Override + public void closeNow() + { + if (!closed.compareAndSet(false, true)) { + log.debug("Appenderator already closed, skipping closeNow() call."); + return; + } + + log.debug("Shutting down immediately..."); + for (Map.Entry entry : sinks.entrySet()) { + try { + segmentAnnouncer.unannounceSegment(entry.getValue().getSegment()); + } + catch (Exception e) { + log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) + .addData("identifier", entry.getKey().toString()) + .emit(); + } + } + try { + shutdownExecutors(); + // We don't wait for pushExecutor to be terminated. See Javadoc for more details. + Preconditions.checkState( + persistExecutor == null || persistExecutor.awaitTermination(365, TimeUnit.DAYS), + "persistExecutor not terminated" + ); + Preconditions.checkState( + intermediateTempExecutor == null || intermediateTempExecutor.awaitTermination(365, TimeUnit.DAYS), + "intermediateTempExecutor not terminated" + ); + persistExecutor = null; + intermediateTempExecutor = null; + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ISE("Failed to shutdown executors during close()"); + } + } + + public boolean isOpenSegments() + { + return isOpenSegments; + } + + private void lockBasePersistDirectory() + { + if (basePersistDirLock == null) { + try { + basePersistDirLockChannel = FileChannel.open( + computeLockFile().toPath(), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE + ); + + basePersistDirLock = basePersistDirLockChannel.tryLock(); + if (basePersistDirLock == null) { + throw new ISE("Cannot acquire lock on basePersistDir: %s", computeLockFile()); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + private void unlockBasePersistDirectory() + { + try { + if (basePersistDirLock != null) { + basePersistDirLock.release(); + basePersistDirLockChannel.close(); + basePersistDirLock = null; + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + private void initializeExecutors() + { + final int maxPendingPersists = tuningConfig.getMaxPendingPersists(); + + if (persistExecutor == null) { + // use a blocking single threaded executor to throttle the firehose when write to disk is slow + persistExecutor = MoreExecutors.listeningDecorator( + Execs.newBlockingSingleThreaded( + "[" + StringUtils.encodeForFormat(myId) + "]-appenderator-persist", + maxPendingPersists + ) + ); + } + + if (pushExecutor == null) { + // use a blocking single threaded executor to throttle the firehose when write to disk is slow + pushExecutor = MoreExecutors.listeningDecorator( + Execs.newBlockingSingleThreaded("[" + StringUtils.encodeForFormat(myId) + "]-appenderator-merge", 1) + ); + } + + if (intermediateTempExecutor == null) { + // use single threaded executor with SynchronousQueue so that all abandon operations occur sequentially + intermediateTempExecutor = MoreExecutors.listeningDecorator( + Execs.newBlockingSingleThreaded("[" + StringUtils.encodeForFormat(myId) + "]-appenderator-abandon", 0) + ); + } + } + + private void shutdownExecutors() + { + if (persistExecutor != null) { + persistExecutor.shutdownNow(); + } + + if (pushExecutor != null) { + pushExecutor.shutdownNow(); + } + + if (intermediateTempExecutor != null) { + intermediateTempExecutor.shutdownNow(); + } + } + + private void resetNextFlush() + { + nextFlush = DateTimes.nowUtc().plus(tuningConfig.getIntermediatePersistPeriod()).getMillis(); + } + + /** + * Populate "sinks" and "sinkTimeline" with committed segments, and announce them with the segmentAnnouncer. + * + * @return persisted commit metadata + */ + private Object bootstrapSinksFromDisk() + { + Preconditions.checkState(sinks.isEmpty(), "Already bootstrapped?!"); + + final File baseDir = tuningConfig.getBasePersistDirectory(); + if (!baseDir.exists()) { + return null; + } + + final File[] files = baseDir.listFiles(); + if (files == null) { + return null; + } + + + final Committed committed; + File commitFile = null; + try { + commitLock.lock(); + commitFile = computeCommitFile(); + if (commitFile.exists()) { + committed = objectMapper.readValue(commitFile, Committed.class); + } else { + committed = Committed.nil(); + } + } + catch (Exception e) { + throw new ISE(e, "Failed to read commitFile: %s", commitFile); + } + finally { + commitLock.unlock(); + } + + int rowsSoFar = 0; + + if (committed.equals(Committed.nil())) { + log.debug("No previously committed metadata."); + } else { + log.info( + "Loading partially-persisted segments[%s] from[%s] with commit metadata: %s", + String.join(", ", committed.getHydrants().keySet()), + baseDir, + committed.getMetadata() + ); + } + + for (File sinkDir : files) { + final File identifierFile = new File(sinkDir, IDENTIFIER_FILE_NAME); + if (!identifierFile.isFile()) { + // No identifier in this sinkDir; it must not actually be a sink directory. Skip it. + continue; + } + + try { + final SegmentIdWithShardSpec identifier = objectMapper.readValue( + new File(sinkDir, "identifier.json"), + SegmentIdWithShardSpec.class + ); + + final int committedHydrants = committed.getCommittedHydrants(identifier.toString()); + + if (committedHydrants <= 0) { + log.info("Removing uncommitted segment at [%s].", sinkDir); + FileUtils.deleteDirectory(sinkDir); + continue; + } + + // To avoid reading and listing of "merged" dir and other special files + final File[] sinkFiles = sinkDir.listFiles( + (dir, fileName) -> !(Ints.tryParse(fileName) == null) + ); + + Arrays.sort( + sinkFiles, + (o1, o2) -> Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName())) + ); + + List hydrants = new ArrayList<>(); + for (File hydrantDir : sinkFiles) { + final int hydrantNumber = Integer.parseInt(hydrantDir.getName()); + + if (hydrantNumber >= committedHydrants) { + log.info("Removing uncommitted partial segment at [%s]", hydrantDir); + FileUtils.deleteDirectory(hydrantDir); + } else { + log.debug("Loading previously persisted partial segment at [%s]", hydrantDir); + if (hydrantNumber != hydrants.size()) { + throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir); + } + + hydrants.add( + new FireHydrant( + new QueryableIndexSegment(indexIO.loadIndex(hydrantDir), identifier.asSegmentId()), + hydrantNumber + ) + ); + } + } + + // Make sure we loaded enough hydrants. + if (committedHydrants != hydrants.size()) { + throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir); + } + + Sink currSink = new Sink( + identifier.getInterval(), + schema, + identifier.getShardSpec(), + identifier.getVersion(), + tuningConfig.getAppendableIndexSpec(), + tuningConfig.getMaxRowsInMemory(), + maxBytesTuningConfig, + null, + hydrants + ); + rowsSoFar += currSink.getNumRows(); + sinks.put(identifier, currSink); + sinkTimeline.add( + currSink.getInterval(), + currSink.getVersion(), + identifier.getShardSpec().createChunk(currSink) + ); + + segmentAnnouncer.announceSegment(currSink.getSegment()); + } + catch (IOException e) { + log.makeAlert(e, "Problem loading sink[%s] from disk.", schema.getDataSource()) + .addData("sinkDir", sinkDir) + .emit(); + } + } + + // Make sure we loaded all committed sinks. + final Set loadedSinks = Sets.newHashSet( + Iterables.transform(sinks.keySet(), SegmentIdWithShardSpec::toString) + ); + final Set missingSinks = Sets.difference(committed.getHydrants().keySet(), loadedSinks); + if (!missingSinks.isEmpty()) { + throw new ISE("Missing committed sinks [%s]", Joiner.on(", ").join(missingSinks)); + } + + totalRows.set(rowsSoFar); + return committed.getMetadata(); + } + + private ListenableFuture abandonSegment( + final SegmentIdWithShardSpec identifier, + final Sink sink, + final boolean removeOnDiskData + ) + { + // Ensure no future writes will be made to this sink. + if (sink.finishWriting()) { + // Decrement this sink's rows from the counters. we only count active sinks so that we don't double decrement, + // i.e. those that haven't been persisted for *InMemory counters, or pushed to deep storage for the total counter. + rowsCurrentlyInMemory.addAndGet(-sink.getNumRowsInMemory()); + bytesCurrentlyInMemory.addAndGet(-sink.getBytesInMemory()); + bytesCurrentlyInMemory.addAndGet(-calculateSinkMemoryInUsed(sink)); + for (FireHydrant hydrant : sink) { + // Decrement memory used by all Memory Mapped Hydrant + if (!hydrant.equals(sink.getCurrHydrant())) { + bytesCurrentlyInMemory.addAndGet(-calculateMMappedHydrantMemoryInUsed(hydrant)); + } + } + totalRows.addAndGet(-sink.getNumRows()); + } + + // Mark this identifier as dropping, so no future push tasks will pick it up. + droppingSinks.add(identifier); + + // Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread. + return Futures.transform( + pushBarrier(), + new Function() + { + @Nullable + @Override + public Void apply(@Nullable Object input) + { + if (!sinks.remove(identifier, sink)) { + log.error("Sink for segment[%s] no longer valid, not abandoning.", identifier); + return null; + } + + metrics.setSinkCount(sinks.size()); + + if (removeOnDiskData) { + // Remove this segment from the committed list. This must be done from the persist thread. + log.debug("Removing commit metadata for segment[%s].", identifier); + try { + commitLock.lock(); + final Committed oldCommit = readCommit(); + if (oldCommit != null) { + writeCommit(oldCommit.without(identifier.toString())); + } + } + catch (Exception e) { + log.makeAlert(e, "Failed to update committed segments[%s]", schema.getDataSource()) + .addData("identifier", identifier.toString()) + .emit(); + throw new RuntimeException(e); + } + finally { + commitLock.unlock(); + } + } + + // Unannounce the segment. + try { + segmentAnnouncer.unannounceSegment(sink.getSegment()); + } + catch (Exception e) { + log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) + .addData("identifier", identifier.toString()) + .emit(); + } + + droppingSinks.remove(identifier); + sinkTimeline.remove( + sink.getInterval(), + sink.getVersion(), + identifier.getShardSpec().createChunk(sink) + ); + for (FireHydrant hydrant : sink) { + if (cache != null) { + cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant)); + } + hydrant.swapSegment(null); + // remove hydrant from persisted metadata: + persistedHydrantMetadata.remove(hydrant); + } + + if (removeOnDiskData) { + removeDirectory(computePersistDir(identifier)); + } + + log.info("Dropped segment[%s].", identifier); + + return null; + } + }, + // use persistExecutor to make sure that all the pending persists completes before + // starting to abandon segments + persistExecutor + ); + } + + private Committed readCommit() throws IOException + { + final File commitFile = computeCommitFile(); + if (commitFile.exists()) { + // merge current hydrants with existing hydrants + return objectMapper.readValue(commitFile, Committed.class); + } else { + return null; + } + } + + private void writeCommit(Committed newCommit) throws IOException + { + final File commitFile = computeCommitFile(); + objectMapper.writeValue(commitFile, newCommit); + } + + private File computeCommitFile() + { + return new File(tuningConfig.getBasePersistDirectory(), "commit.json"); + } + + private File computeLockFile() + { + return new File(tuningConfig.getBasePersistDirectory(), ".lock"); + } + + private File computePersistDir(SegmentIdWithShardSpec identifier) + { + return new File(tuningConfig.getBasePersistDirectory(), identifier.toString()); + } + + private File computeIdentifierFile(SegmentIdWithShardSpec identifier) + { + return new File(computePersistDir(identifier), IDENTIFIER_FILE_NAME); + } + + private File computeDescriptorFile(SegmentIdWithShardSpec identifier) + { + return new File(computePersistDir(identifier), "descriptor.json"); + } + + private File createPersistDirIfNeeded(SegmentIdWithShardSpec identifier) throws IOException + { + final File persistDir = computePersistDir(identifier); + org.apache.commons.io.FileUtils.forceMkdir(persistDir); + + objectMapper.writeValue(computeIdentifierFile(identifier), identifier); + + return persistDir; + } + + /** + * Persists the given hydrant and returns the number of rows persisted. Must only be called in the single-threaded + * persistExecutor. + * + * @param indexToPersist hydrant to persist + * @param identifier the segment this hydrant is going to be part of + * + * @return the number of rows persisted + */ + private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec identifier) + { + synchronized (indexToPersist) { + if (indexToPersist.hasSwapped()) { + log.info( + "Segment[%s] hydrant[%s] already swapped. Ignoring request to persist.", + identifier, + indexToPersist + ); + return 0; + } + + log.debug("Segment[%s], persisting Hydrant[%s]", identifier, indexToPersist); + + try { + final long startTime = System.nanoTime(); + int numRows = indexToPersist.getIndex().size(); + + final File persistedFile; + final File persistDir = createPersistDirIfNeeded(identifier); + persistedFile = indexMerger.persist( + indexToPersist.getIndex(), + identifier.getInterval(), + new File(persistDir, String.valueOf(indexToPersist.getCount())), + tuningConfig.getIndexSpecForIntermediatePersists(), + tuningConfig.getSegmentWriteOutMediumFactory() + ); + + log.info( + "Flushed in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).", + indexToPersist.getSegmentId(), + indexToPersist.getCount(), + (System.nanoTime() - startTime) / 1000000, + numRows + ); + + // Map only when this appenderator is being driven by a real time task: + Segment segmentToSwap = null; + if (isOpenSegments()) { + segmentToSwap = new QueryableIndexSegment(indexIO.loadIndex(persistedFile), indexToPersist.getSegmentId()); + } else { + // remember file path & segment id to rebuild the queryable index for merge: + persistedHydrantMetadata.put(indexToPersist, new Pair<>(persistedFile, indexToPersist.getSegmentId())); + } + indexToPersist.swapSegment(segmentToSwap); + + return numRows; + } + catch (IOException e) { + log.makeAlert("Incremental persist failed") + .addData("segment", identifier.toString()) + .addData("dataSource", schema.getDataSource()) + .addData("count", indexToPersist.getCount()) + .emit(); + + throw new RuntimeException(e); + } + } + } + + private void removeDirectory(final File target) + { + if (target.exists()) { + try { + FileUtils.deleteDirectory(target); + } + catch (Exception e) { + log.makeAlert(e, "Failed to remove directory[%s]", schema.getDataSource()) + .addData("file", target) + .emit(); + } + } + } + + private int calculateMMappedHydrantMemoryInUsed(FireHydrant hydrant) + { + if (skipBytesInMemoryOverheadCheck) { + return 0; + } + // These calculations are approximated from actual heap dumps. + // Memory footprint includes count integer in FireHydrant, shorts in ReferenceCountingSegment, + // Objects in SimpleQueryableIndex (such as SmooshedFileMapper, each ColumnHolder in column map, etc.) + int total; + total = Integer.BYTES + (4 * Short.BYTES) + ROUGH_OVERHEAD_PER_HYDRANT; + if (isOpenSegments()) { + // for real time add references to byte memory mapped references.. + total += (hydrant.getSegmentNumDimensionColumns() * ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER) + + (hydrant.getSegmentNumMetricColumns() * ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + + ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER; + } + return total; + } + + private int calculateSinkMemoryInUsed(Sink sink) + { + if (skipBytesInMemoryOverheadCheck) { + return 0; + } + // Rough estimate of memory footprint of empty Sink based on actual heap dumps + return ROUGH_OVERHEAD_PER_SINK; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 743df6bb4eb..08949cceaa5 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -102,30 +102,12 @@ public class Appenderators IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useLegacyBatchProcessing + ParseExceptionHandler parseExceptionHandler ) { - if (useLegacyBatchProcessing) { - // fallback to code known to be working, this is just a fallback option in case new - // batch appenderator has some early bugs but we will remove this fallback as soon as - // we determine that batch appenderator code is stable - return new StreamAppenderator( - id, - schema, - config, - metrics, - dataSegmentPusher, - objectMapper, - new NoopDataSegmentAnnouncer(), - null, - indexIO, - indexMerger, - null, - rowIngestionMeters, - parseExceptionHandler - ); - } + // Use newest, slated to be the permanent batch appenderator but for now keeping it as a non-default + // option due to risk mitigation...will become default and the two other appenderators eliminated when + // stability is proven... return new BatchAppenderator( id, schema, @@ -139,4 +121,69 @@ public class Appenderators parseExceptionHandler ); } + + public static Appenderator createOpenSegmentsOffline( + String id, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler + ) + { + // fallback to original code known to be working, this is just a fallback option in case new + // batch appenderator has some early bugs but we will remove this fallback as soon as + // we determine that batch appenderator code is stable + return new AppenderatorImpl( + id, + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + new NoopDataSegmentAnnouncer(), + null, + indexIO, + indexMerger, + null, + rowIngestionMeters, + parseExceptionHandler, + true + ); + } + + public static Appenderator createClosedSegmentsOffline( + String id, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler + ) + { + return new AppenderatorImpl( + id, + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + new NoopDataSegmentAnnouncer(), + null, + indexIO, + indexMerger, + null, + rowIngestionMeters, + parseExceptionHandler, + false + ); + } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java index 3e25bc5b097..f578a20e360 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java @@ -87,6 +87,32 @@ public interface AppenderatorsManager /** * Creates an Appenderator suited for batch ingestion. */ + Appenderator createOpenSegmentsOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler + ); + + Appenderator createClosedSegmentsOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler + ); + Appenderator createOfflineAppenderatorForTask( String taskId, DataSchema schema, @@ -97,8 +123,7 @@ public interface AppenderatorsManager IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useLegacyBatchProcessing + ParseExceptionHandler parseExceptionHandler ); /** diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 5c4bd2b5609..f4a212d551c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -21,23 +21,30 @@ package org.apache.druid.segment.realtime.appenderator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Stopwatch; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.Query; @@ -61,7 +68,6 @@ import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import javax.annotation.Nullable; -import javax.annotation.concurrent.NotThreadSafe; import java.io.Closeable; import java.io.File; import java.io.IOException; @@ -75,6 +81,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -87,12 +94,7 @@ import java.util.stream.Collectors; * hydrant creation phase (append) of this class or in the merge hydrants phase. Therefore, a new class, * {@code BatchAppenderator}, this class, was created to specialize in batch ingestion and the old class * for stream ingestion was renamed to {@link StreamAppenderator}. - *

- * This class is not thread safe!. - * It is important to realize that this class is completely synchronous despite the {@link Appenderator} - * interface suggesting otherwise. The concurrency was not required so it has been completely removed. */ -@NotThreadSafe public class BatchAppenderator implements Appenderator { public static final int ROUGH_OVERHEAD_PER_SINK = 5000; @@ -110,15 +112,30 @@ public class BatchAppenderator implements Appenderator private final ObjectMapper objectMapper; private final IndexIO indexIO; private final IndexMerger indexMerger; - private final Map sinks = new HashMap<>(); private final long maxBytesTuningConfig; private final boolean skipBytesInMemoryOverheadCheck; + private volatile ListeningExecutorService persistExecutor = null; + private volatile ListeningExecutorService pushExecutor = null; + private final int maxPendingPersists; + private static final int PERSIST_WARN_DELAY = 1000; + private volatile Throwable persistError; + + + /** + * The following map used to be accessed concurrently but not anymore since it is fully copied + * then initalized just before scheduling the persit callable in the {@link #persistAll(Committer)} + * method, so no longer need to use a syncronized map. + */ + private final Map sinks = new HashMap<>(); /** * The following sinks metadata map and associated class are the way to retain metadata now that sinks - * are being completely removed from memory after each incremental persist. + * are being completely removed from memory after each incremental persist. This map needs to be concurrent + * since it is mutated in various methods potentially in different threads. + * For example mutated in {@link #add} when adding rows + * and accessed/mutated in {@link #persistHydrant} during persists. */ - private final Map sinksMetadata = new HashMap<>(); + private final ConcurrentHashMap sinksMetadata = new ConcurrentHashMap<>(); // This variable updated in add(), persist(), and drop() private int rowsCurrentlyInMemory = 0; @@ -158,6 +175,7 @@ public class BatchAppenderator implements Appenderator maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault(); skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck(); + maxPendingPersists = tuningConfig.getMaxPendingPersists(); } @Override @@ -177,9 +195,53 @@ public class BatchAppenderator implements Appenderator { tuningConfig.getBasePersistDirectory().mkdirs(); lockBasePersistDirectory(); + initializeExecutors(); return null; } + private void throwPersistErrorIfExists() + { + if (persistError != null) { + throw new RE(persistError, "Error while persisting"); + } + } + + private void initializeExecutors() + { + log.debug("There will be up to[%d] pending persists", maxPendingPersists); + + if (persistExecutor == null) { + // use a blocking single threaded executor to throttle the firehose when write to disk is slow + persistExecutor = MoreExecutors.listeningDecorator( + Execs.newBlockingSingleThreaded( + "[" + StringUtils.encodeForFormat(myId) + "]-batch-appenderator-persist", + maxPendingPersists + ) + ); + } + + if (pushExecutor == null) { + // use a blocking single threaded executor to throttle the firehose when write to disk is slow + pushExecutor = MoreExecutors.listeningDecorator( + Execs.newBlockingSingleThreaded( + "[" + StringUtils.encodeForFormat(myId) + "]-batch-appenderator-push", + 1 + ) + ); + } + + } + + private void shutdownExecutors() + { + if (persistExecutor != null) { + persistExecutor.shutdownNow(); + } + if (pushExecutor != null) { + pushExecutor.shutdownNow(); + } + } + @Override public AppenderatorAddResult add( final SegmentIdWithShardSpec identifier, @@ -189,6 +251,8 @@ public class BatchAppenderator implements Appenderator ) throws IndexSizeExceededException, SegmentNotWritableException { + throwPersistErrorIfExists(); + Preconditions.checkArgument( committerSupplier == null, "Batch appenderator does not need a committer!" @@ -240,7 +304,7 @@ public class BatchAppenderator implements Appenderator final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd; rowsCurrentlyInMemory += numAddedRows; - bytesCurrentlyInMemory += bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd; + bytesCurrentlyInMemory += (bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd); totalRows += numAddedRows; sinksMetadata.computeIfAbsent(identifier, unused -> new SinkMetadata()).addRows(numAddedRows); @@ -267,6 +331,7 @@ public class BatchAppenderator implements Appenderator maxBytesTuningConfig )); } + if (persist) { // persistAll clears rowsCurrentlyInMemory, no need to update it. log.info("Incremental persist to disk because %s.", String.join(",", persistReasons)); @@ -277,7 +342,7 @@ public class BatchAppenderator implements Appenderator if (sinkEntry != null) { bytesToBePersisted += sinkEntry.getBytesInMemory(); if (sinkEntry.swappable()) { - // Code for batch no longer memory maps hydrants but they still take memory... + // Code for batch no longer memory maps hydrants, but they still take memory... int memoryStillInUse = calculateMemoryUsedByHydrant(); bytesCurrentlyInMemory += memoryStillInUse; } @@ -315,8 +380,23 @@ public class BatchAppenderator implements Appenderator throw new RuntimeException(errorMessage); } - persistAllAndRemoveSinks(); + Futures.addCallback( + persistAll(null), + new FutureCallback() + { + @Override + public void onSuccess(@Nullable Object result) + { + // do nothing + } + @Override + public void onFailure(Throwable t) + { + persistError = t; + } + } + ); } return new AppenderatorAddResult(identifier, sinksMetadata.get(identifier).numRowsInSegment, false); } @@ -388,7 +468,6 @@ public class BatchAppenderator implements Appenderator null ); bytesCurrentlyInMemory += calculateSinkMemoryInUsed(); - sinks.put(identifier, retVal); metrics.setSinkCount(sinks.size()); } @@ -411,20 +490,22 @@ public class BatchAppenderator implements Appenderator @Override public void clear() { - clear(true); + throwPersistErrorIfExists(); + clear(sinks, true); } - private void clear(boolean removeOnDiskData) + private void clear(Map sinksToClear, boolean removeOnDiskData) { // Drop commit metadata, then abandon all segments. - log.info("Clearing all[%d] sinks & their hydrants, removing data on disk: [%s]", sinks.size(), removeOnDiskData); + log.info("Clearing all[%d] sinks & their hydrants, removing data on disk: [%s]", + sinksToClear.size(), removeOnDiskData); // Drop everything. - Iterator> sinksIterator = sinks.entrySet().iterator(); + Iterator> sinksIterator = sinksToClear.entrySet().iterator(); sinksIterator.forEachRemaining(entry -> { - clearSinkMetadata(entry.getKey(), entry.getValue(), removeOnDiskData); + clearSinkMemoryCountersAndDiskStoredData(entry.getKey(), entry.getValue(), removeOnDiskData); sinksIterator.remove(); }); - metrics.setSinkCount(sinks.size()); + metrics.setSinkCount(sinksToClear.size()); } @Override @@ -442,7 +523,7 @@ public class BatchAppenderator implements Appenderator totalRows = Math.max(totalRowsAfter, 0); } if (sink != null) { - clearSinkMetadata(identifier, sink, true); + clearSinkMemoryCountersAndDiskStoredData(identifier, sink, true); if (sinks.remove(identifier) == null) { log.warn("Sink for identifier[%s] not found, skipping", identifier); } @@ -453,100 +534,124 @@ public class BatchAppenderator implements Appenderator @Override public ListenableFuture persistAll(@Nullable final Committer committer) { + + throwPersistErrorIfExists(); + if (committer != null) { throw new ISE("committer must be null for BatchAppenderator"); } - persistAllAndRemoveSinks(); - return Futures.immediateFuture(null); + // Get ready to persist all sinks: + final Map sinksToPersist = swapSinks(); + + final Stopwatch runExecStopwatch = Stopwatch.createStarted(); + ListenableFuture future = persistExecutor.submit( + () -> { + log.info("Spawning intermediate persist"); + + // figure out hydrants (indices) to persist: + final List> indexesToPersist = new ArrayList<>(); + int numPersistedRows = 0; + long bytesPersisted = 0; + int totalHydrantsCount = 0; + final long totalSinks = sinksToPersist.size(); + for (Map.Entry entry : sinksToPersist.entrySet()) { + final SegmentIdWithShardSpec identifier = entry.getKey(); + final Sink sink = entry.getValue(); + if (sink == null) { + throw new ISE("No sink for identifier: %s", identifier); + } + + final List hydrants = Lists.newArrayList(sink); + // Since everytime we persist we also get rid of the in-memory references to sink & hydrants + // the invariant of exactly one, always swappable, sink with exactly one unpersisted hydrant must hold + int totalHydrantsForSink = hydrants.size(); + if (totalHydrantsForSink != 1) { + throw new ISE("There should be only one hydrant for identifier[%s] but there are[%s]", + identifier, totalHydrantsForSink + ); + } + totalHydrantsCount++; + numPersistedRows += sink.getNumRowsInMemory(); + bytesPersisted += sink.getBytesInMemory(); + + if (!sink.swappable()) { + throw new ISE("Sink is not swappable![%s]", identifier); + } + indexesToPersist.add(Pair.of(sink.swap(), identifier)); + + } + + if (indexesToPersist.isEmpty()) { + log.info("No indexes will be persisted"); + } + final Stopwatch persistStopwatch = Stopwatch.createStarted(); + try { + for (Pair pair : indexesToPersist) { + metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs)); + } + + log.info( + "Persisted in-memory data for segments: %s", + indexesToPersist.stream() + .filter(itp -> itp.rhs != null) + .map(itp -> itp.rhs.asSegmentId().toString()) + .distinct() + .collect(Collectors.joining(", ")) + ); + log.info( + "Persisted stats: processed rows: [%d], persisted rows[%d], persisted sinks: [%d], persisted fireHydrants (across sinks): [%d]", + rowIngestionMeters.getProcessed(), + numPersistedRows, + totalSinks, + totalHydrantsCount + ); + + // note that we do not need to reset sinks metadata since we did it at the start... + + } + catch (Exception e) { + metrics.incrementFailedPersists(); + throw e; + } + finally { + metrics.incrementNumPersists(); + long persistMillis = persistStopwatch.elapsed(TimeUnit.MILLISECONDS); + metrics.incrementPersistTimeMillis(persistMillis); + persistStopwatch.stop(); + // make sure no push can start while persisting: + log.info("Persisted rows[%,d] and bytes[%,d] and removed all sinks & hydrants from memory in[%d] millis", + numPersistedRows, bytesPersisted, persistMillis + ); + log.info("Persist is done."); + } + return null; + } + ); + + final long startDelay = runExecStopwatch.elapsed(TimeUnit.MILLISECONDS); + metrics.incrementPersistBackPressureMillis(startDelay); + if (startDelay > PERSIST_WARN_DELAY) { + log.warn("Ingestion was throttled for [%,d] millis because persists were pending.", startDelay); + } + runExecStopwatch.stop(); + return future; } /** - * Persist all sinks & their hydrants, keep their metadata, and then remove them completely from - * memory (to be resurrected right before merge & push) + * All sinks will be persisted so do a shallow copy of the Sinks map, reset + * the map and metadata (i.e. memory consumption counters) so that ingestion can go on + * @return The map of sinks to persist, this map will be garbage collected after + * persist is complete since we will not be keeping a reference to it... */ - private void persistAllAndRemoveSinks() + Map swapSinks() { - - final List> indexesToPersist = new ArrayList<>(); - int numPersistedRows = 0; - long bytesPersisted = 0L; - int totalHydrantsCount = 0; - final long totalSinks = sinks.size(); - for (Map.Entry entry : sinks.entrySet()) { - final SegmentIdWithShardSpec identifier = entry.getKey(); - final Sink sink = entry.getValue(); - if (sink == null) { - throw new ISE("No sink for identifier: %s", identifier); - } - - final List hydrants = Lists.newArrayList(sink); - // Since everytime we persist we also get rid of the in-memory references to sinks & hydrants - // the invariant of exactly one, always swappable, sink with exactly one unpersisted hydrant must hold - int totalHydrantsForSink = hydrants.size(); - if (totalHydrantsForSink != 1) { - throw new ISE("There should be only one hydrant for identifier[%s] but there are[%s]", - identifier, totalHydrantsForSink - ); - } - totalHydrantsCount += 1; - numPersistedRows += sink.getNumRowsInMemory(); - bytesPersisted += sink.getBytesInMemory(); - - if (!sink.swappable()) { - throw new ISE("Sink is not swappable![%s]", identifier); - } - indexesToPersist.add(Pair.of(sink.swap(), identifier)); - - } - - if (indexesToPersist.isEmpty()) { - log.info("No indexes will be persisted"); - } - final Stopwatch persistStopwatch = Stopwatch.createStarted(); - try { - for (Pair pair : indexesToPersist) { - metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs)); - } - - log.info( - "Persisted in-memory data for segments: %s", - indexesToPersist.stream() - .filter(itp -> itp.rhs != null) - .map(itp -> itp.rhs.asSegmentId().toString()) - .distinct() - .collect(Collectors.joining(", ")) - ); - log.info( - "Persisted stats: processed rows: [%d], persisted rows[%d], persisted sinks: [%d], persisted fireHydrants (across sinks): [%d]", - rowIngestionMeters.getProcessed(), - numPersistedRows, - totalSinks, - totalHydrantsCount - ); - - } - catch (Exception e) { - metrics.incrementFailedPersists(); - throw e; - } - finally { - metrics.incrementNumPersists(); - metrics.incrementPersistTimeMillis(persistStopwatch.elapsed(TimeUnit.MILLISECONDS)); - persistStopwatch.stop(); - } - - // NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes. - rowsCurrentlyInMemory -= numPersistedRows; - bytesCurrentlyInMemory -= bytesPersisted; - - // remove all sinks after persisting: - clear(false); - - log.info("Persisted rows[%,d] and bytes[%,d] and removed all sinks & hydrants from memory", - numPersistedRows, bytesPersisted); - + Map retVal = ImmutableMap.copyOf(sinks); + sinks.clear(); + resetSinkMetadata(); + return retVal; } + @Override public ListenableFuture push( final Collection identifiers, @@ -563,53 +668,56 @@ public class BatchAppenderator implements Appenderator throw new ISE("Batch ingestion does not require uniquePath"); } - // Any sinks not persisted so far need to be persisted before push: - persistAllAndRemoveSinks(); - - log.info("Preparing to push..."); - - // Traverse identifiers, load their sink, and push it: - int totalHydrantsMerged = 0; final List dataSegments = new ArrayList<>(); - for (SegmentIdWithShardSpec identifier : identifiers) { - SinkMetadata sm = sinksMetadata.get(identifier); - if (sm == null) { - throw new ISE("No sink has been processed for identifier[%s]", identifier); - } - File persistedDir = sm.getPersistedFileDir(); - if (persistedDir == null) { - throw new ISE("Sink for identifier[%s] not found in local file system", identifier); - } - totalHydrantsMerged += sm.getNumHydrants(); - // retrieve sink from disk: - Sink sinkForIdentifier; - try { - sinkForIdentifier = getSinkForIdentifierPath(identifier, persistedDir); - } - catch (IOException e) { - throw new ISE(e, "Failed to retrieve sinks for identifier[%s] in path[%s]", identifier, persistedDir); - } + return Futures.transform( + persistAll(null), // make sure persists is done before push... + (Function) commitMetadata -> { - // push sink: - final DataSegment dataSegment = mergeAndPush( - identifier, - sinkForIdentifier - ); + log.info("Push started, processsing[%d] sinks", identifiers.size()); - // record it: - if (dataSegment != null) { - dataSegments.add(dataSegment); - } else { - log.warn("mergeAndPush[%s] returned null, skipping.", identifier); - } + int totalHydrantsMerged = 0; + for (SegmentIdWithShardSpec identifier : identifiers) { + SinkMetadata sm = sinksMetadata.get(identifier); + if (sm == null) { + throw new ISE("No sink has been processed for identifier[%s]", identifier); + } + File persistedDir = sm.getPersistedFileDir(); + if (persistedDir == null) { + throw new ISE("Persisted directory for identifier[%s] is null in sink metadata", identifier); + } + totalHydrantsMerged += sm.getNumHydrants(); - } + // retrieve sink from disk: + Sink sinkForIdentifier; + try { + sinkForIdentifier = getSinkForIdentifierPath(identifier, persistedDir); + } + catch (IOException e) { + throw new ISE(e, "Failed to retrieve sinks for identifier[%s]", identifier); + } - log.info("Push complete: total sinks merged[%d], total hydrants merged[%d]", - identifiers.size(), totalHydrantsMerged); + // push it: + final DataSegment dataSegment = mergeAndPush( + identifier, + sinkForIdentifier + ); - return Futures.immediateFuture(new SegmentsAndCommitMetadata(dataSegments, null)); + // record it: + if (dataSegment != null) { + dataSegments.add(dataSegment); + } else { + log.warn("mergeAndPush[%s] returned null, skipping.", identifier); + } + } + log.info("Push done: total sinks merged[%d], total hydrants merged[%d]", + identifiers.size(), totalHydrantsMerged + ); + return new SegmentsAndCommitMetadata(dataSegments, commitMetadata); + }, + pushExecutor // push it in the background, pushAndClear in BaseAppenderatorDriver guarantees + // that segments are dropped before next add row + ); } /** @@ -762,11 +870,37 @@ public class BatchAppenderator implements Appenderator log.debug("Shutting down..."); - clear(false); + try { - unlockBasePersistDirectory(); + log.debug("Shutdown & wait for persistExecutor"); + if (persistExecutor != null) { + persistExecutor.shutdown(); + if (!persistExecutor.awaitTermination(365, TimeUnit.DAYS)) { + log.warn("persistExecutor not terminated"); + } + persistExecutor = null; + } + + log.debug("Shutdown & wait for pushExecutor"); + if (pushExecutor != null) { + pushExecutor.shutdown(); + if (!pushExecutor.awaitTermination(365, TimeUnit.DAYS)) { + log.warn("pushExecutor not terminated"); + } + pushExecutor = null; + } + + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ISE("Failed to wait & shutdown executors during close()"); + } + + log.debug("Waited for and shutdown executors..."); // cleanup: + clear(sinks, false); + unlockBasePersistDirectory(); List persistedIdentifiers = getPersistedidentifierPaths(); if (persistedIdentifiers != null) { for (File identifier : persistedIdentifiers) { @@ -778,9 +912,6 @@ public class BatchAppenderator implements Appenderator sinksMetadata.clear(); } - /** - Nothing to do since there are no executors - */ @Override public void closeNow() { @@ -788,8 +919,8 @@ public class BatchAppenderator implements Appenderator log.debug("Appenderator already closed, skipping closeNow() call."); return; } - log.debug("Shutting down immediately..."); + shutdownExecutors(); } private void lockBasePersistDirectory() @@ -904,10 +1035,17 @@ public class BatchAppenderator implements Appenderator return retVal; } + private void resetSinkMetadata() + { + rowsCurrentlyInMemory = 0; + bytesCurrentlyInMemory = 0; + metrics.setSinkCount(0); + } + // This function does not remove the sink from its tracking Map (sinks), the caller is responsible for that // this is because the Map is not synchronized and removing elements from a map while traversing it - // throws a concurrent access exception - private void clearSinkMetadata( + // throws concurrent access exception + private void clearSinkMemoryCountersAndDiskStoredData( final SegmentIdWithShardSpec identifier, final Sink sink, final boolean removeOnDiskData @@ -926,7 +1064,7 @@ public class BatchAppenderator implements Appenderator bytesCurrentlyInMemory -= calculateMemoryUsedByHydrant(); } } - // totalRows are not decremented when removing the sink from memory, sink was just persisted and it + // totalRows are not decremented when removing the sink from memory, sink was just persisted, and it // still "lives" but it is in hibernation. It will be revived later just before push. } @@ -992,7 +1130,7 @@ public class BatchAppenderator implements Appenderator int numRows = indexToPersist.getIndex().size(); // since the sink may have been persisted before it may have lost its - // hydrant count, we remember that value in the sinks metadata so we have + // hydrant count, we remember that value in the sinks' metadata, so we have // to pull it from there.... SinkMetadata sm = sinksMetadata.get(identifier); if (sm == null) { @@ -1009,7 +1147,7 @@ public class BatchAppenderator implements Appenderator sm.setPersistedFileDir(persistDir); log.info( - "Persisted in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).", + "About to persist in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).", indexToPersist.getSegmentId(), indexToPersist.getCount(), (System.nanoTime() - startTime) / 1000000, @@ -1072,7 +1210,7 @@ public class BatchAppenderator implements Appenderator * This class is used for information that needs to be kept related to Sinks as * they are persisted and removed from memory at every incremental persist. * The information is used for sanity checks and as information required - * for functionality, depending in the field that is used. More info about the + * for functionality, depending on the field that is used. More info about the * fields is annotated as comments in the class */ private static class SinkMetadata diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java index 8abd0c29767..eb308528909 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java @@ -58,7 +58,7 @@ public class DefaultOfflineAppenderatorFactory implements AppenderatorFactory public Appenderator build(DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics) { final RowIngestionMeters rowIngestionMeters = new NoopRowIngestionMeters(); - return Appenderators.createOffline( + return Appenderators.createClosedSegmentsOffline( schema.getDataSource(), schema, config, @@ -73,8 +73,7 @@ public class DefaultOfflineAppenderatorFactory implements AppenderatorFactory false, config.isReportParseExceptions() ? 0 : Integer.MAX_VALUE, 0 - ), - false + ) ); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java index 0dcd7d61a76..699c32478a9 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java @@ -89,8 +89,41 @@ public class DummyForInjectionAppenderatorsManager implements AppenderatorsManag IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useLegacyBatchProcessing + ParseExceptionHandler parseExceptionHandler + ) + { + throw new UOE(ERROR_MSG); + } + + @Override + public Appenderator createOpenSegmentsOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler + ) + { + throw new UOE(ERROR_MSG); + } + + @Override + public Appenderator createClosedSegmentsOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler ) { throw new UOE(ERROR_MSG); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java index 1c44b301cc3..b666565125d 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java @@ -121,8 +121,7 @@ public class PeonAppenderatorsManager implements AppenderatorsManager IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useLegacyBatchProcessing + ParseExceptionHandler parseExceptionHandler ) { // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators @@ -139,13 +138,79 @@ public class PeonAppenderatorsManager implements AppenderatorsManager indexIO, indexMerger, rowIngestionMeters, - parseExceptionHandler, - useLegacyBatchProcessing + parseExceptionHandler ); return batchAppenderator; } } + @Override + public Appenderator createOpenSegmentsOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler + ) + { + // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators + if (realtimeAppenderator != null) { + throw new ISE("A realtime appenderator was already created for this peon's task."); + } else { + batchAppenderator = Appenderators.createOpenSegmentsOffline( + taskId, + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + parseExceptionHandler + ); + return batchAppenderator; + } + } + + @Override + public Appenderator createClosedSegmentsOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler + ) + { + // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators + if (realtimeAppenderator != null) { + throw new ISE("A realtime appenderator was already created for this peon's task."); + } else { + batchAppenderator = Appenderators.createClosedSegmentsOffline( + taskId, + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + parseExceptionHandler + ); + return batchAppenderator; + } + } @Override public void removeAppenderatorsForTask(String taskId, String dataSource) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index f390f767ae7..23a00c72a46 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -207,8 +207,7 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useLegacyBatchProcessing + ParseExceptionHandler parseExceptionHandler ) { synchronized (this) { @@ -227,8 +226,81 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager indexIO, wrapIndexMerger(indexMerger), rowIngestionMeters, - parseExceptionHandler, - useLegacyBatchProcessing + parseExceptionHandler + ); + datasourceBundle.addAppenderator(taskId, appenderator); + return appenderator; + } + } + + @Override + public Appenderator createOpenSegmentsOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler + ) + { + synchronized (this) { + DatasourceBundle datasourceBundle = datasourceBundles.computeIfAbsent( + schema.getDataSource(), + DatasourceBundle::new + ); + + Appenderator appenderator = Appenderators.createOpenSegmentsOffline( + taskId, + schema, + rewriteAppenderatorConfigMemoryLimits(config), + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + wrapIndexMerger(indexMerger), + rowIngestionMeters, + parseExceptionHandler + ); + datasourceBundle.addAppenderator(taskId, appenderator); + return appenderator; + } + } + + @Override + public Appenderator createClosedSegmentsOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler + ) + { + synchronized (this) { + DatasourceBundle datasourceBundle = datasourceBundles.computeIfAbsent( + schema.getDataSource(), + DatasourceBundle::new + ); + + Appenderator appenderator = Appenderators.createClosedSegmentsOffline( + taskId, + schema, + rewriteAppenderatorConfigMemoryLimits(config), + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + wrapIndexMerger(indexMerger), + rowIngestionMeters, + parseExceptionHandler ); datasourceBundle.addAppenderator(taskId, appenderator); return appenderator; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java similarity index 96% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java index af628ba2767..a3c09c2e27d 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java @@ -62,7 +62,7 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.CopyOnWriteArrayList; -public class BatchAppenderatorTester implements AutoCloseable +public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable { public static final String DATASOURCE = "foo"; @@ -75,14 +75,14 @@ public class BatchAppenderatorTester implements AutoCloseable private final List pushedSegments = new CopyOnWriteArrayList<>(); - public BatchAppenderatorTester( + public ClosedSegmensSinksBatchAppenderatorTester( final int maxRowsInMemory ) { this(maxRowsInMemory, -1, null, false); } - public BatchAppenderatorTester( + public ClosedSegmensSinksBatchAppenderatorTester( final int maxRowsInMemory, final boolean enablePushFailure ) @@ -90,7 +90,7 @@ public class BatchAppenderatorTester implements AutoCloseable this(maxRowsInMemory, -1, null, enablePushFailure); } - public BatchAppenderatorTester( + public ClosedSegmensSinksBatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final boolean enablePushFailure @@ -99,7 +99,7 @@ public class BatchAppenderatorTester implements AutoCloseable this(maxRowsInMemory, maxSizeInBytes, null, enablePushFailure); } - public BatchAppenderatorTester( + public ClosedSegmensSinksBatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final File basePersistDirectory, @@ -112,12 +112,11 @@ public class BatchAppenderatorTester implements AutoCloseable basePersistDirectory, enablePushFailure, new SimpleRowIngestionMeters(), - false, false ); } - public BatchAppenderatorTester( + public ClosedSegmensSinksBatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, @Nullable final File basePersistDirectory, @@ -126,18 +125,17 @@ public class BatchAppenderatorTester implements AutoCloseable ) { this(maxRowsInMemory, maxSizeInBytes, basePersistDirectory, enablePushFailure, rowIngestionMeters, - false, false + false ); } - public BatchAppenderatorTester( + public ClosedSegmensSinksBatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, @Nullable final File basePersistDirectory, final boolean enablePushFailure, final RowIngestionMeters rowIngestionMeters, - final boolean skipBytesInMemoryOverheadCheck, - final boolean useLegacyBatchProcessing + final boolean skipBytesInMemoryOverheadCheck ) { objectMapper = new DefaultObjectMapper(); @@ -248,8 +246,7 @@ public class BatchAppenderatorTester implements AutoCloseable indexIO, indexMerger, rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - useLegacyBatchProcessing + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0) ); } @@ -302,7 +299,7 @@ public class BatchAppenderatorTester implements AutoCloseable } - private static class TestIndexTuningConfig implements AppenderatorConfig + static class TestIndexTuningConfig implements AppenderatorConfig { private final AppendableIndexSpec appendableIndexSpec; private final int maxRowsInMemory; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java similarity index 96% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java index 2cf78cc937f..6f8f533646e 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java @@ -51,7 +51,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Collectors; -public class BatchAppenderatorDriverTest extends EasyMockSupport +public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupport { private static final String DATA_SOURCE = "foo"; private static final String VERSION = "abc123"; @@ -77,7 +77,7 @@ public class BatchAppenderatorDriverTest extends EasyMockSupport ); private SegmentAllocator allocator; - private BatchAppenderatorTester appenderatorTester; + private ClosedSegmensSinksBatchAppenderatorTester appenderatorTester; private BatchAppenderatorDriver driver; private DataSegmentKiller dataSegmentKiller; @@ -88,7 +88,7 @@ public class BatchAppenderatorDriverTest extends EasyMockSupport @Before public void setup() { - appenderatorTester = new BatchAppenderatorTester(MAX_ROWS_IN_MEMORY); + appenderatorTester = new ClosedSegmensSinksBatchAppenderatorTester(MAX_ROWS_IN_MEMORY); allocator = new TestSegmentAllocator(DATA_SOURCE, Granularities.HOUR); dataSegmentKiller = createStrictMock(DataSegmentKiller.class); driver = new BatchAppenderatorDriver( @@ -142,7 +142,7 @@ public class BatchAppenderatorDriverTest extends EasyMockSupport Assert.assertNull(published.getCommitMetadata()); } - @Test + @Test(timeout = 5000L) public void testIncrementalPush() throws Exception { Assert.assertNull(driver.startJob(null)); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorTest.java similarity index 86% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorTest.java index 912e8634c72..ba0f98e5932 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorTest.java @@ -22,6 +22,7 @@ package org.apache.druid.segment.realtime.appenderator; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; @@ -42,7 +43,7 @@ import java.util.Collections; import java.util.List; import java.util.stream.Collectors; -public class BatchAppenderatorTest extends InitializedNullHandlingTest +public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHandlingTest { private static final List IDENTIFIERS = ImmutableList.of( createSegmentId("2000/2001", "A", 0), // should be in seg_0 @@ -53,14 +54,14 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest @Test public void testSimpleIngestion() throws Exception { - try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { + try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); // startJob Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // add #1 Assert.assertEquals( @@ -82,7 +83,6 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest appenderator.getSegments().stream().sorted().collect(Collectors.toList()) ); - // add #3, this hits max rows in memory: Assert.assertEquals( 2, @@ -90,7 +90,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest .getNumRowsInSegment() ); - // since we just added three rows and the max rows in memory is three, all the segments (sinks etc) + // since we just added three rows and the max rows in memory is three, all the segments (sinks etc.) // above should be cleared now Assert.assertEquals( Collections.emptyList(), @@ -104,7 +104,6 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest .getNumRowsInSegment() ); - // push all final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push( appenderator.getSegments(), @@ -136,14 +135,14 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest @Test public void testPeriodGranularityNonUTCIngestion() throws Exception { - try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(1, true)) { + try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(1, true)) { final Appenderator appenderator = tester.getAppenderator(); // startJob Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // Create a segment identifier with a non-utc interval SegmentIdWithShardSpec segmentIdWithNonUTCTime = @@ -194,110 +193,26 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest } } - @Test - public void testSimpleIngestionWithFallbackCodePath() throws Exception - { - try (final BatchAppenderatorTester tester = new BatchAppenderatorTester( - 3, - -1, - null, - true, - new SimpleRowIngestionMeters(), - true, - true - )) { - final Appenderator appenderator = tester.getAppenderator(); - - // startJob - Assert.assertNull(appenderator.startJob()); - - // getDataSource - Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); - - // add #1 - Assert.assertEquals( - 1, - appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null) - .getNumRowsInSegment() - ); - - // add #2 - Assert.assertEquals( - 1, - appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 2), null) - .getNumRowsInSegment() - ); - - // getSegments - Assert.assertEquals( - IDENTIFIERS.subList(0, 2), - appenderator.getSegments().stream().sorted().collect(Collectors.toList()) - ); - - - // add #3, this hits max rows in memory: - Assert.assertEquals( - 2, - appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "sux", 1), null) - .getNumRowsInSegment() - ); - - // since we just added three rows and the max rows in memory is three BUT we are using - // the old, fallback, code path that does not remove sinks, the segments should still be there - Assert.assertEquals( - 2, - appenderator.getSegments().size() - ); - - // add #4, this will add one more temporary segment: - Assert.assertEquals( - 1, - appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "qux", 4), null) - .getNumRowsInSegment() - ); - - - // push all - final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push( - appenderator.getSegments(), - null, - false - ).get(); - Assert.assertEquals( - IDENTIFIERS.subList(0, 3), - Lists.transform( - segmentsAndCommitMetadata.getSegments(), - SegmentIdWithShardSpec::fromDataSegment - ).stream().sorted().collect(Collectors.toList()) - ); - Assert.assertEquals( - tester.getPushedSegments().stream().sorted().collect(Collectors.toList()), - segmentsAndCommitMetadata.getSegments().stream().sorted().collect(Collectors.toList()) - ); - - appenderator.close(); - Assert.assertTrue(appenderator.getSegments().isEmpty()); - } - } @Test public void testMaxBytesInMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final BatchAppenderatorTester tester = new BatchAppenderatorTester( + final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( 100, 1024, null, true, new SimpleRowIngestionMeters(), - true, - false + true ) ) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); - //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 54 (dimsKeySize) = 182 + 1 byte when null handling is enabled + //expectedSizeInBytes = + // 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 54 (dimsKeySize) = + // 182 + 1 byte when null handling is enabled int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; Assert.assertEquals( 182 + nullHandlingOverhead, @@ -317,14 +232,13 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final BatchAppenderatorTester tester = new BatchAppenderatorTester( + final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( 100, 1024, null, true, new SimpleRowIngestionMeters(), - true, - false + true ) ) { final Appenderator appenderator = tester.getAppenderator(); @@ -348,7 +262,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest @Test public void testMaxBytesInMemory() throws Exception { - try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(100, 15000, true)) { + try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(100, 15000, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -372,7 +286,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest for (int i = 0; i < 53; i++) { appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null); } - sinkSizeOverhead = BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + // currHydrant size is 0 since we just persist all indexes to disk. currentInMemoryIndexSize = 0; // We are now over maxSizeInBytes after the add. Hence, we do a persist. @@ -381,7 +295,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest currentInMemoryIndexSize, ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); - // no sinks no hydrants after a persist so we should have zero bytes currently in memory + // no sinks no hydrants after a persist, so we should have zero bytes currently in memory Assert.assertEquals( currentInMemoryIndexSize, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() @@ -404,6 +318,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest for (int i = 0; i < 53; i++) { appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null); } + // currHydrant size is 0 since we just persist all indexes to disk. currentInMemoryIndexSize = 0; // We are now over maxSizeInBytes after the add. Hence, we do a persist. @@ -416,17 +331,18 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest currentInMemoryIndexSize, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() ); + appenderator.persistAll(null).get(); appenderator.close(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory()); } } - @Test(expected = RuntimeException.class) + @Test(expected = RuntimeException.class, timeout = 5000L) public void testTaskFailAsPersistCannotFreeAnyMoreMemory() throws Exception { - try (final BatchAppenderatorTester tester = - new BatchAppenderatorTester(100, 5180, true)) { + try (final ClosedSegmensSinksBatchAppenderatorTester tester = + new ClosedSegmensSinksBatchAppenderatorTester(100, 5180, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); @@ -437,27 +353,27 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final BatchAppenderatorTester tester = new BatchAppenderatorTester( + final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( 100, 10, null, true, new SimpleRowIngestionMeters(), - true, - false + true ) ) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + // Expected 0 since we persisted after the add Assert.assertEquals( 0, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() ); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); - // Expected 0 since we persisted after the add + Assert.assertEquals( 0, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() @@ -468,8 +384,8 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest @Test public void testTaskCleanupInMemoryCounterAfterCloseWithRowInMemory() throws Exception { - try (final BatchAppenderatorTester tester = - new BatchAppenderatorTester(100, 10000, true)) { + try (final ClosedSegmensSinksBatchAppenderatorTester tester = + new ClosedSegmensSinksBatchAppenderatorTester(100, 10000, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -495,8 +411,8 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest @Test public void testMaxBytesInMemoryInMultipleSinks() throws Exception { - try (final BatchAppenderatorTester tester = - new BatchAppenderatorTester(1000, 28748, true)) { + try (final ClosedSegmensSinksBatchAppenderatorTester tester = + new ClosedSegmensSinksBatchAppenderatorTester(1000, 28748, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -529,6 +445,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null); appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar_" + i, 1), null); } + // sinks + currHydrant size is 0 since we just persist all indexes to disk. currentInMemoryIndexSize = 0; // We are now over maxSizeInBytes after the add. Hence, we do a persist. @@ -586,6 +503,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null); appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar_" + i, 1), null); } + // currHydrant size is 0 since we just persist all indexes to disk. currentInMemoryIndexSize = 0; // We are now over maxSizeInBytes after the add. Hence, we do a persist. @@ -614,8 +532,8 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest @Test public void testIgnoreMaxBytesInMemory() throws Exception { - try (final BatchAppenderatorTester tester = - new BatchAppenderatorTester(100, -1, true)) { + try (final ClosedSegmensSinksBatchAppenderatorTester tester = + new ClosedSegmensSinksBatchAppenderatorTester(100, -1, true)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -631,7 +549,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 1), null); - // we added two rows only and we told that maxSizeInBytes should be ignored, so it should not have been + // we added two rows only, and we told that maxSizeInBytes should be ignored, so it should not have been // persisted: int sinkSizeOverhead = 2 * BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; Assert.assertEquals( @@ -647,7 +565,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest @Test public void testMaxRowsInMemory() throws Exception { - try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { + try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -663,14 +581,15 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bat", 1), null); // persist expected ^ (3) rows added Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "baz", 1), null); Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 1), null); Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bob", 1), null); - Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); // persist expected ^ (3) rows added - //appenderator.persistAll(null); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.close(); } @@ -679,7 +598,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest @Test public void testAllHydrantsAreRecovered() throws Exception { - try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(1, false)) { + try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(1, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -690,7 +609,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo3", 1), null); // Since maxRowsInMemory is one there ought to be three hydrants stored and recovered - // just before push, internally the code has a sanity check to make sure that this works..if it does not it throws + // just before push, internally the code has a sanity check to make sure that this works. If it does not it throws // an exception final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push( appenderator.getSegments(), @@ -712,7 +631,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest @Test public void testTotalRowsPerSegment() throws Exception { - try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { + try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -772,22 +691,25 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest @Test public void testRestoreFromDisk() throws Exception { - final BatchAppenderatorTester tester = new BatchAppenderatorTester(2, true); + final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(2, true); final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 2), null); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "baz", 3), null); appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 4), null); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "bob", 5), null); Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.persistAll(null).get(); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); List segmentPaths = ((BatchAppenderator) appenderator).getPersistedidentifierPaths(); @@ -808,7 +730,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest @Test public void testCleanupFromDiskAfterClose() throws Exception { - final BatchAppenderatorTester tester = new BatchAppenderatorTester(2, true); + final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(2, true); final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -845,10 +767,10 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest } - @Test(timeout = 60_000L) + @Test(timeout = 5000L) public void testTotalRowCount() throws Exception { - try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { + try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, appenderator.getTotalRowCount()); @@ -889,10 +811,10 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest public void testVerifyRowIngestionMetrics() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final BatchAppenderatorTester tester = - new BatchAppenderatorTester(5, - 10000L, - null, false, rowIngestionMeters + try (final ClosedSegmensSinksBatchAppenderatorTester tester = + new ClosedSegmensSinksBatchAppenderatorTester(5, + 10000L, + null, false, rowIngestionMeters )) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -908,14 +830,14 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest } } - @Test + @Test(timeout = 10000L) public void testPushContract() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final BatchAppenderatorTester tester = - new BatchAppenderatorTester(1, - 50000L, - null, false, rowIngestionMeters + try (final ClosedSegmensSinksBatchAppenderatorTester tester = + new ClosedSegmensSinksBatchAppenderatorTester(1, + 50000L, + null, false, rowIngestionMeters )) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -957,11 +879,62 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest } } + @Test(timeout = 5000L) + public void testCloseContract() throws Exception + { + final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); + try (final ClosedSegmensSinksBatchAppenderatorTester tester = + new ClosedSegmensSinksBatchAppenderatorTester(1, + 50000L, + null, false, rowIngestionMeters + )) { + final Appenderator appenderator = tester.getAppenderator(); + appenderator.startJob(); + + // each one of these adds will trigger a persist since maxRowsInMemory is set to one above + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 1), null); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar2", 1), null); + + // push only a single segment + ListenableFuture firstFuture = appenderator.push( + Collections.singletonList(IDENTIFIERS.get(0)), + null, + false + ); + + // push remaining segments: + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar3", 1), null); + ListenableFuture secondFuture = appenderator.push( + Collections.singletonList(IDENTIFIERS.get(1)), + null, + false + ); + + // close should wait for all pushes and persists to end: + appenderator.close(); + + Assert.assertTrue(!firstFuture.isCancelled()); + Assert.assertTrue(!secondFuture.isCancelled()); + + Assert.assertTrue(firstFuture.isDone()); + Assert.assertTrue(secondFuture.isDone()); + + final SegmentsAndCommitMetadata segmentsAndCommitMetadataForFirstFuture = firstFuture.get(); + final SegmentsAndCommitMetadata segmentsAndCommitMetadataForSecondFuture = secondFuture.get(); + + // all segments must have been pushed: + Assert.assertEquals(segmentsAndCommitMetadataForFirstFuture.getSegments().size(), 1); + Assert.assertEquals(segmentsAndCommitMetadataForSecondFuture.getSegments().size(), 1); + + } + } + + private static SegmentIdWithShardSpec createNonUTCSegmentId(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( - BatchAppenderatorTester.DATASOURCE, + ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, new Interval(interval, ISOChronology.getInstance(DateTimes.inferTzFromString("Asia/Seoul"))), version, new LinearShardSpec(partitionNum) @@ -972,7 +945,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest private static SegmentIdWithShardSpec createSegmentId(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( - BatchAppenderatorTester.DATASOURCE, + ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, Intervals.of(interval), version, new LinearShardSpec(partitionNum) @@ -994,6 +967,5 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest ); } - } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index f02458b51fc..f2cdf9cba6c 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -168,13 +168,13 @@ public class DefaultOfflineAppenderatorFactoryTest "A", new LinearShardSpec(0) ); - Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "bar", 1), null); - Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "baz", 1), null); - Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.close(); - Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); } finally { appenderator.close(); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java new file mode 100644 index 00000000000..ed89ac76fe1 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java @@ -0,0 +1,293 @@ +/* + * 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.druid.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.core.NoopEmitter; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.incremental.ParseExceptionHandler; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CopyOnWriteArrayList; + +public class OpenAndClosedSegmentsAppenderatorTester implements AutoCloseable +{ + public static final String DATASOURCE = "foo"; + + private final DataSchema schema; + private final FireDepartmentMetrics metrics; + private final DataSegmentPusher dataSegmentPusher; + private final ObjectMapper objectMapper; + private final Appenderator appenderator; + private final IndexIO indexIO; + private final IndexMerger indexMerger; + private final ServiceEmitter emitter; + private final AppenderatorConfig tuningConfig; + + + private final List pushedSegments = new CopyOnWriteArrayList<>(); + + public OpenAndClosedSegmentsAppenderatorTester( + final int maxRowsInMemory, + final boolean enablePushFailure, + boolean batchMemoryMappedIndex + ) + { + this(maxRowsInMemory, -1, null, enablePushFailure, batchMemoryMappedIndex); + } + + public OpenAndClosedSegmentsAppenderatorTester( + final int maxRowsInMemory, + final long maxSizeInBytes, + final File basePersistDirectory, + final boolean enablePushFailure, + boolean batchMemoryMappedIndex + ) + { + this( + maxRowsInMemory, + maxSizeInBytes, + basePersistDirectory, + enablePushFailure, + new SimpleRowIngestionMeters(), + false, + batchMemoryMappedIndex + ); + } + + public OpenAndClosedSegmentsAppenderatorTester( + final int maxRowsInMemory, + final long maxSizeInBytes, + final File basePersistDirectory, + final boolean enablePushFailure, + final RowIngestionMeters rowIngestionMeters, + final boolean skipBytesInMemoryOverheadCheck, + boolean batchMemoryMappedIndex + ) + { + objectMapper = new DefaultObjectMapper(); + objectMapper.registerSubtypes(LinearShardSpec.class); + + final Map parserMap = objectMapper.convertValue( + new MapInputRowParser( + new JSONParseSpec( + new TimestampSpec("ts", "auto", null), + new DimensionsSpec(null, null, null), + null, + null, + null + ) + ), + Map.class + ); + schema = new DataSchema( + DATASOURCE, + parserMap, + new AggregatorFactory[]{ + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("met", "met") + }, + new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), + null, + objectMapper + ); + tuningConfig = + new ClosedSegmensSinksBatchAppenderatorTester.TestIndexTuningConfig( + TuningConfig.DEFAULT_APPENDABLE_INDEX, + maxRowsInMemory, + maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, + skipBytesInMemoryOverheadCheck, + new IndexSpec(), + 0, + false, + 0L, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, + basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory + ); + + metrics = new FireDepartmentMetrics(); + + indexIO = new IndexIO( + objectMapper, + new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 0; + } + } + ); + indexMerger = new IndexMergerV9(objectMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); + + emitter = new ServiceEmitter( + "test", + "test", + new NoopEmitter() + ); + emitter.start(); + EmittingLogger.registerEmitter(emitter); + dataSegmentPusher = new DataSegmentPusher() + { + private boolean mustFail = true; + + @Deprecated + @Override + public String getPathForHadoop(String dataSource) + { + return getPathForHadoop(); + } + + @Override + public String getPathForHadoop() + { + throw new UnsupportedOperationException(); + } + + @Override + public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException + { + if (enablePushFailure && mustFail) { + mustFail = false; + throw new IOException("Push failure test"); + } else if (enablePushFailure) { + mustFail = true; + } + pushedSegments.add(segment); + return segment; + } + + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException(); + } + }; + if (batchMemoryMappedIndex) { + appenderator = Appenderators.createOpenSegmentsOffline( + schema.getDataSource(), + schema, + tuningConfig, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0)); + } else { + appenderator = Appenderators.createClosedSegmentsOffline( + schema.getDataSource(), + schema, + tuningConfig, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0)); + } + } + + private long getDefaultMaxBytesInMemory() + { + return (Runtime.getRuntime().totalMemory()) / 3; + } + + public DataSchema getSchema() + { + return schema; + } + + public AppenderatorConfig getTuningConfig() + { + return tuningConfig; + } + + public FireDepartmentMetrics getMetrics() + { + return metrics; + } + + public DataSegmentPusher getDataSegmentPusher() + { + return dataSegmentPusher; + } + + public ObjectMapper getObjectMapper() + { + return objectMapper; + } + + public Appenderator getAppenderator() + { + return appenderator; + } + + public List getPushedSegments() + { + return pushedSegments; + } + + @Override + public void close() throws Exception + { + appenderator.close(); + emitter.close(); + FileUtils.deleteDirectory(tuningConfig.getBasePersistDirectory()); + } + + private static File createNewBasePersistDirectory() + { + return FileUtils.createTempDir("druid-batch-persist"); + } +} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java new file mode 100644 index 00000000000..67dcae99e92 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java @@ -0,0 +1,205 @@ +/* + * 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.druid.segment.realtime.appenderator; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.segment.loading.DataSegmentKiller; +import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver.SegmentsForSequence; +import org.apache.druid.segment.realtime.appenderator.SegmentWithState.SegmentState; +import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriverTest.TestSegmentAllocator; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.easymock.EasyMock; +import org.easymock.EasyMockSupport; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class OpenAndClosedSegmentsBatchAppenderatorDriverTest extends EasyMockSupport +{ + private static final String DATA_SOURCE = "foo"; + private static final String VERSION = "abc123"; + private static final int MAX_ROWS_IN_MEMORY = 100; + private static final long TIMEOUT = 1000; + + private static final List ROWS = Arrays.asList( + new MapBasedInputRow( + DateTimes.of("2000"), + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", "foo", "met1", "1") + ), + new MapBasedInputRow( + DateTimes.of("2000T01"), + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", "foo", "met1", 2.0) + ), + new MapBasedInputRow( + DateTimes.of("2000T01"), + ImmutableList.of("dim2"), + ImmutableMap.of("dim2", "bar", "met1", 2.0) + ) + ); + + private SegmentAllocator allocator; + private OpenAndClosedSegmentsAppenderatorTester openAndClosedSegmentsAppenderatorTester; + private BatchAppenderatorDriver driver; + private DataSegmentKiller dataSegmentKiller; + + static { + NullHandling.initializeForTests(); + } + + @Before + public void setup() + { + openAndClosedSegmentsAppenderatorTester = + new OpenAndClosedSegmentsAppenderatorTester(MAX_ROWS_IN_MEMORY, false, + false + ); + allocator = new TestSegmentAllocator(DATA_SOURCE, Granularities.HOUR); + dataSegmentKiller = createStrictMock(DataSegmentKiller.class); + driver = new BatchAppenderatorDriver( + openAndClosedSegmentsAppenderatorTester.getAppenderator(), + allocator, + new TestUsedSegmentChecker(openAndClosedSegmentsAppenderatorTester.getPushedSegments()), + dataSegmentKiller + ); + + EasyMock.replay(dataSegmentKiller); + } + + @After + public void tearDown() throws Exception + { + EasyMock.verify(dataSegmentKiller); + + driver.clear(); + driver.close(); + } + + @Test (timeout = 2000L) + public void testSimple() throws Exception + { + Assert.assertNull(driver.startJob(null)); + + for (InputRow row : ROWS) { + Assert.assertTrue(driver.add(row, "dummy").isOk()); + } + + checkSegmentStates(2, SegmentState.APPENDING); + + driver.pushAllAndClear(TIMEOUT); + + checkSegmentStates(2, SegmentState.PUSHED_AND_DROPPED); + + final SegmentsAndCommitMetadata published = + driver.publishAll(null, null, makeOkPublisher(), Function.identity()).get(TIMEOUT, TimeUnit.MILLISECONDS); + + Assert.assertEquals( + ImmutableSet.of( + new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000/PT1H"), VERSION, new NumberedShardSpec(0, 0)), + new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000T01/PT1H"), VERSION, new NumberedShardSpec(0, 0)) + ), + published.getSegments() + .stream() + .map(SegmentIdWithShardSpec::fromDataSegment) + .collect(Collectors.toSet()) + ); + + Assert.assertNull(published.getCommitMetadata()); + } + + @Test(timeout = 5000L) + public void testIncrementalPush() throws Exception + { + Assert.assertNull(driver.startJob(null)); + + int i = 0; + for (InputRow row : ROWS) { + Assert.assertTrue(driver.add(row, "dummy").isOk()); + + checkSegmentStates(1, SegmentState.APPENDING); + checkSegmentStates(i, SegmentState.PUSHED_AND_DROPPED); + + driver.pushAllAndClear(TIMEOUT); + checkSegmentStates(0, SegmentState.APPENDING); + checkSegmentStates(++i, SegmentState.PUSHED_AND_DROPPED); + } + + final SegmentsAndCommitMetadata published = + driver.publishAll(null, null, makeOkPublisher(), Function.identity()).get(TIMEOUT, TimeUnit.MILLISECONDS); + + Assert.assertEquals( + ImmutableSet.of( + new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000/PT1H"), VERSION, new NumberedShardSpec(0, 0)), + new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000T01/PT1H"), VERSION, new NumberedShardSpec(0, 0)), + new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000T01/PT1H"), VERSION, new NumberedShardSpec(1, 0)) + ), + published.getSegments() + .stream() + .map(SegmentIdWithShardSpec::fromDataSegment) + .collect(Collectors.toSet()) + ); + + Assert.assertNull(published.getCommitMetadata()); + } + + @Test + public void testRestart() + { + Assert.assertNull(driver.startJob(null)); + driver.close(); + openAndClosedSegmentsAppenderatorTester.getAppenderator().close(); + + Assert.assertNull(driver.startJob(null)); + } + + private void checkSegmentStates(int expectedNumSegmentsInState, SegmentState expectedState) + { + final SegmentsForSequence segmentsForSequence = driver.getSegments().get("dummy"); + Assert.assertNotNull(segmentsForSequence); + final List segmentWithStates = segmentsForSequence + .allSegmentStateStream() + .filter(segmentWithState -> segmentWithState.getState() == expectedState) + .collect(Collectors.toList()); + + Assert.assertEquals(expectedNumSegmentsInState, segmentWithStates.size()); + } + + static TransactionalSegmentPublisher makeOkPublisher() + { + return (segmentsToBeOverwritten, segmentsToBeDropped, segmentsToPublish, commitMetadata) -> SegmentPublishResult.ok(ImmutableSet.of()); + } +} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorTest.java new file mode 100644 index 00000000000..04dc528220f --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorTest.java @@ -0,0 +1,229 @@ +/* + * 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.druid.segment.realtime.appenderator; + +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; +import java.util.stream.Collectors; + +public class OpenAndClosedSegmentsBatchAppenderatorTest extends InitializedNullHandlingTest +{ + private static final List IDENTIFIERS = ImmutableList.of( + createSegmentId("2000/2001", "A", 0), + createSegmentId("2000/2001", "A", 1), + createSegmentId("2001/2002", "A", 0) + ); + + @Test + public void testSimpleIngestionWithIndexesNotMapped() throws Exception + { + try (final OpenAndClosedSegmentsAppenderatorTester tester = + new OpenAndClosedSegmentsAppenderatorTester(2, + false, + false)) { + final Appenderator appenderator = tester.getAppenderator(); + boolean thrown; + + // startJob + Assert.assertEquals(null, appenderator.startJob()); + + // getDataSource + Assert.assertEquals(OpenAndClosedSegmentsAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + + // add + Assert.assertEquals( + 1, + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null) + .getNumRowsInSegment() + ); + + Assert.assertEquals( + 2, + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 2), null) + .getNumRowsInSegment() + ); + + Assert.assertEquals( + 1, + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 4), null) + .getNumRowsInSegment() + ); + + // getSegments + Assert.assertEquals(IDENTIFIERS.subList(0, 2), + appenderator.getSegments().stream().sorted().collect(Collectors.toList())); + + // getRowCount + Assert.assertEquals(2, appenderator.getRowCount(IDENTIFIERS.get(0))); + Assert.assertEquals(1, appenderator.getRowCount(IDENTIFIERS.get(1))); + thrown = false; + try { + appenderator.getRowCount(IDENTIFIERS.get(2)); + } + catch (IllegalStateException e) { + thrown = true; + } + Assert.assertTrue(thrown); + + // push all + final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push( + appenderator.getSegments(), + null, + false + ).get(); + Assert.assertEquals( + IDENTIFIERS.subList(0, 2), + Lists.transform( + segmentsAndCommitMetadata.getSegments(), + new Function() + { + @Override + public SegmentIdWithShardSpec apply(DataSegment input) + { + return SegmentIdWithShardSpec.fromDataSegment(input); + } + } + ).stream().sorted().collect(Collectors.toList()) + ); + Assert.assertEquals(tester.getPushedSegments().stream().sorted().collect(Collectors.toList()), + segmentsAndCommitMetadata.getSegments().stream().sorted().collect(Collectors.toList())); + + appenderator.clear(); + Assert.assertTrue(appenderator.getSegments().isEmpty()); + } + } + + @Test + public void testSimpleIngestionWithIndexesMapped() throws Exception + { + try (final OpenAndClosedSegmentsAppenderatorTester tester = new OpenAndClosedSegmentsAppenderatorTester(2, + false, + true)) { + final Appenderator appenderator = tester.getAppenderator(); + boolean thrown; + + // startJob + Assert.assertEquals(null, appenderator.startJob()); + + // getDataSource + Assert.assertEquals(OpenAndClosedSegmentsAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + + // add + Assert.assertEquals( + 1, + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null) + .getNumRowsInSegment() + ); + + Assert.assertEquals( + 2, + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 2), null) + .getNumRowsInSegment() + ); + + Assert.assertEquals( + 1, + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 4), null) + .getNumRowsInSegment() + ); + + // getSegments + Assert.assertEquals(IDENTIFIERS.subList(0, 2), + appenderator.getSegments().stream().sorted().collect(Collectors.toList())); + + // getRowCount + Assert.assertEquals(2, appenderator.getRowCount(IDENTIFIERS.get(0))); + Assert.assertEquals(1, appenderator.getRowCount(IDENTIFIERS.get(1))); + thrown = false; + try { + appenderator.getRowCount(IDENTIFIERS.get(2)); + } + catch (IllegalStateException e) { + thrown = true; + } + Assert.assertTrue(thrown); + + // push all + final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push( + appenderator.getSegments(), + null, + false + ).get(); + Assert.assertEquals( + IDENTIFIERS.subList(0, 2), + Lists.transform( + segmentsAndCommitMetadata.getSegments(), + new Function() + { + @Override + public SegmentIdWithShardSpec apply(DataSegment input) + { + return SegmentIdWithShardSpec.fromDataSegment(input); + } + } + ).stream().sorted().collect(Collectors.toList()) + ); + Assert.assertEquals(tester.getPushedSegments().stream().sorted().collect(Collectors.toList()), + segmentsAndCommitMetadata.getSegments().stream().sorted().collect(Collectors.toList())); + + appenderator.clear(); + Assert.assertTrue(appenderator.getSegments().isEmpty()); + } + } + private static SegmentIdWithShardSpec createSegmentId(String interval, String version, int partitionNum) + { + return new SegmentIdWithShardSpec( + OpenAndClosedSegmentsAppenderatorTester.DATASOURCE, + Intervals.of(interval), + version, + new LinearShardSpec(partitionNum) + + ); + } + + static InputRow createInputRow(String ts, String dim, Object met) + { + return new MapBasedInputRow( + DateTimes.of(ts).getMillis(), + ImmutableList.of("dim"), + ImmutableMap.of( + "dim", + dim, + "met", + met + ) + ); + } + +} + diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java index daa14cae356..ade737f8653 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java @@ -44,6 +44,7 @@ import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactor import org.apache.druid.server.metrics.NoopServiceEmitter; import org.easymock.EasyMock; import org.junit.Assert; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -67,30 +68,42 @@ public class UnifiedIndexerAppenderatorsManagerTest () -> new DefaultQueryRunnerFactoryConglomerate(ImmutableMap.of()) ); - private final Appenderator appenderator = manager.createOfflineAppenderatorForTask( - "taskId", - new DataSchema( - "myDataSource", - new TimestampSpec("__time", "millis", null), - null, - null, - new UniformGranularitySpec(Granularities.HOUR, Granularities.HOUR, false, Collections.emptyList()), - null - ), - EasyMock.createMock(AppenderatorConfig.class), - new FireDepartmentMetrics(), - new NoopDataSegmentPusher(), - TestHelper.makeJsonMapper(), - TestHelper.getTestIndexIO(), - TestHelper.getTestIndexMergerV9(OnHeapMemorySegmentWriteOutMediumFactory.instance()), - new NoopRowIngestionMeters(), - new ParseExceptionHandler(new NoopRowIngestionMeters(), false, 0, 0), - false - ); + private AppenderatorConfig appenderatorConfig; + private Appenderator appenderator; + + @Before + public void setup() + { + appenderatorConfig = EasyMock.createMock(AppenderatorConfig.class); + EasyMock.expect(appenderatorConfig.getMaxPendingPersists()).andReturn(0); + EasyMock.expect(appenderatorConfig.isSkipBytesInMemoryOverheadCheck()).andReturn(false); + EasyMock.replay(appenderatorConfig); + appenderator = manager.createClosedSegmentsOfflineAppenderatorForTask( + "taskId", + new DataSchema( + "myDataSource", + new TimestampSpec("__time", "millis", null), + null, + null, + new UniformGranularitySpec(Granularities.HOUR, Granularities.HOUR, false, Collections.emptyList()), + null + ), + appenderatorConfig, + new FireDepartmentMetrics(), + new NoopDataSegmentPusher(), + TestHelper.makeJsonMapper(), + TestHelper.getTestIndexIO(), + TestHelper.getTestIndexMergerV9(OnHeapMemorySegmentWriteOutMediumFactory.instance()), + new NoopRowIngestionMeters(), + new ParseExceptionHandler(new NoopRowIngestionMeters(), false, 0, 0) + ); + } @Test public void test_getBundle_knownDataSource() { + + final UnifiedIndexerAppenderatorsManager.DatasourceBundle bundle = manager.getBundle( Druids.newScanQueryBuilder() .dataSource(appenderator.getDataSource())