From 02b8738c00ccdbc22a37914c371b8baf076e5b10 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 22 Jul 2024 13:56:44 -0700 Subject: [PATCH] remove batchProcessingMode from task config, remove AppenderatorImpl (#16765) changes: * removes `druid.indexer.task.batchProcessingMode` in favor of always using `CLOSED_SEGMENT_SINKS` which uses `BatchAppenderator`. This was intended to become the default for native batch, but that was missed so `CLOSED_SEGMENTS` was the default (using `AppenderatorImpl`), however MSQ has been exclusively using `BatchAppenderator` with no problems so it seems safe to just roll it out as the only option for batch ingestion everywhere. * with `batchProcessingMode` gone, there is no use for `AppenderatorImpl` so it has been removed * implify `Appenderator` construction since there are only separate stream and batch versions now * simplify tests since `batchProcessingMode` is gone --- docs/configuration/index.md | 1 - ...SegmentGeneratorFrameProcessorFactory.java | 3 +- .../indexing/common/config/TaskConfig.java | 62 - .../common/task/BatchAppenderators.java | 64 +- .../indexing/common/TaskToolboxTest.java | 1 - .../common/config/TaskConfigBuilder.java | 16 - .../common/task/BatchAppenderatorsTest.java | 604 ------ .../common/task/CompactionTaskRunTest.java | 1 - .../common/task/CompactionTaskTest.java | 1 - .../indexing/common/task/HadoopTaskTest.java | 1 - .../common/task/IngestionTestBase.java | 1 - .../common/task/TestAppenderatorsManager.java | 68 +- ...stractParallelIndexSupervisorTaskTest.java | 3 - .../overlord/ForkingTaskRunnerTest.java | 3 +- .../SingleTaskBackgroundRunnerTest.java | 1 - .../indexing/overlord/TaskLifecycleTest.java | 1 - .../SeekableStreamIndexTaskTestBase.java | 1 - .../worker/WorkerTaskManagerTest.java | 1 - .../worker/WorkerTaskMonitorTest.java | 1 - ...ntermediaryDataManagerAutoCleanupTest.java | 1 - ...iaryDataManagerManualAddAndDeleteTest.java | 1 - .../shuffle/ShuffleDataSegmentPusherTest.java | 1 - .../worker/shuffle/ShuffleResourceTest.java | 1 - .../BuildingHashBasedNumberedShardSpec.java | 4 +- .../partition/BuildingNumberedShardSpec.java | 4 +- .../appenderator/AppenderatorImpl.java | 1638 ----------------- .../realtime/appenderator/Appenderators.java | 76 +- .../appenderator/AppenderatorsManager.java | 41 +- ...DummyForInjectionAppenderatorsManager.java | 40 +- .../PeonAppenderatorsManager.java | 79 +- .../UnifiedIndexerAppenderatorsManager.java | 88 +- .../appenderator/AppenderatorsTest.java | 245 --- ....java => BatchAppenderatorDriverTest.java} | 6 +- ...orTest.java => BatchAppenderatorTest.java} | 78 +- ...ster.java => BatchAppenderatorTester.java} | 16 +- ...enAndClosedSegmentsAppenderatorTester.java | 289 --- ...edSegmentsBatchAppenderatorDriverTest.java | 205 --- ...ndClosedSegmentsBatchAppenderatorTest.java | 228 --- ...nifiedIndexerAppenderatorsManagerTest.java | 2 +- 39 files changed, 87 insertions(+), 3790 deletions(-) delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java delete mode 100644 server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java rename server/src/test/java/org/apache/druid/segment/realtime/appenderator/{ClosedSegmentsSinksBatchAppenderatorDriverTest.java => BatchAppenderatorDriverTest.java} (97%) rename server/src/test/java/org/apache/druid/segment/realtime/appenderator/{ClosedSegmentsSinksBatchAppenderatorTest.java => BatchAppenderatorTest.java} (91%) rename server/src/test/java/org/apache/druid/segment/realtime/appenderator/{ClosedSegmensSinksBatchAppenderatorTester.java => BatchAppenderatorTester.java} (95%) delete mode 100644 server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java delete mode 100644 server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java delete 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 39c14b77c46..3b3c2711d3b 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1500,7 +1500,6 @@ Additional Peon configs include: |`druid.peon.mode`|One of `local` or `remote`. Setting this property to `local` means you intend to run the Peon as a standalone process which is 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.batchProcessingMode`| Batch ingestion tasks have three operating modes to control construction and tracking for intermediary segments: `OPEN_SEGMENTS`, `CLOSED_SEGMENTS`, and `CLOSED_SEGMENT_SINKS`. `OPEN_SEGMENTS` uses the streaming ingestion code path and performs a `mmap` on intermediary segments to build a timeline to make these segments available to realtime queries. Batch ingestion doesn't require intermediary segments, so the default mode, `CLOSED_SEGMENTS`, eliminates `mmap` of intermediary segments. `CLOSED_SEGMENTS` mode still tracks the entire set of segments in heap. The `CLOSED_SEGMENTS_SINKS` mode is the most aggressive configuration and should have the smallest memory footprint. It eliminates in-memory tracking and `mmap` of intermediary segments produced during segment creation. `CLOSED_SEGMENTS_SINKS` mode isn't as well tested as other modes so is currently considered experimental. You can use `OPEN_SEGMENTS` mode if 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-api:3.3.6`, `org.apache.hadoop:hadoop-client-runtime:3.3.6`| |`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/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java index 3817e63ca49..16f9deff63d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java @@ -177,9 +177,8 @@ public class SegmentGeneratorFrameProcessorFactory // Create directly, without using AppenderatorsManager, because we need different memory overrides due to // using one Appenderator per processing thread instead of per task. - // Note: "createOffline" ignores the batchProcessingMode and always acts like CLOSED_SEGMENTS_SINKS. final Appenderator appenderator = - Appenderators.createOffline( + Appenderators.createBatch( idString, dataSchema, makeAppenderatorConfig( 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 db48d6f07f7..b8025e415a8 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 @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.EnumUtils; import org.apache.druid.common.config.Configs; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.java.util.common.ISE; @@ -65,16 +64,6 @@ public class TaskConfig } } - // 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"); private static final boolean DEFAULT_STORE_EMPTY_COLUMNS = true; @@ -110,12 +99,6 @@ public class TaskConfig @JsonProperty private final boolean ignoreTimestampSpecForDruidInputSource; - @JsonProperty - private final boolean batchMemoryMappedIndex; - - @JsonProperty - private final BatchProcessingMode batchProcessingMode; - @JsonProperty private final boolean storeEmptyColumns; @@ -137,9 +120,6 @@ public class TaskConfig @JsonProperty("directoryLockTimeout") Period directoryLockTimeout, @JsonProperty("shuffleDataLocations") List shuffleDataLocations, @JsonProperty("ignoreTimestampSpecForDruidInputSource") boolean ignoreTimestampSpecForDruidInputSource, - @JsonProperty("batchMemoryMappedIndex") boolean batchMemoryMappedIndex, - // deprecated, only set to true to fall back to older behavior - @JsonProperty("batchProcessingMode") String batchProcessingMode, @JsonProperty("storeEmptyColumns") @Nullable Boolean storeEmptyColumns, @JsonProperty("encapsulatedTask") boolean enableTaskLevelLogPush, @JsonProperty("tmpStorageBytesPerTask") @Nullable Long tmpStorageBytesPerTask @@ -171,26 +151,8 @@ public class TaskConfig ); this.ignoreTimestampSpecForDruidInputSource = ignoreTimestampSpecForDruidInputSource; - this.batchMemoryMappedIndex = batchMemoryMappedIndex; this.encapsulatedTask = enableTaskLevelLogPush; - // 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.debug("Batch processing mode:[%s]", this.batchProcessingMode); - this.storeEmptyColumns = Configs.valueOrDefault(storeEmptyColumns, DEFAULT_STORE_EMPTY_COLUMNS); this.tmpStorageBytesPerTask = Configs.valueOrDefault(tmpStorageBytesPerTask, DEFAULT_TMP_STORAGE_BYTES_PER_TASK); } @@ -206,8 +168,6 @@ public class TaskConfig Period directoryLockTimeout, List shuffleDataLocations, boolean ignoreTimestampSpecForDruidInputSource, - boolean batchMemoryMappedIndex, - BatchProcessingMode batchProcessingMode, boolean storeEmptyColumns, boolean encapsulatedTask, long tmpStorageBytesPerTask @@ -223,8 +183,6 @@ public class TaskConfig this.directoryLockTimeout = directoryLockTimeout; this.shuffleDataLocations = shuffleDataLocations; this.ignoreTimestampSpecForDruidInputSource = ignoreTimestampSpecForDruidInputSource; - this.batchMemoryMappedIndex = batchMemoryMappedIndex; - this.batchProcessingMode = batchProcessingMode; this.storeEmptyColumns = storeEmptyColumns; this.encapsulatedTask = encapsulatedTask; this.tmpStorageBytesPerTask = tmpStorageBytesPerTask; @@ -310,22 +268,6 @@ public class TaskConfig return ignoreTimestampSpecForDruidInputSource; } - @JsonProperty - public BatchProcessingMode getBatchProcessingMode() - { - return batchProcessingMode; - } - - /** - * Do not use in code! use {@link TaskConfig#getBatchProcessingMode() instead} - */ - @Deprecated - @JsonProperty - public boolean getbatchMemoryMappedIndex() - { - return batchMemoryMappedIndex; - } - @JsonProperty public boolean isStoreEmptyColumns() { @@ -366,8 +308,6 @@ public class TaskConfig directoryLockTimeout, shuffleDataLocations, ignoreTimestampSpecForDruidInputSource, - batchMemoryMappedIndex, - batchProcessingMode, storeEmptyColumns, encapsulatedTask, tmpStorageBytesPerTask @@ -387,8 +327,6 @@ public class TaskConfig directoryLockTimeout, shuffleDataLocations, ignoreTimestampSpecForDruidInputSource, - batchMemoryMappedIndex, - batchProcessingMode, storeEmptyColumns, encapsulatedTask, tmpStorageBytesPerTask 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 087464b48ac..6af4402c35a 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,8 +21,6 @@ package org.apache.druid.indexing.common.task; import org.apache.druid.indexing.appenderator.ActionBasedPublishedSegmentRetriever; 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; @@ -75,54 +73,20 @@ public final class BatchAppenderators boolean useMaxMemoryEstimates ) { - 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, - useMaxMemoryEstimates, - toolbox.getCentralizedTableSchemaConfig() - ); - } 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, - useMaxMemoryEstimates, - toolbox.getCentralizedTableSchemaConfig() - ); - } 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, - useMaxMemoryEstimates, - toolbox.getCentralizedTableSchemaConfig() - ); - } else { - throw new IAE("Invalid batchProcesingMode[%s]", toolbox.getConfig().getBatchProcessingMode()); - } + return appenderatorsManager.createBatchAppenderatorForTask( + taskId, + dataSchema, + appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()), + metrics, + segmentPusher, + toolbox.getJsonMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9(), + rowIngestionMeters, + parseExceptionHandler, + useMaxMemoryEstimates, + toolbox.getCentralizedTableSchemaConfig() + ); } public static BatchAppenderatorDriver newDriver( 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 a4f297c8e33..78cbb88c3cf 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 @@ -114,7 +114,6 @@ public class TaskToolboxTest TaskConfig taskConfig = new TaskConfigBuilder() .setBaseDir(temporaryFolder.newFile().toString()) .setDefaultRowFlushBoundary(50000) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); taskToolbox = new TaskToolboxFactory( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java index af920ebbeb7..1213b552514 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java @@ -36,8 +36,6 @@ public class TaskConfigBuilder private Period directoryLockTimeout; private List shuffleDataLocations; private boolean ignoreTimestampSpecForDruidInputSource; - private boolean batchMemoryMappedIndex; // deprecated; only set to true to fall back to older behavior - private String batchProcessingMode; private Boolean storeEmptyColumns; private boolean enableTaskLevelLogPush; private Long tmpStorageBytesPerTask; @@ -102,18 +100,6 @@ public class TaskConfigBuilder return this; } - public TaskConfigBuilder setBatchMemoryMappedIndex(boolean batchMemoryMappedIndex) - { - this.batchMemoryMappedIndex = batchMemoryMappedIndex; - return this; - } - - public TaskConfigBuilder setBatchProcessingMode(String batchProcessingMode) - { - this.batchProcessingMode = batchProcessingMode; - return this; - } - public TaskConfigBuilder setStoreEmptyColumns(Boolean storeEmptyColumns) { this.storeEmptyColumns = storeEmptyColumns; @@ -145,8 +131,6 @@ public class TaskConfigBuilder directoryLockTimeout, shuffleDataLocations, ignoreTimestampSpecForDruidInputSource, - batchMemoryMappedIndex, - batchProcessingMode, storeEmptyColumns, enableTaskLevelLogPush, tmpStorageBytesPerTask 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 deleted file mode 100644 index 278a27f24bb..00000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java +++ /dev/null @@ -1,604 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.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.indexing.common.config.TaskConfigBuilder; -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.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.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.SegmentGenerationMetrics; -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.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 SegmentGenerationMetrics 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), - DimensionsSpec.EMPTY, - 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, - IndexSpec.DEFAULT, - 0, - false, - 0L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, - basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory, - null - ); - metrics = new SegmentGenerationMetrics(); - - IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); - 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, - makeTaskToolbox( - objectMapper, - indexMerger, - TaskConfig.BatchProcessingMode.OPEN_SEGMENTS - ), - schema, - tuningConfig, - dataSegmentPusher, - rowIngestionMeters, - new ParseExceptionHandler( - rowIngestionMeters, - false, - Integer.MAX_VALUE, - 0 - ), - false - ); - break; - case "CLOSED_SEGMENTS": - appenderator = BatchAppenderators.newAppenderator( - "foo", - new TestAppenderatorsManager(), - metrics, - makeTaskToolbox( - objectMapper, - indexMerger, - TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS - ), - schema, - tuningConfig, - dataSegmentPusher, - rowIngestionMeters, - new ParseExceptionHandler( - rowIngestionMeters, - false, - Integer.MAX_VALUE, - 0 - ), - false - ); - - break; - case "CLOSED_SEGMENTS_SINKS": - appenderator = BatchAppenderators.newAppenderator( - "foo", - new TestAppenderatorsManager(), - metrics, - makeTaskToolbox( - objectMapper, - indexMerger, - TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS_SINKS - ), - schema, - tuningConfig, - dataSegmentPusher, - rowIngestionMeters, - new ParseExceptionHandler( - rowIngestionMeters, - false, - Integer.MAX_VALUE, - 0 - ), - false - ); - 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 SegmentGenerationMetrics 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; - private final int numPersistThreads; - - 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, - Integer numPersistThreads - ) - { - 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; - - this.numPersistThreads = numPersistThreads == null ? DEFAULT_NUM_PERSIST_THREADS : numPersistThreads; - } - - @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 int getNumPersistThreads() - { - return numPersistThreads; - } - - @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 && - numPersistThreads == that.numPersistThreads && - 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, - numPersistThreads - ); - } - - @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 + - ", numPersistThreads=" + numPersistThreads + - '}'; - } - } - - private static TaskToolbox makeTaskToolbox( - ObjectMapper mapper, - IndexMergerV9 indexMergerV9, - TaskConfig.BatchProcessingMode mode - ) - { - TaskConfig config = new TaskConfigBuilder() - .setBatchProcessingMode(mode.name()) - .build(); - return new TaskToolbox.Builder() - .config(config) - .joinableFactory(NoopJoinableFactory.INSTANCE) - .jsonMapper(mapper) - .indexIO(new IndexIO(new ObjectMapper(), ColumnConfig.DEFAULT)) - .indexMergerV9(indexMergerV9) - .taskReportFileWriter(new NoopTestTaskReportFileWriter()) - .authorizerMapper(AuthTestUtils.TEST_AUTHORIZER_MAPPER) - .chatHandlerProvider(new NoopChatHandlerProvider()) - .appenderatorsManager(new TestAppenderatorsManager()) - .taskLogPusher(null) - .attemptId("1") - .centralizedTableSchemaConfig(CentralizedDatasourceSchemaConfig.create()) - .build(); - - } - } -} - 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 c00a926442c..7ebde700bee 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 @@ -2030,7 +2030,6 @@ public class CompactionTaskRunTest extends IngestionTestBase ); final TaskConfig config = new TaskConfigBuilder() - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = CentralizedDatasourceSchemaConfig.create(); centralizedDatasourceSchemaConfig.setEnabled(true); 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 562347a7f97..f9849b1483d 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 @@ -1769,7 +1769,6 @@ public class CompactionTaskTest }; final TaskConfig config = new TaskConfigBuilder() - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); return new TaskToolbox.Builder() .config(config) 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 ceca135c810..130258e85d4 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 @@ -111,7 +111,6 @@ public class HadoopTaskTest new TaskConfigBuilder() .setBaseDir(temporaryFolder.newFolder().toString()) .setDefaultHadoopCoordinates(ImmutableList.of("something:hadoop:1")) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build() ).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 f3b2729b887..67a0c518f57 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 @@ -443,7 +443,6 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest ); final TaskConfig config = new TaskConfigBuilder() - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = new CentralizedDatasourceSchemaConfig(); centralizedDatasourceSchemaConfig.setEnabled(true); 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 e6186329126..417195326fb 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 @@ -101,7 +101,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager } @Override - public Appenderator createOpenSegmentsOfflineAppenderatorForTask( + public Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, @@ -116,71 +116,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig ) { - return Appenderators.createOpenSegmentsOffline( - taskId, - schema, - config, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - parseExceptionHandler, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - } - - @Override - public Appenderator createClosedSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - return Appenderators.createClosedSegmentsOffline( - taskId, - schema, - config, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - parseExceptionHandler, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - } - - @Override - public Appenderator createOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - return Appenderators.createOffline( + return Appenderators.createBatch( taskId, schema, config, 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 19feeaf44c4..1cd6a29e10d 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 @@ -225,7 +225,6 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase indexingServiceClient = new LocalOverlordClient(objectMapper, taskRunner); final TaskConfig taskConfig = new TaskConfigBuilder() .setShuffleDataLocations(ImmutableList.of(new StorageLocationConfig(temporaryFolder.newFolder(), null, null))) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); intermediaryDataManager = new LocalIntermediaryDataManager(new WorkerConfig(), taskConfig, null); remoteApiExecutor = Execs.singleThreaded("coordinator-api-executor"); @@ -616,7 +615,6 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase public void prepareObjectMapper(ObjectMapper objectMapper, IndexIO indexIO) { final TaskConfig taskConfig = new TaskConfigBuilder() - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); objectMapper.setInjectableValues( @@ -652,7 +650,6 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase protected TaskToolbox createTaskToolbox(Task task, TaskActionClient actionClient) throws IOException { TaskConfig config = new TaskConfigBuilder() - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = CentralizedDatasourceSchemaConfig.create(true); return new TaskToolbox.Builder() 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 708e690db6c..0c490856363 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 @@ -541,8 +541,7 @@ public class ForkingTaskRunnerTest .setDefaultHadoopCoordinates(ImmutableList.of()) .setGracefulShutdownTimeout(new Period("PT0S")) .setDirectoryLockTimeout(new Period("PT10S")) - .setShuffleDataLocations(ImmutableList.of()) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()); + .setShuffleDataLocations(ImmutableList.of()); } @Nonnull 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 5ee6ac6f4c1..3acf9ff6f0e 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 @@ -94,7 +94,6 @@ public class SingleTaskBackgroundRunnerTest .setBaseDir(temporaryFolder.newFile().toString()) .setDefaultRowFlushBoundary(50000) .setRestoreTasksOnRestart(true) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); 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 2df65120d35..ddbed6be7c7 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 @@ -562,7 +562,6 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest taskConfig = new TaskConfigBuilder() .setBaseDir(temporaryFolder.newFolder().toString()) .setDefaultRowFlushBoundary(50000) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .setTmpStorageBytesPerTask(-1L) .build(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index ea6d97448f8..258ebff7b50 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -573,7 +573,6 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport .setBaseTaskDir(new File(directory, "baseTaskDir").getPath()) .setDefaultRowFlushBoundary(50000) .setRestoreTasksOnRestart(true) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); final TestDerbyConnector derbyConnector = derby.getConnector(); derbyConnector.createDataSourceTable(); 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 3eea9d33fb3..6b08be3a3c6 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 @@ -116,7 +116,6 @@ public class WorkerTaskManagerTest .setBaseDir(FileUtils.createTempDir().toString()) .setDefaultRowFlushBoundary(0) .setRestoreTasksOnRestart(restoreTasksOnRestart) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); TaskActionClientFactory taskActionClientFactory = EasyMock.createNiceMock(TaskActionClientFactory.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 fcdada15f5d..e1822505b97 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 @@ -160,7 +160,6 @@ public class WorkerTaskMonitorTest final TaskConfig taskConfig = new TaskConfigBuilder() .setBaseDir(FileUtils.createTempDir().toString()) .setDefaultRowFlushBoundary(0) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); TaskActionClientFactory taskActionClientFactory = EasyMock.createNiceMock(TaskActionClientFactory.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 f5b0ca5644e..5505160d989 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 @@ -67,7 +67,6 @@ public class LocalIntermediaryDataManagerAutoCleanupTest { this.taskConfig = new TaskConfigBuilder() .setShuffleDataLocations(ImmutableList.of(new StorageLocationConfig(tempDir.newFolder(), null, null))) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); this.overlordClient = new NoopOverlordClient() { 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 fabb1cfb196..cc14d4505e9 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 @@ -70,7 +70,6 @@ public class LocalIntermediaryDataManagerManualAddAndDeleteTest siblingLocation = tempDir.newFolder(); final TaskConfig taskConfig = new TaskConfigBuilder() .setShuffleDataLocations(ImmutableList.of(new StorageLocationConfig(intermediarySegmentsLocation, 1200L, null))) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); final OverlordClient overlordClient = new NoopOverlordClient(); intermediaryDataManager = new LocalIntermediaryDataManager(workerConfig, taskConfig, overlordClient); 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 ceb74f3631a..c8c9b7fe066 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 @@ -101,7 +101,6 @@ public class ShuffleDataSegmentPusherTest final WorkerConfig workerConfig = new WorkerConfig(); final TaskConfig taskConfig = new TaskConfigBuilder() .setShuffleDataLocations(ImmutableList.of(new StorageLocationConfig(temporaryFolder.newFolder(), null, null))) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); final OverlordClient overlordClient = new NoopOverlordClient(); 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 40e8ad3e276..69d48417dee 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 @@ -93,7 +93,6 @@ public class ShuffleResourceTest }; final TaskConfig taskConfig = new TaskConfigBuilder() .setShuffleDataLocations(ImmutableList.of(new StorageLocationConfig(tempDir.newFolder(), null, null))) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) .build(); final OverlordClient overlordClient = new NoopOverlordClient() { diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java index 98588de1d53..77188db5857 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java @@ -98,8 +98,8 @@ public class BuildingHashBasedNumberedShardSpec implements BuildingShardSpec PartitionChunk createChunk(T obj) { - // This method can be called in AppenderatorImpl to create a sinkTimeline. - // The sinkTimeline doesn't seem in use in batch ingestion, let's set 'chunks' to 0 for now. + // This method can be called in StreamAppenderator to create a sinkTimeline. + // The sinkTimeline isn't used in batch ingestion, let's set 'chunks' to 0 for now. // HashBasedNumberedShardSpec is using NumberedPartitionChunk, so we use it here too. return new NumberedPartitionChunk<>(partitionId, 0, obj); } diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java index 52b3069f0b4..3989340c7b4 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java @@ -61,8 +61,8 @@ public class BuildingNumberedShardSpec implements BuildingShardSpec PartitionChunk createChunk(T obj) { - // This method can be called in AppenderatorImpl to create a sinkTimeline. - // The sinkTimeline doesn't seem in use in batch ingestion, let's set 'chunks' to 0 for now. + // This method can be called in StreamAppenderator to create a sinkTimeline. + // The sinkTimeline isn't used in batch ingestion, let's set 'chunks' to 0 for now. return new NumberedPartitionChunk<>(partitionId, 0, obj); } 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 deleted file mode 100644 index d9911366bdf..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ /dev/null @@ -1,1638 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.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.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.BaseProgressIndicator; -import org.apache.druid.segment.DataSegmentWithMetadata; -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.SchemaPayload; -import org.apache.druid.segment.SchemaPayloadPlus; -import org.apache.druid.segment.Segment; -import org.apache.druid.segment.SegmentSchemaMapping; -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.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.FingerprintGenerator; -import org.apache.druid.segment.realtime.FireHydrant; -import org.apache.druid.segment.realtime.SegmentGenerationMetrics; -import org.apache.druid.segment.realtime.sink.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 - */ -@SuppressWarnings("CheckReturnValue") -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 SegmentGenerationMetrics 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; - private final boolean useMaxMemoryEstimates; - - /** - * 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<>()); - - private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - - private final FingerprintGenerator fingerprintGenerator; - - /** - * 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, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - DataSegmentAnnouncer segmentAnnouncer, - @Nullable SinkQuerySegmentWalker sinkQuerySegmentWalker, - IndexIO indexIO, - IndexMerger indexMerger, - Cache cache, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean isOpenSegments, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - 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; - this.useMaxMemoryEstimates = useMaxMemoryEstimates; - - 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.debug("Running open segments appenderator"); - } else { - log.debug("Running closed segments appenderator"); - } - this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; - this.fingerprintGenerator = new FingerprintGenerator(objectMapper); - } - - @Override - public String getId() - { - return myId; - } - - @Override - public String getDataSource() - { - return schema.getDataSource(); - } - - @Override - public Object startJob() - { - 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; - } - }, - MoreExecutors.directExecutor() - ); - } 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, - useMaxMemoryEstimates - ); - bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed()); - - 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<>(); - final SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); - - 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 DataSegmentWithMetadata dataSegmentWithMetadata = mergeAndPush( - entry.getKey(), - entry.getValue(), - useUniquePath - ); - - if (dataSegmentWithMetadata != null) { - DataSegment segment = dataSegmentWithMetadata.getDataSegment(); - dataSegments.add(segment); - SchemaPayloadPlus schemaPayloadPlus = dataSegmentWithMetadata.getSegmentSchemaMetadata(); - if (schemaPayloadPlus != null) { - SchemaPayload schemaPayload = schemaPayloadPlus.getSchemaPayload(); - segmentSchemaMapping.addSchema( - segment.getId(), - schemaPayloadPlus, - fingerprintGenerator.generateFingerprint( - schemaPayload, - segment.getDataSource(), - CentralizedDatasourceSchemaConfig.SCHEMA_VERSION - ) - ); - } - } else { - log.warn("mergeAndPush[%s] returned null, skipping.", entry.getKey()); - } - } - - log.info("Push complete..."); - - return new SegmentsAndCommitMetadata(dataSegments, commitMetadata, segmentSchemaMapping); - }, - 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 DataSegmentWithMetadata 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 new DataSegmentWithMetadata( - objectMapper.readValue(descriptorFile, DataSegment.class), - centralizedDatasourceSchemaConfig.isEnabled() ? TaskSegmentSchemaUtil.getSegmentSchema( - mergedTarget, - indexIO - ) : null - ); - } - } - - 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.getIndexSpecForIntermediatePersists(), - new BaseProgressIndicator(), - 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()) - ); - - // 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 retries internally when appropriate; no need for retries here. - final DataSegment segment = dataSegmentPusher.push(mergedFile, segmentToPush, useUniquePath); - - 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 new DataSegmentWithMetadata( - segment, - centralizedDatasourceSchemaConfig.isEnabled() - ? TaskSegmentSchemaUtil.getSegmentSchema(mergedTarget, indexIO) - : null - ); - } - 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 { - FileUtils.mkdirp(tuningConfig.getBasePersistDirectory()); - - 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 input source when write to disk is slow - persistExecutor = MoreExecutors.listeningDecorator( - Execs.newBlockingThreaded( - "[" + StringUtils.encodeForFormat(myId) + "]-appenderator-persist", - tuningConfig.getNumPersistThreads(), maxPendingPersists - ) - ); - } - - if (pushExecutor == null) { - // use a blocking single threaded executor to throttle the input source 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, - useMaxMemoryEstimates, - 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()); - 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); - FileUtils.mkdirp(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() - { - 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 56af5f3f62c..28b4379f7b9 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 @@ -37,7 +37,6 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; import org.apache.druid.timeline.VersionedIntervalTimeline; public class Appenderators @@ -97,7 +96,7 @@ public class Appenderators ); } - public static Appenderator createOffline( + public static Appenderator createBatch( String id, DataSchema schema, AppenderatorConfig config, @@ -130,77 +129,4 @@ public class Appenderators centralizedDatasourceSchemaConfig ); } - - public static Appenderator createOpenSegmentsOffline( - String id, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - // 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, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - } - - public static Appenderator createClosedSegmentsOffline( - String id, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - return new AppenderatorImpl( - id, - schema, - config, - metrics, - dataSegmentPusher, - objectMapper, - new NoopDataSegmentAnnouncer(), - null, - indexIO, - indexMerger, - null, - rowIngestionMeters, - parseExceptionHandler, - false, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - } } 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 3bbf364656e..ec328c3b3cd 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 @@ -62,8 +62,9 @@ import org.joda.time.Interval; public interface AppenderatorsManager { /** - * Creates an Appenderator suited for realtime ingestion. Note that this method's parameters include objects - * used for query processing. + * Creates an {@link StreamAppenderator} suited for realtime ingestion. Note that this method's parameters include + * objects used for query processing. Intermediary segments are persisted to disk and memory mapped to be available + * for query processing. */ Appenderator createRealtimeAppenderatorForTask( SegmentLoaderConfig segmentLoaderConfig, @@ -90,39 +91,11 @@ public interface AppenderatorsManager ); /** - * Creates an Appenderator suited for batch ingestion. + * Creates a {@link BatchAppenderator} suitable for batch ingestion with no ability to process queries against + * the processed data. Intermediary segments are persisted to temporary disk and then merged into the final set of + * segments at publishing time. */ - Appenderator createOpenSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ); - - Appenderator createClosedSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ); - - Appenderator createOfflineAppenderatorForTask( + Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, 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 f44fffe20e1..d613f3ff59c 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 @@ -84,45 +84,7 @@ public class DummyForInjectionAppenderatorsManager implements AppenderatorsManag } @Override - public Appenderator createOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - throw new UOE(ERROR_MSG); - } - - @Override - public Appenderator createOpenSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - throw new UOE(ERROR_MSG); - } - - @Override - public Appenderator createClosedSegmentsOfflineAppenderatorForTask( + public Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, 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 52f75f72e47..998f674daf7 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 @@ -118,7 +118,7 @@ public class PeonAppenderatorsManager implements AppenderatorsManager } @Override - public Appenderator createOfflineAppenderatorForTask( + public Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, @@ -137,7 +137,7 @@ public class PeonAppenderatorsManager implements AppenderatorsManager if (realtimeAppenderator != null) { throw new ISE("A realtime appenderator was already created for this peon's task."); } else { - batchAppenderator = Appenderators.createOffline( + batchAppenderator = Appenderators.createBatch( taskId, schema, config, @@ -155,81 +155,6 @@ public class PeonAppenderatorsManager implements AppenderatorsManager } } - @Override - public Appenderator createOpenSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - // 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, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - return batchAppenderator; - } - } - - @Override - public Appenderator createClosedSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - // 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, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - 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 ffdfb8d1eb0..0088e33ca7a 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 @@ -205,7 +205,7 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager } @Override - public Appenderator createOfflineAppenderatorForTask( + public Appenderator createBatchAppenderatorForTask( String taskId, DataSchema schema, AppenderatorConfig config, @@ -226,89 +226,7 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager DatasourceBundle::new ); - Appenderator appenderator = Appenderators.createOffline( - taskId, - schema, - rewriteAppenderatorConfigMemoryLimits(config), - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - wrapIndexMerger(indexMerger), - rowIngestionMeters, - parseExceptionHandler, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - datasourceBundle.addAppenderator(taskId, appenderator); - return appenderator; - } - } - - @Override - public Appenderator createOpenSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - 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, - useMaxMemoryEstimates, - centralizedDatasourceSchemaConfig - ); - datasourceBundle.addAppenderator(taskId, appenderator); - return appenderator; - } - } - - @Override - public Appenderator createClosedSegmentsOfflineAppenderatorForTask( - String taskId, - DataSchema schema, - AppenderatorConfig config, - SegmentGenerationMetrics metrics, - DataSegmentPusher dataSegmentPusher, - ObjectMapper objectMapper, - IndexIO indexIO, - IndexMerger indexMerger, - RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean useMaxMemoryEstimates, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - synchronized (this) { - DatasourceBundle datasourceBundle = datasourceBundles.computeIfAbsent( - schema.getDataSource(), - DatasourceBundle::new - ); - - Appenderator appenderator = Appenderators.createClosedSegmentsOffline( + Appenderator appenderator = Appenderators.createBatch( taskId, schema, rewriteAppenderatorConfigMemoryLimits(config), @@ -595,7 +513,7 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager /** * This wrapper around IndexMerger limits concurrent calls to the merge/persist methods used by * {@link StreamAppenderator} with a shared executor service. Merge/persist methods that are not used by - * AppenderatorImpl will throw an exception if called. + * StreamAppenderator will throw an exception if called. */ public static class LimitedPoolIndexMerger implements IndexMerger { diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java deleted file mode 100644 index 66b2281d33c..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java +++ /dev/null @@ -1,245 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.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.NoopDataSegmentPusher; -import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.SegmentGenerationMetrics; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import org.apache.druid.timeline.partition.LinearShardSpec; -import org.junit.Assert; -import org.junit.Test; - -import javax.annotation.Nullable; -import java.io.File; -import java.util.Map; - - -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 AppenderatorConfig tuningConfig; - private final Appenderator appenderator; - private final ServiceEmitter emitter; - - public AppenderatorTester(final String batchMode) - { - this(100, 100, null, new SimpleRowIngestionMeters(), false, batchMode); - } - - public AppenderatorTester( - final int maxRowsInMemory, - final long maxSizeInBytes, - @Nullable final File basePersistDirectory, - final RowIngestionMeters rowIngestionMeters, - final boolean skipBytesInMemoryOverheadCheck, - String batchMode - ) - { - ObjectMapper objectMapper = new DefaultObjectMapper(); - objectMapper.registerSubtypes(LinearShardSpec.class); - - final Map parserMap = objectMapper.convertValue( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec("ts", "auto", null), - DimensionsSpec.EMPTY, - null, - null, - null - ) - ), - Map.class - ); - - DataSchema 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 TestAppenderatorConfig( - TuningConfig.DEFAULT_APPENDABLE_INDEX, - maxRowsInMemory, - maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, - skipBytesInMemoryOverheadCheck, - IndexSpec.DEFAULT, - 0, - false, - 0L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, - basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory - ); - SegmentGenerationMetrics metrics = new SegmentGenerationMetrics(); - - IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); - IndexMergerV9 indexMerger = new IndexMergerV9( - objectMapper, - indexIO, - OffHeapMemorySegmentWriteOutMediumFactory.instance() - ); - - emitter = new ServiceEmitter( - "test", - "test", - new NoopEmitter() - ); - emitter.start(); - EmittingLogger.registerEmitter(emitter); - - switch (batchMode) { - case "OPEN_SEGMENTS": - appenderator = Appenderators.createOpenSegmentsOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - new NoopDataSegmentPusher(), - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false, - CentralizedDatasourceSchemaConfig.create() - ); - break; - case "CLOSED_SEGMENTS": - appenderator = Appenderators.createClosedSegmentsOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - new NoopDataSegmentPusher(), - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false, - CentralizedDatasourceSchemaConfig.create() - ); - - break; - case "CLOSED_SEGMENTS_SINKS": - appenderator = Appenderators.createOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - new NoopDataSegmentPusher(), - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false, - CentralizedDatasourceSchemaConfig.create() - ); - break; - default: - throw new IllegalArgumentException("Unrecognized batchMode: " + batchMode); - } - } - - private long getDefaultMaxBytesInMemory() - { - return (Runtime.getRuntime().totalMemory()) / 3; - } - - @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/ClosedSegmentsSinksBatchAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java similarity index 97% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java index 269aeaca7c4..ed63bca3195 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java @@ -52,7 +52,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Collectors; -public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupport +public class BatchAppenderatorDriverTest extends EasyMockSupport { private static final String DATA_SOURCE = "foo"; private static final String VERSION = "abc123"; @@ -78,7 +78,7 @@ public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupp ); private SegmentAllocator allocator; - private ClosedSegmensSinksBatchAppenderatorTester appenderatorTester; + private BatchAppenderatorTester appenderatorTester; private BatchAppenderatorDriver driver; private DataSegmentKiller dataSegmentKiller; @@ -89,7 +89,7 @@ public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupp @Before public void setup() { - appenderatorTester = new ClosedSegmensSinksBatchAppenderatorTester(MAX_ROWS_IN_MEMORY); + appenderatorTester = new BatchAppenderatorTester(MAX_ROWS_IN_MEMORY); allocator = new TestSegmentAllocator(DATA_SOURCE, Granularities.HOUR); dataSegmentKiller = createStrictMock(DataSegmentKiller.class); driver = new BatchAppenderatorDriver( diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java similarity index 91% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java index 55b5f235506..8b9117705f5 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java @@ -49,7 +49,7 @@ import java.util.List; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; -public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHandlingTest +public class BatchAppenderatorTest extends InitializedNullHandlingTest { private static final List IDENTIFIERS = ImmutableList.of( createSegmentId("2000/2001", "A", 0), // should be in seg_0 @@ -60,14 +60,14 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testSimpleIngestion() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); // startJob Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // add #1 Assert.assertEquals( @@ -136,14 +136,14 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testPushFailure() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, true)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); // startJob Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // add #1 Assert.assertEquals( @@ -219,14 +219,14 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testPeriodGranularityNonUTCIngestion() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(1, false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(1, false)) { final Appenderator appenderator = tester.getAppenderator(); // startJob Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // Create a segment identifier with a non-utc interval SegmentIdWithShardSpec segmentIdWithNonUTCTime = @@ -281,7 +281,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan public void testMaxBytesInMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( + final BatchAppenderatorTester tester = new BatchAppenderatorTester( 100, 1024, null, @@ -316,7 +316,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( + final BatchAppenderatorTester tester = new BatchAppenderatorTester( 100, 1024, null, @@ -346,7 +346,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testMaxBytesInMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(100, 15000, true)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(100, 15000, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -425,8 +425,8 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test(expected = RuntimeException.class, timeout = 5000L) public void testTaskFailAsPersistCannotFreeAnyMoreMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(100, 5180, true)) { + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(100, 5180, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); @@ -437,7 +437,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( + final BatchAppenderatorTester tester = new BatchAppenderatorTester( 100, 10, null, @@ -468,8 +468,8 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testTaskCleanupInMemoryCounterAfterCloseWithRowInMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(100, 10000, true)) { + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(100, 10000, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -495,8 +495,8 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testMaxBytesInMemoryInMultipleSinks() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(1000, 28748, true)) { + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(1000, 28748, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -616,8 +616,8 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testIgnoreMaxBytesInMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(100, -1, true)) { + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(100, -1, true)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -649,7 +649,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testMaxRowsInMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -682,7 +682,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testAllHydrantsAreRecovered() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(1, false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(1, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -715,7 +715,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testTotalRowsPerSegment() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -775,7 +775,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testRestoreFromDisk() throws Exception { - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(2, false); + final BatchAppenderatorTester tester = new BatchAppenderatorTester(2, false); final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -814,7 +814,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testCleanupFromDiskAfterClose() throws Exception { - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(2, false); + final BatchAppenderatorTester tester = new BatchAppenderatorTester(2, false); final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -854,7 +854,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test(timeout = 5000L) public void testTotalRowCount() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, appenderator.getTotalRowCount()); @@ -895,10 +895,10 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan public void testVerifyRowIngestionMetrics() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(5, - 10000L, - null, false, rowIngestionMeters + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(5, + 10000L, + null, false, rowIngestionMeters )) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -918,10 +918,10 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan public void testPushContract() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(1, - 50000L, - null, false, rowIngestionMeters + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(1, + 50000L, + null, false, rowIngestionMeters )) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -967,10 +967,10 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan public void testCloseContract() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(1, - 50000L, - null, false, rowIngestionMeters + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(1, + 50000L, + null, false, rowIngestionMeters )) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -1018,7 +1018,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan private static SegmentIdWithShardSpec createNonUTCSegmentId(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( - ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, + BatchAppenderatorTester.DATASOURCE, new Interval(interval, ISOChronology.getInstance(DateTimes.inferTzFromString("Asia/Seoul"))), version, new LinearShardSpec(partitionNum) @@ -1029,7 +1029,7 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan private static SegmentIdWithShardSpec createSegmentId(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( - ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, + BatchAppenderatorTester.DATASOURCE, Intervals.of(interval), version, new LinearShardSpec(partitionNum) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java similarity index 95% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java index cf2d7f79898..22034aa33aa 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java @@ -59,7 +59,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; -public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable +public class BatchAppenderatorTester implements AutoCloseable { public static final String DATASOURCE = "foo"; @@ -72,14 +72,14 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable private final List pushedSegments = new CopyOnWriteArrayList<>(); - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory ) { this(maxRowsInMemory, -1, null, false); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final boolean enablePushFailure ) @@ -87,7 +87,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable this(maxRowsInMemory, -1, null, enablePushFailure); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final boolean enablePushFailure @@ -96,7 +96,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable this(maxRowsInMemory, maxSizeInBytes, null, enablePushFailure); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final File basePersistDirectory, @@ -113,7 +113,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable ); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, @Nullable final File basePersistDirectory, @@ -126,7 +126,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable ); } - public ClosedSegmensSinksBatchAppenderatorTester( + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, @Nullable final File basePersistDirectory, @@ -230,7 +230,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable throw new UnsupportedOperationException(); } }; - appenderator = Appenderators.createOffline( + appenderator = Appenderators.createBatch( schema.getDataSource(), schema, tuningConfig, 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 deleted file mode 100644 index 33a0ed2f8a4..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java +++ /dev/null @@ -1,289 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.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.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.SegmentGenerationMetrics; -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 SegmentGenerationMetrics metrics; - private final DataSegmentPusher dataSegmentPusher; - private final ObjectMapper objectMapper; - private final Appenderator appenderator; - private final IndexIO indexIO; - private final IndexMergerV9 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), - DimensionsSpec.EMPTY, - 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 TestAppenderatorConfig( - TuningConfig.DEFAULT_APPENDABLE_INDEX, - maxRowsInMemory, - maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, - skipBytesInMemoryOverheadCheck, - IndexSpec.DEFAULT, - 0, - false, - 0L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, - basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory - ); - - metrics = new SegmentGenerationMetrics(); - - indexIO = new IndexIO( - objectMapper, - new ColumnConfig() - { - } - ); - indexMerger = new IndexMergerV9(objectMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); - - emitter = new ServiceEmitter( - "test", - "test", - new NoopEmitter() - ); - emitter.start(); - EmittingLogger.registerEmitter(emitter); - dataSegmentPusher = new DataSegmentPusher() - { - @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) { - throw new IOException("Push failure test"); - } - 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), - true, - CentralizedDatasourceSchemaConfig.create() - ); - } else { - appenderator = Appenderators.createClosedSegmentsOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - true, - CentralizedDatasourceSchemaConfig.create() - ); - } - } - - private long getDefaultMaxBytesInMemory() - { - return (Runtime.getRuntime().totalMemory()) / 3; - } - - public DataSchema getSchema() - { - return schema; - } - - public AppenderatorConfig getTuningConfig() - { - return tuningConfig; - } - - public SegmentGenerationMetrics 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 deleted file mode 100644 index 0c6fb552a4d..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorDriverTest.java +++ /dev/null @@ -1,205 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.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 TestPublishedSegmentRetriever(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(), null).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(), null).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, segmentsToPublish, commitMetadata, schema) -> 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 deleted file mode 100644 index 2f5e5cde7ed..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsBatchAppenderatorTest.java +++ /dev/null @@ -1,228 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.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 9bf629931b5..23ac93db009 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 @@ -96,7 +96,7 @@ public class UnifiedIndexerAppenderatorsManagerTest extends InitializedNullHandl EasyMock.expect(appenderatorConfig.getMaxPendingPersists()).andReturn(0); EasyMock.expect(appenderatorConfig.isSkipBytesInMemoryOverheadCheck()).andReturn(false); EasyMock.replay(appenderatorConfig); - appenderator = manager.createClosedSegmentsOfflineAppenderatorForTask( + appenderator = manager.createBatchAppenderatorForTask( "taskId", new DataSchema( "myDataSource",