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
This commit is contained in:
Clint Wylie 2024-07-22 13:56:44 -07:00 committed by GitHub
parent 6a2348b78b
commit 02b8738c00
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
39 changed files with 87 additions and 3790 deletions

View File

@ -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|

View File

@ -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(

View File

@ -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<StorageLocationConfig> 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<StorageLocationConfig> 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

View File

@ -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,8 +73,7 @@ public final class BatchAppenderators
boolean useMaxMemoryEstimates
)
{
if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.OPEN_SEGMENTS) {
return appenderatorsManager.createOpenSegmentsOfflineAppenderatorForTask(
return appenderatorsManager.createBatchAppenderatorForTask(
taskId,
dataSchema,
appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()),
@ -90,39 +87,6 @@ public final class BatchAppenderators
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());
}
}
public static BatchAppenderatorDriver newDriver(

View File

@ -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(

View File

@ -36,8 +36,6 @@ public class TaskConfigBuilder
private Period directoryLockTimeout;
private List<StorageLocationConfig> 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

View File

@ -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<DataSegment> 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<String, Object> 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<String, Object> 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<DataSegment> 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();
}
}
}

View File

@ -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);

View File

@ -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)

View File

@ -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);

View File

@ -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);

View File

@ -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,

View File

@ -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()

View File

@ -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

View File

@ -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);

View File

@ -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();

View File

@ -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();

View File

@ -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);

View File

@ -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);

View File

@ -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()
{

View File

@ -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);

View File

@ -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)) {

View File

@ -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()
{

View File

@ -98,8 +98,8 @@ public class BuildingHashBasedNumberedShardSpec implements BuildingShardSpec<Has
@Override
public <T> PartitionChunk<T> 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);
}

View File

@ -61,8 +61,8 @@ public class BuildingNumberedShardSpec implements BuildingShardSpec<NumberedShar
@Override
public <T> PartitionChunk<T> 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);
}

View File

@ -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
);
}
}

View File

@ -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,

View File

@ -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,

View File

@ -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)
{

View File

@ -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
{

View File

@ -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<String, Object> 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");
}
}
}

View File

@ -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(

View File

@ -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<SegmentIdWithShardSpec> 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,8 +895,8 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan
public void testVerifyRowIngestionMetrics() throws Exception
{
final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters();
try (final ClosedSegmensSinksBatchAppenderatorTester tester =
new ClosedSegmensSinksBatchAppenderatorTester(5,
try (final BatchAppenderatorTester tester =
new BatchAppenderatorTester(5,
10000L,
null, false, rowIngestionMeters
)) {
@ -918,8 +918,8 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan
public void testPushContract() throws Exception
{
final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters();
try (final ClosedSegmensSinksBatchAppenderatorTester tester =
new ClosedSegmensSinksBatchAppenderatorTester(1,
try (final BatchAppenderatorTester tester =
new BatchAppenderatorTester(1,
50000L,
null, false, rowIngestionMeters
)) {
@ -967,8 +967,8 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan
public void testCloseContract() throws Exception
{
final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters();
try (final ClosedSegmensSinksBatchAppenderatorTester tester =
new ClosedSegmensSinksBatchAppenderatorTester(1,
try (final BatchAppenderatorTester tester =
new BatchAppenderatorTester(1,
50000L,
null, false, rowIngestionMeters
)) {
@ -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)

View File

@ -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<DataSegment> 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,

View File

@ -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<DataSegment> 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<String, Object> 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<String, Object> 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<DataSegment> 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");
}
}

View File

@ -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<InputRow> 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<SegmentWithState> 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());
}
}

View File

@ -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<SegmentIdWithShardSpec> 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<DataSegment, SegmentIdWithShardSpec>()
{
@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<DataSegment, SegmentIdWithShardSpec>()
{
@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
)
);
}
}

View File

@ -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",