mirror of https://github.com/apache/druid.git
Make persists concurrent with adding rows in batch ingestion (#11536)
* Make persists concurrent with ingestion * Remove semaphore but keep concurrent persists (with add) and add push in the backround as well * Go back to documented default persists (zero) * Move to debug * Remove unnecessary Atomics * Comments on synchronization (or not) for sinks & sinkMetadata * Some cleanup for unit tests but they still need further work * Shutdown & wait for persists and push on close * Provide support for three existing batch appenderators using batchProcessingMode flag * Fix reference to wrong appenderator * Fix doc typos * Add BatchAppenderators class test coverage * Add log message to batchProcessingMode final value, fix typo in enum name * Another typo and minor fix to log message * LEGACY->OPEN_SEGMENTS, Edit docs * Minor update legacy->open segments log message * More code comments, mostly small adjustments to naming etc * fix spelling * Exclude BtachAppenderators from Jacoco since it is fully tested but Jacoco still refuses to ack coverage * Coverage for Appenderators & BatchAppenderators, name change of a method that was still using "legacy" rather than "openSegments" Co-authored-by: Clint Wylie <cjwylie@gmail.com>
This commit is contained in:
parent
dcee99df78
commit
9efa6cc9c8
|
@ -1343,7 +1343,7 @@ Additional peon configs include:
|
|||
|`druid.peon.mode`|Choices are "local" and "remote". Setting this to local means you intend to run the peon as a standalone process (Not recommended).|remote|
|
||||
|`druid.indexer.task.baseDir`|Base temporary working directory.|`System.getProperty("java.io.tmpdir")`|
|
||||
|`druid.indexer.task.baseTaskDir`|Base temporary working directory for tasks.|`${druid.indexer.task.baseDir}/persistent/task`|
|
||||
|`druid.indexer.task.useLegacyBatchProcessing`|If false, native batch ingestion will use a new, recommended, code path with memory optimized code for the segment creation phase. If true it will use the previous code path for the create segments phase of batch ingestion. This does not apply to streaming ingestion, just to batch. This setting should only be used when a bug is suspected or found in the new optimized batch ingestion code. If a bug is suspected or found, you can set this flag to `true` to fall back to previous, working but more memory intensive, code path.|`false`|
|
||||
|`druid.indexer.task.batchProcessingMode`| Batch ingestion tasks have three operating modes that control how intermediary segments are constructed and tracked: `OPEN_SEGMENTS`, `CLOSED_SEGMENTS`, and `CLOSED_SEGMENT_SINKS`. `OPEN_SEGMENTS` will use code based on the original batch ingestion path and performs a `mmap` on intermediary segments to build a timeline so that these segments can be queryable by realtime queries. This is not needed at all for batch, so the default mode, `CLOSED_SEGMENTS`, eliminates `mmap` of intermediary segments, but still tracks the entire set of segments in heap. The `CLOSED_SEGMENTS_SINKS` mode is the most aggressive and should have the smallest memory footprint, and works by eliminating in memory tracking and `mmap` of intermediary segments produced during segment creation. This mode isn't as well tested as other modes so is currently considered experimental. `OPEN_SEGMENTS` mode can be selected if any problems occur with the 2 newer modes. |`CLOSED_SEGMENTS`|
|
||||
|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.8.5|
|
||||
|`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|75000|
|
||||
|`druid.indexer.task.directoryLockTimeout`|Wait this long for zombie peons to exit before giving up on their replacements.|PT10M|
|
||||
|
|
|
@ -2783,7 +2783,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
);
|
||||
final TestDerbyConnector derbyConnector = derby.getConnector();
|
||||
derbyConnector.createDataSourceTable();
|
||||
|
|
|
@ -2869,7 +2869,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
);
|
||||
final TestDerbyConnector derbyConnector = derby.getConnector();
|
||||
derbyConnector.createDataSourceTable();
|
||||
|
|
|
@ -22,6 +22,8 @@ package org.apache.druid.indexing.common.config;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.commons.lang3.EnumUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.segment.loading.StorageLocationConfig;
|
||||
import org.joda.time.Period;
|
||||
|
||||
|
@ -39,10 +41,22 @@ import java.util.List;
|
|||
*/
|
||||
public class TaskConfig
|
||||
{
|
||||
private static final Logger log = new Logger(TaskConfig.class);
|
||||
|
||||
public static final List<String> DEFAULT_DEFAULT_HADOOP_COORDINATES = ImmutableList.of(
|
||||
"org.apache.hadoop:hadoop-client:2.8.5"
|
||||
);
|
||||
|
||||
// This enum controls processing mode of batch ingestion "segment creation" phase (i.e. appenderator logic)
|
||||
public enum BatchProcessingMode
|
||||
{
|
||||
OPEN_SEGMENTS, /* mmap segments, legacy code */
|
||||
CLOSED_SEGMENTS, /* Do not mmap segments but keep most other legacy code */
|
||||
CLOSED_SEGMENTS_SINKS /* Most aggressive memory optimization, do not mmap segments and eliminate sinks, etc. */
|
||||
}
|
||||
|
||||
public static final BatchProcessingMode BATCH_PROCESSING_MODE_DEFAULT = BatchProcessingMode.CLOSED_SEGMENTS;
|
||||
|
||||
private static final Period DEFAULT_DIRECTORY_LOCK_TIMEOUT = new Period("PT10M");
|
||||
private static final Period DEFAULT_GRACEFUL_SHUTDOWN_TIMEOUT = new Period("PT5M");
|
||||
|
||||
|
@ -77,7 +91,10 @@ public class TaskConfig
|
|||
private final boolean ignoreTimestampSpecForDruidInputSource;
|
||||
|
||||
@JsonProperty
|
||||
private final boolean useLegacyBatchProcessing;
|
||||
private final boolean batchMemoryMappedIndex;
|
||||
|
||||
@JsonProperty
|
||||
private final BatchProcessingMode batchProcessingMode;
|
||||
|
||||
@JsonCreator
|
||||
public TaskConfig(
|
||||
|
@ -91,7 +108,8 @@ public class TaskConfig
|
|||
@JsonProperty("directoryLockTimeout") Period directoryLockTimeout,
|
||||
@JsonProperty("shuffleDataLocations") List<StorageLocationConfig> shuffleDataLocations,
|
||||
@JsonProperty("ignoreTimestampSpecForDruidInputSource") boolean ignoreTimestampSpecForDruidInputSource,
|
||||
@JsonProperty("useLegacyBatchProcessing") boolean useLegacyBatchProcessing // only set to true to fall back to older behavior
|
||||
@JsonProperty("batchMemoryMappedIndex") boolean batchMemoryMappedIndex, // deprecated, only set to true to fall back to older behavior
|
||||
@JsonProperty("batchProcessingMode") String batchProcessingMode
|
||||
)
|
||||
{
|
||||
this.baseDir = baseDir == null ? System.getProperty("java.io.tmpdir") : baseDir;
|
||||
|
@ -117,7 +135,23 @@ public class TaskConfig
|
|||
this.shuffleDataLocations = shuffleDataLocations;
|
||||
}
|
||||
this.ignoreTimestampSpecForDruidInputSource = ignoreTimestampSpecForDruidInputSource;
|
||||
this.useLegacyBatchProcessing = useLegacyBatchProcessing;
|
||||
|
||||
this.batchMemoryMappedIndex = batchMemoryMappedIndex;
|
||||
// Conflict resolution. Assume that if batchMemoryMappedIndex is set (since false is the default) that
|
||||
// the user changed it intentionally to use legacy, in this case oveeride batchProcessingMode and also
|
||||
// set it to legacy else just use batchProcessingMode and don't pay attention to batchMemoryMappedIndexMode:
|
||||
if (batchMemoryMappedIndex) {
|
||||
this.batchProcessingMode = BatchProcessingMode.OPEN_SEGMENTS;
|
||||
} else if (EnumUtils.isValidEnum(BatchProcessingMode.class, batchProcessingMode)) {
|
||||
this.batchProcessingMode = BatchProcessingMode.valueOf(batchProcessingMode);
|
||||
} else {
|
||||
// batchProcessingMode input string is invalid, log & use the default.
|
||||
this.batchProcessingMode = BatchProcessingMode.CLOSED_SEGMENTS; // Default
|
||||
log.warn("Batch processing mode argument value is null or not valid:[%s], defaulting to[%s] ",
|
||||
batchProcessingMode, this.batchProcessingMode
|
||||
);
|
||||
}
|
||||
log.info("Batch processing mode:[%s]", this.batchProcessingMode);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -201,9 +235,19 @@ public class TaskConfig
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean getuseLegacyBatchProcessing()
|
||||
public BatchProcessingMode getBatchProcessingMode()
|
||||
{
|
||||
return useLegacyBatchProcessing;
|
||||
return batchProcessingMode;
|
||||
}
|
||||
|
||||
/**
|
||||
* Do not use in code! use {@link TaskConfig#getBatchProcessingMode() instead}
|
||||
*/
|
||||
@Deprecated
|
||||
@JsonProperty
|
||||
public boolean getbatchMemoryMappedIndex()
|
||||
{
|
||||
return batchMemoryMappedIndex;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -21,6 +21,8 @@ package org.apache.druid.indexing.common.task;
|
|||
|
||||
import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.segment.incremental.ParseExceptionHandler;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
|
@ -70,21 +72,51 @@ public final class BatchAppenderators
|
|||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
return appenderatorsManager.createOfflineAppenderatorForTask(
|
||||
taskId,
|
||||
dataSchema,
|
||||
appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()),
|
||||
metrics,
|
||||
segmentPusher,
|
||||
toolbox.getJsonMapper(),
|
||||
toolbox.getIndexIO(),
|
||||
toolbox.getIndexMergerV9(),
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler,
|
||||
toolbox.getConfig().getuseLegacyBatchProcessing()
|
||||
);
|
||||
if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.OPEN_SEGMENTS) {
|
||||
return appenderatorsManager.createOpenSegmentsOfflineAppenderatorForTask(
|
||||
taskId,
|
||||
dataSchema,
|
||||
appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()),
|
||||
metrics,
|
||||
segmentPusher,
|
||||
toolbox.getJsonMapper(),
|
||||
toolbox.getIndexIO(),
|
||||
toolbox.getIndexMergerV9(),
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler
|
||||
);
|
||||
} else if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS) {
|
||||
return appenderatorsManager.createClosedSegmentsOfflineAppenderatorForTask(
|
||||
taskId,
|
||||
dataSchema,
|
||||
appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()),
|
||||
metrics,
|
||||
segmentPusher,
|
||||
toolbox.getJsonMapper(),
|
||||
toolbox.getIndexIO(),
|
||||
toolbox.getIndexMergerV9(),
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler
|
||||
);
|
||||
} else if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS_SINKS) {
|
||||
return appenderatorsManager.createOfflineAppenderatorForTask(
|
||||
taskId,
|
||||
dataSchema,
|
||||
appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()),
|
||||
metrics,
|
||||
segmentPusher,
|
||||
toolbox.getJsonMapper(),
|
||||
toolbox.getIndexIO(),
|
||||
toolbox.getIndexMergerV9(),
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler
|
||||
);
|
||||
} else {
|
||||
throw new IAE("Invalid batchProcesingMode[%s]", toolbox.getConfig().getBatchProcessingMode());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static BatchAppenderatorDriver newDriver(
|
||||
final Appenderator appenderator,
|
||||
final TaskToolbox toolbox,
|
||||
|
|
|
@ -113,7 +113,8 @@ public class TaskToolboxTest
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
),
|
||||
new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false),
|
||||
mockTaskActionClientFactory,
|
||||
|
|
|
@ -1518,7 +1518,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
);
|
||||
|
||||
final TaskActionToolbox taskActionToolbox = new TaskActionToolbox(
|
||||
|
|
|
@ -0,0 +1,534 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.indexing.common.task;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.JSONParseSpec;
|
||||
import org.apache.druid.data.input.impl.MapInputRowParser;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.FileUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.java.util.emitter.core.NoopEmitter;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.IndexMerger;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.incremental.AppendableIndexSpec;
|
||||
import org.apache.druid.segment.incremental.ParseExceptionHandler;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
import org.apache.druid.segment.incremental.SimpleRowIngestionMeters;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.TuningConfig;
|
||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
import org.apache.druid.segment.loading.DataSegmentPusher;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
import org.apache.druid.segment.realtime.appenderator.Appenderator;
|
||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig;
|
||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl;
|
||||
import org.apache.druid.segment.realtime.appenderator.Appenderators;
|
||||
import org.apache.druid.segment.realtime.appenderator.BatchAppenderator;
|
||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.joda.time.Period;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
|
||||
|
||||
public class AppenderatorsTest
|
||||
{
|
||||
@Test
|
||||
public void testOpenSegmentsOfflineAppenderator() throws Exception
|
||||
{
|
||||
try (final AppenderatorTester tester = new AppenderatorTester("OPEN_SEGMENTS")) {
|
||||
Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl);
|
||||
AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator;
|
||||
Assert.assertTrue(appenderator.isOpenSegments());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClosedSegmentsOfflineAppenderator() throws Exception
|
||||
{
|
||||
try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS")) {
|
||||
Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl);
|
||||
AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator;
|
||||
Assert.assertFalse(appenderator.isOpenSegments());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClosedSegmentsSinksOfflineAppenderator() throws Exception
|
||||
{
|
||||
try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS_SINKS")) {
|
||||
Assert.assertTrue(tester.appenderator instanceof BatchAppenderator);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private static class AppenderatorTester implements AutoCloseable
|
||||
{
|
||||
public static final String DATASOURCE = "foo";
|
||||
|
||||
private final DataSchema schema;
|
||||
private final AppenderatorConfig tuningConfig;
|
||||
private final FireDepartmentMetrics metrics;
|
||||
private final ObjectMapper objectMapper;
|
||||
private final Appenderator appenderator;
|
||||
private final ServiceEmitter emitter;
|
||||
|
||||
private final List<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),
|
||||
new DimensionsSpec(null, null, null),
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
),
|
||||
Map.class
|
||||
);
|
||||
|
||||
schema = new DataSchema(
|
||||
DATASOURCE,
|
||||
null,
|
||||
null,
|
||||
new AggregatorFactory[]{
|
||||
new CountAggregatorFactory("count"),
|
||||
new LongSumAggregatorFactory("met", "met")
|
||||
},
|
||||
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null),
|
||||
null,
|
||||
parserMap,
|
||||
objectMapper
|
||||
);
|
||||
|
||||
tuningConfig = new TestIndexTuningConfig(
|
||||
TuningConfig.DEFAULT_APPENDABLE_INDEX,
|
||||
maxRowsInMemory,
|
||||
maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes,
|
||||
skipBytesInMemoryOverheadCheck,
|
||||
new IndexSpec(),
|
||||
0,
|
||||
false,
|
||||
0L,
|
||||
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
|
||||
IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE,
|
||||
basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory
|
||||
);
|
||||
metrics = new FireDepartmentMetrics();
|
||||
|
||||
IndexIO indexIO = new IndexIO(
|
||||
objectMapper,
|
||||
() -> 0
|
||||
);
|
||||
IndexMerger indexMerger = new IndexMergerV9(
|
||||
objectMapper,
|
||||
indexIO,
|
||||
OffHeapMemorySegmentWriteOutMediumFactory.instance()
|
||||
);
|
||||
|
||||
emitter = new ServiceEmitter(
|
||||
"test",
|
||||
"test",
|
||||
new NoopEmitter()
|
||||
);
|
||||
emitter.start();
|
||||
EmittingLogger.registerEmitter(emitter);
|
||||
DataSegmentPusher dataSegmentPusher = new DataSegmentPusher()
|
||||
{
|
||||
private boolean mustFail = true;
|
||||
|
||||
@Deprecated
|
||||
@Override
|
||||
public String getPathForHadoop(String dataSource)
|
||||
{
|
||||
return getPathForHadoop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPathForHadoop()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException
|
||||
{
|
||||
if (enablePushFailure && mustFail) {
|
||||
mustFail = false;
|
||||
throw new IOException("Push failure test");
|
||||
} else if (enablePushFailure) {
|
||||
mustFail = true;
|
||||
}
|
||||
pushedSegments.add(segment);
|
||||
return segment;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> makeLoadSpec(URI uri)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
|
||||
switch (batchMode) {
|
||||
case "OPEN_SEGMENTS":
|
||||
appenderator = Appenderators.createOpenSegmentsOffline(
|
||||
schema.getDataSource(),
|
||||
schema,
|
||||
tuningConfig,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0)
|
||||
);
|
||||
break;
|
||||
case "CLOSED_SEGMENTS":
|
||||
appenderator = Appenderators.createClosedSegmentsOffline(
|
||||
schema.getDataSource(),
|
||||
schema,
|
||||
tuningConfig,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0)
|
||||
);
|
||||
|
||||
break;
|
||||
case "CLOSED_SEGMENTS_SINKS":
|
||||
appenderator = Appenderators.createOffline(
|
||||
schema.getDataSource(),
|
||||
schema,
|
||||
tuningConfig,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0)
|
||||
);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Unrecognized batchMode: " + batchMode);
|
||||
}
|
||||
}
|
||||
|
||||
private long getDefaultMaxBytesInMemory()
|
||||
{
|
||||
return (Runtime.getRuntime().totalMemory()) / 3;
|
||||
}
|
||||
|
||||
public DataSchema getSchema()
|
||||
{
|
||||
return schema;
|
||||
}
|
||||
|
||||
public AppenderatorConfig getTuningConfig()
|
||||
{
|
||||
return tuningConfig;
|
||||
}
|
||||
|
||||
public FireDepartmentMetrics getMetrics()
|
||||
{
|
||||
return metrics;
|
||||
}
|
||||
|
||||
public ObjectMapper getObjectMapper()
|
||||
{
|
||||
return objectMapper;
|
||||
}
|
||||
|
||||
public Appenderator getAppenderator()
|
||||
{
|
||||
return appenderator;
|
||||
}
|
||||
|
||||
public List<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;
|
||||
|
||||
public TestIndexTuningConfig(
|
||||
AppendableIndexSpec appendableIndexSpec,
|
||||
Integer maxRowsInMemory,
|
||||
Long maxBytesInMemory,
|
||||
Boolean skipBytesInMemoryOverheadCheck,
|
||||
IndexSpec indexSpec,
|
||||
Integer maxPendingPersists,
|
||||
Boolean reportParseExceptions,
|
||||
Long pushTimeout,
|
||||
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
|
||||
Integer maxColumnsToMerge,
|
||||
File basePersistDirectory
|
||||
)
|
||||
{
|
||||
this.appendableIndexSpec = appendableIndexSpec;
|
||||
this.maxRowsInMemory = maxRowsInMemory;
|
||||
this.maxBytesInMemory = maxBytesInMemory;
|
||||
this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck;
|
||||
this.indexSpec = indexSpec;
|
||||
this.maxPendingPersists = maxPendingPersists;
|
||||
this.reportParseExceptions = reportParseExceptions;
|
||||
this.pushTimeout = pushTimeout;
|
||||
this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
|
||||
this.maxColumnsToMerge = maxColumnsToMerge;
|
||||
this.basePersistDirectory = basePersistDirectory;
|
||||
|
||||
this.partitionsSpec = null;
|
||||
this.indexSpecForIntermediatePersists = this.indexSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TestIndexTuningConfig withBasePersistDirectory(File dir)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public AppendableIndexSpec getAppendableIndexSpec()
|
||||
{
|
||||
return appendableIndexSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxRowsInMemory()
|
||||
{
|
||||
return maxRowsInMemory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMaxBytesInMemory()
|
||||
{
|
||||
return maxBytesInMemory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isSkipBytesInMemoryOverheadCheck()
|
||||
{
|
||||
return skipBytesInMemoryOverheadCheck;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public PartitionsSpec getPartitionsSpec()
|
||||
{
|
||||
return partitionsSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexSpec getIndexSpec()
|
||||
{
|
||||
return indexSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexSpec getIndexSpecForIntermediatePersists()
|
||||
{
|
||||
return indexSpecForIntermediatePersists;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxPendingPersists()
|
||||
{
|
||||
return maxPendingPersists;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isReportParseExceptions()
|
||||
{
|
||||
return reportParseExceptions;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory()
|
||||
{
|
||||
return segmentWriteOutMediumFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxColumnsToMerge()
|
||||
{
|
||||
return maxColumnsToMerge;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getBasePersistDirectory()
|
||||
{
|
||||
return basePersistDirectory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Period getIntermediatePersistPeriod()
|
||||
{
|
||||
return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
TestIndexTuningConfig that = (TestIndexTuningConfig) o;
|
||||
return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) &&
|
||||
maxRowsInMemory == that.maxRowsInMemory &&
|
||||
maxBytesInMemory == that.maxBytesInMemory &&
|
||||
skipBytesInMemoryOverheadCheck == that.skipBytesInMemoryOverheadCheck &&
|
||||
maxColumnsToMerge == that.maxColumnsToMerge &&
|
||||
maxPendingPersists == that.maxPendingPersists &&
|
||||
reportParseExceptions == that.reportParseExceptions &&
|
||||
pushTimeout == that.pushTimeout &&
|
||||
Objects.equals(partitionsSpec, that.partitionsSpec) &&
|
||||
Objects.equals(indexSpec, that.indexSpec) &&
|
||||
Objects.equals(indexSpecForIntermediatePersists, that.indexSpecForIntermediatePersists) &&
|
||||
Objects.equals(basePersistDirectory, that.basePersistDirectory) &&
|
||||
Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(
|
||||
appendableIndexSpec,
|
||||
maxRowsInMemory,
|
||||
maxBytesInMemory,
|
||||
skipBytesInMemoryOverheadCheck,
|
||||
maxColumnsToMerge,
|
||||
partitionsSpec,
|
||||
indexSpec,
|
||||
indexSpecForIntermediatePersists,
|
||||
basePersistDirectory,
|
||||
maxPendingPersists,
|
||||
reportParseExceptions,
|
||||
pushTimeout,
|
||||
segmentWriteOutMediumFactory
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "IndexTuningConfig{" +
|
||||
"maxRowsInMemory=" + maxRowsInMemory +
|
||||
", maxBytesInMemory=" + maxBytesInMemory +
|
||||
", skipBytesInMemoryOverheadCheck=" + skipBytesInMemoryOverheadCheck +
|
||||
", maxColumnsToMerge=" + maxColumnsToMerge +
|
||||
", partitionsSpec=" + partitionsSpec +
|
||||
", indexSpec=" + indexSpec +
|
||||
", indexSpecForIntermediatePersists=" + indexSpecForIntermediatePersists +
|
||||
", basePersistDirectory=" + basePersistDirectory +
|
||||
", maxPendingPersists=" + maxPendingPersists +
|
||||
", reportParseExceptions=" + reportParseExceptions +
|
||||
", pushTimeout=" + pushTimeout +
|
||||
", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,627 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.indexing.common.task;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.JSONParseSpec;
|
||||
import org.apache.druid.data.input.impl.MapInputRowParser;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.FileUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.java.util.emitter.core.NoopEmitter;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.IndexMerger;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.incremental.AppendableIndexSpec;
|
||||
import org.apache.druid.segment.incremental.ParseExceptionHandler;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
import org.apache.druid.segment.incremental.SimpleRowIngestionMeters;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.TuningConfig;
|
||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
import org.apache.druid.segment.join.NoopJoinableFactory;
|
||||
import org.apache.druid.segment.loading.DataSegmentPusher;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
import org.apache.druid.segment.realtime.appenderator.Appenderator;
|
||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig;
|
||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl;
|
||||
import org.apache.druid.segment.realtime.appenderator.BatchAppenderator;
|
||||
import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
|
||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.server.security.AuthTestUtils;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.joda.time.Period;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
|
||||
|
||||
public class BatchAppenderatorsTest
|
||||
{
|
||||
@Test
|
||||
public void testLegacyOfflineAppenderator() throws Exception
|
||||
{
|
||||
try (final AppenderatorTester tester = new AppenderatorTester("OPEN_SEGMENTS")) {
|
||||
Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl);
|
||||
AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator;
|
||||
Assert.assertTrue(appenderator.isOpenSegments());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClosedSegmentsOfflineAppenderator() throws Exception
|
||||
{
|
||||
try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS")) {
|
||||
Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl);
|
||||
AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator;
|
||||
Assert.assertFalse(appenderator.isOpenSegments());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClosedSegmentsSinksOfflineAppenderator() throws Exception
|
||||
{
|
||||
try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS_SINKS")) {
|
||||
Assert.assertTrue(tester.appenderator instanceof BatchAppenderator);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private static class AppenderatorTester implements AutoCloseable
|
||||
{
|
||||
public static final String DATASOURCE = "foo";
|
||||
|
||||
private final DataSchema schema;
|
||||
private final AppenderatorConfig tuningConfig;
|
||||
private final FireDepartmentMetrics metrics;
|
||||
private final ObjectMapper objectMapper;
|
||||
private final Appenderator appenderator;
|
||||
private final ServiceEmitter emitter;
|
||||
|
||||
private final List<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),
|
||||
new DimensionsSpec(null, null, null),
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
),
|
||||
Map.class
|
||||
);
|
||||
|
||||
schema = new DataSchema(
|
||||
DATASOURCE,
|
||||
null,
|
||||
null,
|
||||
new AggregatorFactory[]{
|
||||
new CountAggregatorFactory("count"),
|
||||
new LongSumAggregatorFactory("met", "met")
|
||||
},
|
||||
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null),
|
||||
null,
|
||||
parserMap,
|
||||
objectMapper
|
||||
);
|
||||
|
||||
tuningConfig = new TestIndexTuningConfig(
|
||||
TuningConfig.DEFAULT_APPENDABLE_INDEX,
|
||||
maxRowsInMemory,
|
||||
maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes,
|
||||
skipBytesInMemoryOverheadCheck,
|
||||
new IndexSpec(),
|
||||
0,
|
||||
false,
|
||||
0L,
|
||||
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
|
||||
IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE,
|
||||
basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory
|
||||
);
|
||||
metrics = new FireDepartmentMetrics();
|
||||
|
||||
IndexIO indexIO = new IndexIO(
|
||||
objectMapper,
|
||||
() -> 0
|
||||
);
|
||||
IndexMergerV9 indexMerger = new IndexMergerV9(
|
||||
objectMapper,
|
||||
indexIO,
|
||||
OffHeapMemorySegmentWriteOutMediumFactory.instance()
|
||||
);
|
||||
|
||||
emitter = new ServiceEmitter(
|
||||
"test",
|
||||
"test",
|
||||
new NoopEmitter()
|
||||
);
|
||||
emitter.start();
|
||||
EmittingLogger.registerEmitter(emitter);
|
||||
DataSegmentPusher dataSegmentPusher = new DataSegmentPusher()
|
||||
{
|
||||
private boolean mustFail = true;
|
||||
|
||||
@Deprecated
|
||||
@Override
|
||||
public String getPathForHadoop(String dataSource)
|
||||
{
|
||||
return getPathForHadoop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPathForHadoop()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException
|
||||
{
|
||||
if (enablePushFailure && mustFail) {
|
||||
mustFail = false;
|
||||
throw new IOException("Push failure test");
|
||||
} else if (enablePushFailure) {
|
||||
mustFail = true;
|
||||
}
|
||||
pushedSegments.add(segment);
|
||||
return segment;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> makeLoadSpec(URI uri)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
|
||||
switch (batchMode) {
|
||||
case "OPEN_SEGMENTS":
|
||||
appenderator = BatchAppenderators.newAppenderator(
|
||||
"foo",
|
||||
new TestAppenderatorsManager(),
|
||||
metrics,
|
||||
new TestTaskToolbox(
|
||||
objectMapper,
|
||||
indexMerger,
|
||||
TaskConfig.BatchProcessingMode.OPEN_SEGMENTS
|
||||
),
|
||||
schema,
|
||||
tuningConfig,
|
||||
dataSegmentPusher,
|
||||
rowIngestionMeters,
|
||||
new ParseExceptionHandler(
|
||||
rowIngestionMeters,
|
||||
false,
|
||||
Integer.MAX_VALUE,
|
||||
0
|
||||
)
|
||||
);
|
||||
break;
|
||||
case "CLOSED_SEGMENTS":
|
||||
appenderator = BatchAppenderators.newAppenderator(
|
||||
"foo",
|
||||
new TestAppenderatorsManager(),
|
||||
metrics,
|
||||
new TestTaskToolbox(
|
||||
objectMapper,
|
||||
indexMerger,
|
||||
TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS
|
||||
),
|
||||
schema,
|
||||
tuningConfig,
|
||||
dataSegmentPusher,
|
||||
rowIngestionMeters,
|
||||
new ParseExceptionHandler(
|
||||
rowIngestionMeters,
|
||||
false,
|
||||
Integer.MAX_VALUE,
|
||||
0
|
||||
)
|
||||
);
|
||||
|
||||
break;
|
||||
case "CLOSED_SEGMENTS_SINKS":
|
||||
appenderator = BatchAppenderators.newAppenderator(
|
||||
"foo",
|
||||
new TestAppenderatorsManager(),
|
||||
metrics,
|
||||
new TestTaskToolbox(
|
||||
objectMapper,
|
||||
indexMerger,
|
||||
TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS_SINKS
|
||||
),
|
||||
schema,
|
||||
tuningConfig,
|
||||
dataSegmentPusher,
|
||||
rowIngestionMeters,
|
||||
new ParseExceptionHandler(
|
||||
rowIngestionMeters,
|
||||
false,
|
||||
Integer.MAX_VALUE,
|
||||
0
|
||||
)
|
||||
);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Unrecognized batchMode: " + batchMode);
|
||||
}
|
||||
}
|
||||
|
||||
private long getDefaultMaxBytesInMemory()
|
||||
{
|
||||
return (Runtime.getRuntime().totalMemory()) / 3;
|
||||
}
|
||||
|
||||
public DataSchema getSchema()
|
||||
{
|
||||
return schema;
|
||||
}
|
||||
|
||||
public AppenderatorConfig getTuningConfig()
|
||||
{
|
||||
return tuningConfig;
|
||||
}
|
||||
|
||||
public FireDepartmentMetrics getMetrics()
|
||||
{
|
||||
return metrics;
|
||||
}
|
||||
|
||||
public ObjectMapper getObjectMapper()
|
||||
{
|
||||
return objectMapper;
|
||||
}
|
||||
|
||||
public Appenderator getAppenderator()
|
||||
{
|
||||
return appenderator;
|
||||
}
|
||||
|
||||
public List<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;
|
||||
|
||||
public TestIndexTuningConfig(
|
||||
AppendableIndexSpec appendableIndexSpec,
|
||||
Integer maxRowsInMemory,
|
||||
Long maxBytesInMemory,
|
||||
Boolean skipBytesInMemoryOverheadCheck,
|
||||
IndexSpec indexSpec,
|
||||
Integer maxPendingPersists,
|
||||
Boolean reportParseExceptions,
|
||||
Long pushTimeout,
|
||||
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
|
||||
Integer maxColumnsToMerge,
|
||||
File basePersistDirectory
|
||||
)
|
||||
{
|
||||
this.appendableIndexSpec = appendableIndexSpec;
|
||||
this.maxRowsInMemory = maxRowsInMemory;
|
||||
this.maxBytesInMemory = maxBytesInMemory;
|
||||
this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck;
|
||||
this.indexSpec = indexSpec;
|
||||
this.maxPendingPersists = maxPendingPersists;
|
||||
this.reportParseExceptions = reportParseExceptions;
|
||||
this.pushTimeout = pushTimeout;
|
||||
this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
|
||||
this.maxColumnsToMerge = maxColumnsToMerge;
|
||||
this.basePersistDirectory = basePersistDirectory;
|
||||
|
||||
this.partitionsSpec = null;
|
||||
this.indexSpecForIntermediatePersists = this.indexSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TestIndexTuningConfig withBasePersistDirectory(File dir)
|
||||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AppendableIndexSpec getAppendableIndexSpec()
|
||||
{
|
||||
return appendableIndexSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxRowsInMemory()
|
||||
{
|
||||
return maxRowsInMemory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMaxBytesInMemory()
|
||||
{
|
||||
return maxBytesInMemory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isSkipBytesInMemoryOverheadCheck()
|
||||
{
|
||||
return skipBytesInMemoryOverheadCheck;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public PartitionsSpec getPartitionsSpec()
|
||||
{
|
||||
return partitionsSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexSpec getIndexSpec()
|
||||
{
|
||||
return indexSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexSpec getIndexSpecForIntermediatePersists()
|
||||
{
|
||||
return indexSpecForIntermediatePersists;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxPendingPersists()
|
||||
{
|
||||
return maxPendingPersists;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isReportParseExceptions()
|
||||
{
|
||||
return reportParseExceptions;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory()
|
||||
{
|
||||
return segmentWriteOutMediumFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxColumnsToMerge()
|
||||
{
|
||||
return maxColumnsToMerge;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getBasePersistDirectory()
|
||||
{
|
||||
return basePersistDirectory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Period getIntermediatePersistPeriod()
|
||||
{
|
||||
return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
TestIndexTuningConfig that = (TestIndexTuningConfig) o;
|
||||
return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) &&
|
||||
maxRowsInMemory == that.maxRowsInMemory &&
|
||||
maxBytesInMemory == that.maxBytesInMemory &&
|
||||
skipBytesInMemoryOverheadCheck == that.skipBytesInMemoryOverheadCheck &&
|
||||
maxColumnsToMerge == that.maxColumnsToMerge &&
|
||||
maxPendingPersists == that.maxPendingPersists &&
|
||||
reportParseExceptions == that.reportParseExceptions &&
|
||||
pushTimeout == that.pushTimeout &&
|
||||
Objects.equals(partitionsSpec, that.partitionsSpec) &&
|
||||
Objects.equals(indexSpec, that.indexSpec) &&
|
||||
Objects.equals(indexSpecForIntermediatePersists, that.indexSpecForIntermediatePersists) &&
|
||||
Objects.equals(basePersistDirectory, that.basePersistDirectory) &&
|
||||
Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(
|
||||
appendableIndexSpec,
|
||||
maxRowsInMemory,
|
||||
maxBytesInMemory,
|
||||
skipBytesInMemoryOverheadCheck,
|
||||
maxColumnsToMerge,
|
||||
partitionsSpec,
|
||||
indexSpec,
|
||||
indexSpecForIntermediatePersists,
|
||||
basePersistDirectory,
|
||||
maxPendingPersists,
|
||||
reportParseExceptions,
|
||||
pushTimeout,
|
||||
segmentWriteOutMediumFactory
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "IndexTuningConfig{" +
|
||||
"maxRowsInMemory=" + maxRowsInMemory +
|
||||
", maxBytesInMemory=" + maxBytesInMemory +
|
||||
", skipBytesInMemoryOverheadCheck=" + skipBytesInMemoryOverheadCheck +
|
||||
", maxColumnsToMerge=" + maxColumnsToMerge +
|
||||
", partitionsSpec=" + partitionsSpec +
|
||||
", indexSpec=" + indexSpec +
|
||||
", indexSpecForIntermediatePersists=" + indexSpecForIntermediatePersists +
|
||||
", basePersistDirectory=" + basePersistDirectory +
|
||||
", maxPendingPersists=" + maxPendingPersists +
|
||||
", reportParseExceptions=" + reportParseExceptions +
|
||||
", pushTimeout=" + pushTimeout +
|
||||
", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private static class TestTaskToolbox extends TaskToolbox
|
||||
{
|
||||
private final Map<DataSegment, File> segmentFileMap;
|
||||
|
||||
TestTaskToolbox(ObjectMapper mapper, IndexMergerV9 indexMergerV9, TaskConfig.BatchProcessingMode mode)
|
||||
{
|
||||
super(
|
||||
new TaskConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false,
|
||||
mode.name()
|
||||
),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
NoopJoinableFactory.INSTANCE,
|
||||
null,
|
||||
null,
|
||||
mapper,
|
||||
null,
|
||||
new IndexIO(
|
||||
new ObjectMapper(),
|
||||
() -> 0
|
||||
),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
indexMergerV9,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new NoopTestTaskReportFileWriter(),
|
||||
null,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
new NoopChatHandlerProvider(),
|
||||
null,
|
||||
new TestAppenderatorsManager(),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
this.segmentFileMap = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -1297,7 +1297,20 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
|||
);
|
||||
|
||||
return new TaskToolbox(
|
||||
new TaskConfig(null, null, null, null, null, false, null, null, null, false, false),
|
||||
new TaskConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
),
|
||||
null,
|
||||
createActionClient(task),
|
||||
null,
|
||||
|
|
|
@ -1747,7 +1747,20 @@ public class CompactionTaskTest
|
|||
)
|
||||
{
|
||||
super(
|
||||
new TaskConfig(null, null, null, null, null, false, null, null, null, false, false),
|
||||
new TaskConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
),
|
||||
null,
|
||||
taskActionClient,
|
||||
null,
|
||||
|
|
|
@ -118,7 +118,8 @@ public class HadoopTaskTest
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
)).once();
|
||||
EasyMock.replay(toolbox);
|
||||
|
||||
|
|
|
@ -314,7 +314,8 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest
|
|||
);
|
||||
|
||||
final TaskToolbox box = new TaskToolbox(
|
||||
new TaskConfig(null, null, null, null, null, false, null, null, null, false, false),
|
||||
new TaskConfig(null, null, null, null, null, false, null, null, null, false, false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()),
|
||||
new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false),
|
||||
taskActionClient,
|
||||
null,
|
||||
|
|
|
@ -899,7 +899,8 @@ public class RealtimeIndexTaskTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
);
|
||||
final TaskLockbox taskLockbox = new TaskLockbox(taskStorage, mdc);
|
||||
try {
|
||||
|
|
|
@ -93,6 +93,62 @@ public class TestAppenderatorsManager implements AppenderatorsManager
|
|||
return realtimeAppenderator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Appenderator createOpenSegmentsOfflineAppenderatorForTask(
|
||||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
return Appenderators.createOpenSegmentsOffline(
|
||||
taskId,
|
||||
schema,
|
||||
config,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Appenderator createClosedSegmentsOfflineAppenderatorForTask(
|
||||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
return Appenderators.createClosedSegmentsOffline(
|
||||
taskId,
|
||||
schema,
|
||||
config,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Appenderator createOfflineAppenderatorForTask(
|
||||
String taskId,
|
||||
|
@ -104,8 +160,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager
|
|||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler,
|
||||
boolean useLegacyBatchProcessing
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
return Appenderators.createOffline(
|
||||
|
@ -118,8 +173,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager
|
|||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler,
|
||||
useLegacyBatchProcessing
|
||||
parseExceptionHandler
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -244,7 +244,8 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
|
|||
null,
|
||||
ImmutableList.of(new StorageLocationConfig(temporaryFolder.newFolder(), null, null)),
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
),
|
||||
null
|
||||
);
|
||||
|
@ -600,7 +601,20 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
|
|||
|
||||
public void prepareObjectMapper(ObjectMapper objectMapper, IndexIO indexIO)
|
||||
{
|
||||
final TaskConfig taskConfig = new TaskConfig(null, null, null, null, null, false, null, null, null, false, false);
|
||||
final TaskConfig taskConfig = new TaskConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
);
|
||||
|
||||
objectMapper.setInjectableValues(
|
||||
new InjectableValues.Std()
|
||||
|
@ -635,7 +649,20 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
|
|||
protected TaskToolbox createTaskToolbox(Task task, TaskActionClient actionClient) throws IOException
|
||||
{
|
||||
return new TaskToolbox(
|
||||
new TaskConfig(null, null, null, null, null, false, null, null, null, false, false),
|
||||
new TaskConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
),
|
||||
new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false),
|
||||
actionClient,
|
||||
null,
|
||||
|
|
|
@ -202,7 +202,8 @@ public class ForkingTaskRunnerTest
|
|||
new Period("PT10S"),
|
||||
ImmutableList.of(),
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
),
|
||||
new WorkerConfig(),
|
||||
new Properties(),
|
||||
|
@ -255,7 +256,8 @@ public class ForkingTaskRunnerTest
|
|||
new Period("PT10S"),
|
||||
ImmutableList.of(),
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
),
|
||||
new WorkerConfig(),
|
||||
new Properties(),
|
||||
|
@ -312,7 +314,8 @@ public class ForkingTaskRunnerTest
|
|||
new Period("PT10S"),
|
||||
ImmutableList.of(),
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
),
|
||||
new WorkerConfig(),
|
||||
new Properties(),
|
||||
|
|
|
@ -96,7 +96,8 @@ public class SingleTaskBackgroundRunnerTest
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
);
|
||||
final ServiceEmitter emitter = new NoopServiceEmitter();
|
||||
EmittingLogger.registerEmitter(emitter);
|
||||
|
|
|
@ -602,7 +602,8 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
|||
new TaskAuditLogConfig(true)
|
||||
);
|
||||
File tmpDir = temporaryFolder.newFolder();
|
||||
taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, false, null, null, null, false, false);
|
||||
taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, false, null, null, null, false, false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name());
|
||||
|
||||
return new TaskToolboxFactory(
|
||||
taskConfig,
|
||||
|
|
|
@ -61,7 +61,8 @@ public class ThreadingTaskRunnerTest
|
|||
new Period("PT10S"),
|
||||
ImmutableList.of(),
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
),
|
||||
new WorkerConfig(),
|
||||
new NoopTaskLogs(),
|
||||
|
|
|
@ -93,7 +93,8 @@ public class WorkerTaskManagerTest
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
);
|
||||
TaskActionClientFactory taskActionClientFactory = EasyMock.createNiceMock(TaskActionClientFactory.class);
|
||||
TaskActionClient taskActionClient = EasyMock.createNiceMock(TaskActionClient.class);
|
||||
|
|
|
@ -164,7 +164,8 @@ public class WorkerTaskMonitorTest
|
|||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
);
|
||||
TaskActionClientFactory taskActionClientFactory = EasyMock.createNiceMock(TaskActionClientFactory.class);
|
||||
TaskActionClient taskActionClient = EasyMock.createNiceMock(TaskActionClient.class);
|
||||
|
|
|
@ -94,7 +94,8 @@ public class LocalIntermediaryDataManagerAutoCleanupTest
|
|||
null,
|
||||
ImmutableList.of(new StorageLocationConfig(tempDir.newFolder(), null, null)),
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
);
|
||||
final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient()
|
||||
{
|
||||
|
|
|
@ -79,7 +79,8 @@ public class LocalIntermediaryDataManagerManualAddAndDeleteTest
|
|||
null,
|
||||
ImmutableList.of(new StorageLocationConfig(intermediarySegmentsLocation, 1200L, null)),
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
);
|
||||
final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient();
|
||||
intermediaryDataManager = new LocalIntermediaryDataManager(workerConfig, taskConfig, indexingServiceClient);
|
||||
|
|
|
@ -109,7 +109,8 @@ public class ShuffleDataSegmentPusherTest
|
|||
null,
|
||||
ImmutableList.of(new StorageLocationConfig(temporaryFolder.newFolder(), null, null)),
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
);
|
||||
final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient();
|
||||
if (LOCAL.equals(intermediateDataStore)) {
|
||||
|
|
|
@ -99,7 +99,8 @@ public class ShuffleResourceTest
|
|||
null,
|
||||
ImmutableList.of(new StorageLocationConfig(tempDir.newFolder(), null, null)),
|
||||
false,
|
||||
false
|
||||
false,
|
||||
TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()
|
||||
);
|
||||
final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient()
|
||||
{
|
||||
|
|
|
@ -46,7 +46,7 @@ import java.util.List;
|
|||
*<p>
|
||||
* Important note: For historical reasons there was a single implementation for this interface ({@code AppenderatorImpl})
|
||||
* but that since has been split into two classes: {@link StreamAppenderator} and {@link BatchAppenderator}. With this change
|
||||
* all the query support & concurrency has been removed from the {@code BatchAppenderator} therefore this class no longer
|
||||
* all the query support & concurrency has been removed/changed in {@code BatchAppenderator} therefore this class no longer
|
||||
* makes sense to have as an {@code Appenderator}. In the future we may want to refactor away the {@code Appenderator}
|
||||
* interface from {@code BatchAppenderator}.
|
||||
*/
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -102,30 +102,12 @@ public class Appenderators
|
|||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler,
|
||||
boolean useLegacyBatchProcessing
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
if (useLegacyBatchProcessing) {
|
||||
// fallback to code known to be working, this is just a fallback option in case new
|
||||
// batch appenderator has some early bugs but we will remove this fallback as soon as
|
||||
// we determine that batch appenderator code is stable
|
||||
return new StreamAppenderator(
|
||||
id,
|
||||
schema,
|
||||
config,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
new NoopDataSegmentAnnouncer(),
|
||||
null,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
null,
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler
|
||||
);
|
||||
}
|
||||
// Use newest, slated to be the permanent batch appenderator but for now keeping it as a non-default
|
||||
// option due to risk mitigation...will become default and the two other appenderators eliminated when
|
||||
// stability is proven...
|
||||
return new BatchAppenderator(
|
||||
id,
|
||||
schema,
|
||||
|
@ -139,4 +121,69 @@ public class Appenderators
|
|||
parseExceptionHandler
|
||||
);
|
||||
}
|
||||
|
||||
public static Appenderator createOpenSegmentsOffline(
|
||||
String id,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
// fallback to original code known to be working, this is just a fallback option in case new
|
||||
// batch appenderator has some early bugs but we will remove this fallback as soon as
|
||||
// we determine that batch appenderator code is stable
|
||||
return new AppenderatorImpl(
|
||||
id,
|
||||
schema,
|
||||
config,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
new NoopDataSegmentAnnouncer(),
|
||||
null,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
null,
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler,
|
||||
true
|
||||
);
|
||||
}
|
||||
|
||||
public static Appenderator createClosedSegmentsOffline(
|
||||
String id,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
return new AppenderatorImpl(
|
||||
id,
|
||||
schema,
|
||||
config,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
new NoopDataSegmentAnnouncer(),
|
||||
null,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
null,
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler,
|
||||
false
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -87,6 +87,32 @@ public interface AppenderatorsManager
|
|||
/**
|
||||
* Creates an Appenderator suited for batch ingestion.
|
||||
*/
|
||||
Appenderator createOpenSegmentsOfflineAppenderatorForTask(
|
||||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
);
|
||||
|
||||
Appenderator createClosedSegmentsOfflineAppenderatorForTask(
|
||||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
);
|
||||
|
||||
Appenderator createOfflineAppenderatorForTask(
|
||||
String taskId,
|
||||
DataSchema schema,
|
||||
|
@ -97,8 +123,7 @@ public interface AppenderatorsManager
|
|||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler,
|
||||
boolean useLegacyBatchProcessing
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
);
|
||||
|
||||
/**
|
||||
|
|
|
@ -21,23 +21,30 @@ package org.apache.druid.segment.realtime.appenderator;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Stopwatch;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.common.util.concurrent.FutureCallback;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import org.apache.druid.data.input.Committer;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.java.util.common.FileUtils;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
import org.apache.druid.java.util.common.RetryUtils;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.query.Query;
|
||||
|
@ -61,7 +68,6 @@ import org.apache.druid.timeline.DataSegment;
|
|||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import javax.annotation.concurrent.NotThreadSafe;
|
||||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
@ -75,6 +81,7 @@ import java.util.HashMap;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.stream.Collectors;
|
||||
|
@ -87,12 +94,7 @@ import java.util.stream.Collectors;
|
|||
* hydrant creation phase (append) of this class or in the merge hydrants phase. Therefore, a new class,
|
||||
* {@code BatchAppenderator}, this class, was created to specialize in batch ingestion and the old class
|
||||
* for stream ingestion was renamed to {@link StreamAppenderator}.
|
||||
* <p>
|
||||
* This class is not thread safe!.
|
||||
* It is important to realize that this class is completely synchronous despite the {@link Appenderator}
|
||||
* interface suggesting otherwise. The concurrency was not required so it has been completely removed.
|
||||
*/
|
||||
@NotThreadSafe
|
||||
public class BatchAppenderator implements Appenderator
|
||||
{
|
||||
public static final int ROUGH_OVERHEAD_PER_SINK = 5000;
|
||||
|
@ -110,15 +112,30 @@ public class BatchAppenderator implements Appenderator
|
|||
private final ObjectMapper objectMapper;
|
||||
private final IndexIO indexIO;
|
||||
private final IndexMerger indexMerger;
|
||||
private final Map<SegmentIdWithShardSpec, Sink> sinks = new HashMap<>();
|
||||
private final long maxBytesTuningConfig;
|
||||
private final boolean skipBytesInMemoryOverheadCheck;
|
||||
|
||||
private volatile ListeningExecutorService persistExecutor = null;
|
||||
private volatile ListeningExecutorService pushExecutor = null;
|
||||
private final int maxPendingPersists;
|
||||
private static final int PERSIST_WARN_DELAY = 1000;
|
||||
private volatile Throwable persistError;
|
||||
|
||||
|
||||
/**
|
||||
* The following map used to be accessed concurrently but not anymore since it is fully copied
|
||||
* then initalized just before scheduling the persit callable in the {@link #persistAll(Committer)}
|
||||
* method, so no longer need to use a syncronized map.
|
||||
*/
|
||||
private final Map<SegmentIdWithShardSpec, Sink> sinks = new HashMap<>();
|
||||
/**
|
||||
* The following sinks metadata map and associated class are the way to retain metadata now that sinks
|
||||
* are being completely removed from memory after each incremental persist.
|
||||
* are being completely removed from memory after each incremental persist. This map needs to be concurrent
|
||||
* since it is mutated in various methods potentially in different threads.
|
||||
* For example mutated in {@link #add} when adding rows
|
||||
* and accessed/mutated in {@link #persistHydrant} during persists.
|
||||
*/
|
||||
private final Map<SegmentIdWithShardSpec, SinkMetadata> sinksMetadata = new HashMap<>();
|
||||
private final ConcurrentHashMap<SegmentIdWithShardSpec, SinkMetadata> sinksMetadata = new ConcurrentHashMap<>();
|
||||
|
||||
// This variable updated in add(), persist(), and drop()
|
||||
private int rowsCurrentlyInMemory = 0;
|
||||
|
@ -158,6 +175,7 @@ public class BatchAppenderator implements Appenderator
|
|||
|
||||
maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault();
|
||||
skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck();
|
||||
maxPendingPersists = tuningConfig.getMaxPendingPersists();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -177,9 +195,53 @@ public class BatchAppenderator implements Appenderator
|
|||
{
|
||||
tuningConfig.getBasePersistDirectory().mkdirs();
|
||||
lockBasePersistDirectory();
|
||||
initializeExecutors();
|
||||
return null;
|
||||
}
|
||||
|
||||
private void throwPersistErrorIfExists()
|
||||
{
|
||||
if (persistError != null) {
|
||||
throw new RE(persistError, "Error while persisting");
|
||||
}
|
||||
}
|
||||
|
||||
private void initializeExecutors()
|
||||
{
|
||||
log.debug("There will be up to[%d] pending persists", maxPendingPersists);
|
||||
|
||||
if (persistExecutor == null) {
|
||||
// use a blocking single threaded executor to throttle the firehose when write to disk is slow
|
||||
persistExecutor = MoreExecutors.listeningDecorator(
|
||||
Execs.newBlockingSingleThreaded(
|
||||
"[" + StringUtils.encodeForFormat(myId) + "]-batch-appenderator-persist",
|
||||
maxPendingPersists
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
if (pushExecutor == null) {
|
||||
// use a blocking single threaded executor to throttle the firehose when write to disk is slow
|
||||
pushExecutor = MoreExecutors.listeningDecorator(
|
||||
Execs.newBlockingSingleThreaded(
|
||||
"[" + StringUtils.encodeForFormat(myId) + "]-batch-appenderator-push",
|
||||
1
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private void shutdownExecutors()
|
||||
{
|
||||
if (persistExecutor != null) {
|
||||
persistExecutor.shutdownNow();
|
||||
}
|
||||
if (pushExecutor != null) {
|
||||
pushExecutor.shutdownNow();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public AppenderatorAddResult add(
|
||||
final SegmentIdWithShardSpec identifier,
|
||||
|
@ -189,6 +251,8 @@ public class BatchAppenderator implements Appenderator
|
|||
) throws IndexSizeExceededException, SegmentNotWritableException
|
||||
{
|
||||
|
||||
throwPersistErrorIfExists();
|
||||
|
||||
Preconditions.checkArgument(
|
||||
committerSupplier == null,
|
||||
"Batch appenderator does not need a committer!"
|
||||
|
@ -240,7 +304,7 @@ public class BatchAppenderator implements Appenderator
|
|||
|
||||
final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd;
|
||||
rowsCurrentlyInMemory += numAddedRows;
|
||||
bytesCurrentlyInMemory += bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd;
|
||||
bytesCurrentlyInMemory += (bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd);
|
||||
totalRows += numAddedRows;
|
||||
sinksMetadata.computeIfAbsent(identifier, unused -> new SinkMetadata()).addRows(numAddedRows);
|
||||
|
||||
|
@ -267,6 +331,7 @@ public class BatchAppenderator implements Appenderator
|
|||
maxBytesTuningConfig
|
||||
));
|
||||
}
|
||||
|
||||
if (persist) {
|
||||
// persistAll clears rowsCurrentlyInMemory, no need to update it.
|
||||
log.info("Incremental persist to disk because %s.", String.join(",", persistReasons));
|
||||
|
@ -277,7 +342,7 @@ public class BatchAppenderator implements Appenderator
|
|||
if (sinkEntry != null) {
|
||||
bytesToBePersisted += sinkEntry.getBytesInMemory();
|
||||
if (sinkEntry.swappable()) {
|
||||
// Code for batch no longer memory maps hydrants but they still take memory...
|
||||
// Code for batch no longer memory maps hydrants, but they still take memory...
|
||||
int memoryStillInUse = calculateMemoryUsedByHydrant();
|
||||
bytesCurrentlyInMemory += memoryStillInUse;
|
||||
}
|
||||
|
@ -315,8 +380,23 @@ public class BatchAppenderator implements Appenderator
|
|||
throw new RuntimeException(errorMessage);
|
||||
}
|
||||
|
||||
persistAllAndRemoveSinks();
|
||||
Futures.addCallback(
|
||||
persistAll(null),
|
||||
new FutureCallback<Object>()
|
||||
{
|
||||
@Override
|
||||
public void onSuccess(@Nullable Object result)
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable t)
|
||||
{
|
||||
persistError = t;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
return new AppenderatorAddResult(identifier, sinksMetadata.get(identifier).numRowsInSegment, false);
|
||||
}
|
||||
|
@ -388,7 +468,6 @@ public class BatchAppenderator implements Appenderator
|
|||
null
|
||||
);
|
||||
bytesCurrentlyInMemory += calculateSinkMemoryInUsed();
|
||||
|
||||
sinks.put(identifier, retVal);
|
||||
metrics.setSinkCount(sinks.size());
|
||||
}
|
||||
|
@ -411,20 +490,22 @@ public class BatchAppenderator implements Appenderator
|
|||
@Override
|
||||
public void clear()
|
||||
{
|
||||
clear(true);
|
||||
throwPersistErrorIfExists();
|
||||
clear(sinks, true);
|
||||
}
|
||||
|
||||
private void clear(boolean removeOnDiskData)
|
||||
private void clear(Map<SegmentIdWithShardSpec, Sink> sinksToClear, boolean removeOnDiskData)
|
||||
{
|
||||
// Drop commit metadata, then abandon all segments.
|
||||
log.info("Clearing all[%d] sinks & their hydrants, removing data on disk: [%s]", sinks.size(), removeOnDiskData);
|
||||
log.info("Clearing all[%d] sinks & their hydrants, removing data on disk: [%s]",
|
||||
sinksToClear.size(), removeOnDiskData);
|
||||
// Drop everything.
|
||||
Iterator<Map.Entry<SegmentIdWithShardSpec, Sink>> sinksIterator = sinks.entrySet().iterator();
|
||||
Iterator<Map.Entry<SegmentIdWithShardSpec, Sink>> sinksIterator = sinksToClear.entrySet().iterator();
|
||||
sinksIterator.forEachRemaining(entry -> {
|
||||
clearSinkMetadata(entry.getKey(), entry.getValue(), removeOnDiskData);
|
||||
clearSinkMemoryCountersAndDiskStoredData(entry.getKey(), entry.getValue(), removeOnDiskData);
|
||||
sinksIterator.remove();
|
||||
});
|
||||
metrics.setSinkCount(sinks.size());
|
||||
metrics.setSinkCount(sinksToClear.size());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -442,7 +523,7 @@ public class BatchAppenderator implements Appenderator
|
|||
totalRows = Math.max(totalRowsAfter, 0);
|
||||
}
|
||||
if (sink != null) {
|
||||
clearSinkMetadata(identifier, sink, true);
|
||||
clearSinkMemoryCountersAndDiskStoredData(identifier, sink, true);
|
||||
if (sinks.remove(identifier) == null) {
|
||||
log.warn("Sink for identifier[%s] not found, skipping", identifier);
|
||||
}
|
||||
|
@ -453,100 +534,124 @@ public class BatchAppenderator implements Appenderator
|
|||
@Override
|
||||
public ListenableFuture<Object> persistAll(@Nullable final Committer committer)
|
||||
{
|
||||
|
||||
throwPersistErrorIfExists();
|
||||
|
||||
if (committer != null) {
|
||||
throw new ISE("committer must be null for BatchAppenderator");
|
||||
}
|
||||
persistAllAndRemoveSinks();
|
||||
return Futures.immediateFuture(null);
|
||||
// Get ready to persist all sinks:
|
||||
final Map<SegmentIdWithShardSpec, Sink> sinksToPersist = swapSinks();
|
||||
|
||||
final Stopwatch runExecStopwatch = Stopwatch.createStarted();
|
||||
ListenableFuture<Object> future = persistExecutor.submit(
|
||||
() -> {
|
||||
log.info("Spawning intermediate persist");
|
||||
|
||||
// figure out hydrants (indices) to persist:
|
||||
final List<Pair<FireHydrant, SegmentIdWithShardSpec>> indexesToPersist = new ArrayList<>();
|
||||
int numPersistedRows = 0;
|
||||
long bytesPersisted = 0;
|
||||
int totalHydrantsCount = 0;
|
||||
final long totalSinks = sinksToPersist.size();
|
||||
for (Map.Entry<SegmentIdWithShardSpec, Sink> entry : sinksToPersist.entrySet()) {
|
||||
final SegmentIdWithShardSpec identifier = entry.getKey();
|
||||
final Sink sink = entry.getValue();
|
||||
if (sink == null) {
|
||||
throw new ISE("No sink for identifier: %s", identifier);
|
||||
}
|
||||
|
||||
final List<FireHydrant> hydrants = Lists.newArrayList(sink);
|
||||
// Since everytime we persist we also get rid of the in-memory references to sink & hydrants
|
||||
// the invariant of exactly one, always swappable, sink with exactly one unpersisted hydrant must hold
|
||||
int totalHydrantsForSink = hydrants.size();
|
||||
if (totalHydrantsForSink != 1) {
|
||||
throw new ISE("There should be only one hydrant for identifier[%s] but there are[%s]",
|
||||
identifier, totalHydrantsForSink
|
||||
);
|
||||
}
|
||||
totalHydrantsCount++;
|
||||
numPersistedRows += sink.getNumRowsInMemory();
|
||||
bytesPersisted += sink.getBytesInMemory();
|
||||
|
||||
if (!sink.swappable()) {
|
||||
throw new ISE("Sink is not swappable![%s]", identifier);
|
||||
}
|
||||
indexesToPersist.add(Pair.of(sink.swap(), identifier));
|
||||
|
||||
}
|
||||
|
||||
if (indexesToPersist.isEmpty()) {
|
||||
log.info("No indexes will be persisted");
|
||||
}
|
||||
final Stopwatch persistStopwatch = Stopwatch.createStarted();
|
||||
try {
|
||||
for (Pair<FireHydrant, SegmentIdWithShardSpec> pair : indexesToPersist) {
|
||||
metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs));
|
||||
}
|
||||
|
||||
log.info(
|
||||
"Persisted in-memory data for segments: %s",
|
||||
indexesToPersist.stream()
|
||||
.filter(itp -> itp.rhs != null)
|
||||
.map(itp -> itp.rhs.asSegmentId().toString())
|
||||
.distinct()
|
||||
.collect(Collectors.joining(", "))
|
||||
);
|
||||
log.info(
|
||||
"Persisted stats: processed rows: [%d], persisted rows[%d], persisted sinks: [%d], persisted fireHydrants (across sinks): [%d]",
|
||||
rowIngestionMeters.getProcessed(),
|
||||
numPersistedRows,
|
||||
totalSinks,
|
||||
totalHydrantsCount
|
||||
);
|
||||
|
||||
// note that we do not need to reset sinks metadata since we did it at the start...
|
||||
|
||||
}
|
||||
catch (Exception e) {
|
||||
metrics.incrementFailedPersists();
|
||||
throw e;
|
||||
}
|
||||
finally {
|
||||
metrics.incrementNumPersists();
|
||||
long persistMillis = persistStopwatch.elapsed(TimeUnit.MILLISECONDS);
|
||||
metrics.incrementPersistTimeMillis(persistMillis);
|
||||
persistStopwatch.stop();
|
||||
// make sure no push can start while persisting:
|
||||
log.info("Persisted rows[%,d] and bytes[%,d] and removed all sinks & hydrants from memory in[%d] millis",
|
||||
numPersistedRows, bytesPersisted, persistMillis
|
||||
);
|
||||
log.info("Persist is done.");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
);
|
||||
|
||||
final long startDelay = runExecStopwatch.elapsed(TimeUnit.MILLISECONDS);
|
||||
metrics.incrementPersistBackPressureMillis(startDelay);
|
||||
if (startDelay > PERSIST_WARN_DELAY) {
|
||||
log.warn("Ingestion was throttled for [%,d] millis because persists were pending.", startDelay);
|
||||
}
|
||||
runExecStopwatch.stop();
|
||||
return future;
|
||||
}
|
||||
|
||||
/**
|
||||
* Persist all sinks & their hydrants, keep their metadata, and then remove them completely from
|
||||
* memory (to be resurrected right before merge & push)
|
||||
* All sinks will be persisted so do a shallow copy of the Sinks map, reset
|
||||
* the map and metadata (i.e. memory consumption counters) so that ingestion can go on
|
||||
* @return The map of sinks to persist, this map will be garbage collected after
|
||||
* persist is complete since we will not be keeping a reference to it...
|
||||
*/
|
||||
private void persistAllAndRemoveSinks()
|
||||
Map<SegmentIdWithShardSpec, Sink> swapSinks()
|
||||
{
|
||||
|
||||
final List<Pair<FireHydrant, SegmentIdWithShardSpec>> indexesToPersist = new ArrayList<>();
|
||||
int numPersistedRows = 0;
|
||||
long bytesPersisted = 0L;
|
||||
int totalHydrantsCount = 0;
|
||||
final long totalSinks = sinks.size();
|
||||
for (Map.Entry<SegmentIdWithShardSpec, Sink> entry : sinks.entrySet()) {
|
||||
final SegmentIdWithShardSpec identifier = entry.getKey();
|
||||
final Sink sink = entry.getValue();
|
||||
if (sink == null) {
|
||||
throw new ISE("No sink for identifier: %s", identifier);
|
||||
}
|
||||
|
||||
final List<FireHydrant> hydrants = Lists.newArrayList(sink);
|
||||
// Since everytime we persist we also get rid of the in-memory references to sinks & hydrants
|
||||
// the invariant of exactly one, always swappable, sink with exactly one unpersisted hydrant must hold
|
||||
int totalHydrantsForSink = hydrants.size();
|
||||
if (totalHydrantsForSink != 1) {
|
||||
throw new ISE("There should be only one hydrant for identifier[%s] but there are[%s]",
|
||||
identifier, totalHydrantsForSink
|
||||
);
|
||||
}
|
||||
totalHydrantsCount += 1;
|
||||
numPersistedRows += sink.getNumRowsInMemory();
|
||||
bytesPersisted += sink.getBytesInMemory();
|
||||
|
||||
if (!sink.swappable()) {
|
||||
throw new ISE("Sink is not swappable![%s]", identifier);
|
||||
}
|
||||
indexesToPersist.add(Pair.of(sink.swap(), identifier));
|
||||
|
||||
}
|
||||
|
||||
if (indexesToPersist.isEmpty()) {
|
||||
log.info("No indexes will be persisted");
|
||||
}
|
||||
final Stopwatch persistStopwatch = Stopwatch.createStarted();
|
||||
try {
|
||||
for (Pair<FireHydrant, SegmentIdWithShardSpec> pair : indexesToPersist) {
|
||||
metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs));
|
||||
}
|
||||
|
||||
log.info(
|
||||
"Persisted in-memory data for segments: %s",
|
||||
indexesToPersist.stream()
|
||||
.filter(itp -> itp.rhs != null)
|
||||
.map(itp -> itp.rhs.asSegmentId().toString())
|
||||
.distinct()
|
||||
.collect(Collectors.joining(", "))
|
||||
);
|
||||
log.info(
|
||||
"Persisted stats: processed rows: [%d], persisted rows[%d], persisted sinks: [%d], persisted fireHydrants (across sinks): [%d]",
|
||||
rowIngestionMeters.getProcessed(),
|
||||
numPersistedRows,
|
||||
totalSinks,
|
||||
totalHydrantsCount
|
||||
);
|
||||
|
||||
}
|
||||
catch (Exception e) {
|
||||
metrics.incrementFailedPersists();
|
||||
throw e;
|
||||
}
|
||||
finally {
|
||||
metrics.incrementNumPersists();
|
||||
metrics.incrementPersistTimeMillis(persistStopwatch.elapsed(TimeUnit.MILLISECONDS));
|
||||
persistStopwatch.stop();
|
||||
}
|
||||
|
||||
// NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes.
|
||||
rowsCurrentlyInMemory -= numPersistedRows;
|
||||
bytesCurrentlyInMemory -= bytesPersisted;
|
||||
|
||||
// remove all sinks after persisting:
|
||||
clear(false);
|
||||
|
||||
log.info("Persisted rows[%,d] and bytes[%,d] and removed all sinks & hydrants from memory",
|
||||
numPersistedRows, bytesPersisted);
|
||||
|
||||
Map<SegmentIdWithShardSpec, Sink> retVal = ImmutableMap.copyOf(sinks);
|
||||
sinks.clear();
|
||||
resetSinkMetadata();
|
||||
return retVal;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public ListenableFuture<SegmentsAndCommitMetadata> push(
|
||||
final Collection<SegmentIdWithShardSpec> identifiers,
|
||||
|
@ -563,53 +668,56 @@ public class BatchAppenderator implements Appenderator
|
|||
throw new ISE("Batch ingestion does not require uniquePath");
|
||||
}
|
||||
|
||||
// Any sinks not persisted so far need to be persisted before push:
|
||||
persistAllAndRemoveSinks();
|
||||
|
||||
log.info("Preparing to push...");
|
||||
|
||||
// Traverse identifiers, load their sink, and push it:
|
||||
int totalHydrantsMerged = 0;
|
||||
final List<DataSegment> dataSegments = new ArrayList<>();
|
||||
for (SegmentIdWithShardSpec identifier : identifiers) {
|
||||
SinkMetadata sm = sinksMetadata.get(identifier);
|
||||
if (sm == null) {
|
||||
throw new ISE("No sink has been processed for identifier[%s]", identifier);
|
||||
}
|
||||
File persistedDir = sm.getPersistedFileDir();
|
||||
if (persistedDir == null) {
|
||||
throw new ISE("Sink for identifier[%s] not found in local file system", identifier);
|
||||
}
|
||||
totalHydrantsMerged += sm.getNumHydrants();
|
||||
|
||||
// retrieve sink from disk:
|
||||
Sink sinkForIdentifier;
|
||||
try {
|
||||
sinkForIdentifier = getSinkForIdentifierPath(identifier, persistedDir);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new ISE(e, "Failed to retrieve sinks for identifier[%s] in path[%s]", identifier, persistedDir);
|
||||
}
|
||||
return Futures.transform(
|
||||
persistAll(null), // make sure persists is done before push...
|
||||
(Function<Object, SegmentsAndCommitMetadata>) commitMetadata -> {
|
||||
|
||||
// push sink:
|
||||
final DataSegment dataSegment = mergeAndPush(
|
||||
identifier,
|
||||
sinkForIdentifier
|
||||
);
|
||||
log.info("Push started, processsing[%d] sinks", identifiers.size());
|
||||
|
||||
// record it:
|
||||
if (dataSegment != null) {
|
||||
dataSegments.add(dataSegment);
|
||||
} else {
|
||||
log.warn("mergeAndPush[%s] returned null, skipping.", identifier);
|
||||
}
|
||||
int totalHydrantsMerged = 0;
|
||||
for (SegmentIdWithShardSpec identifier : identifiers) {
|
||||
SinkMetadata sm = sinksMetadata.get(identifier);
|
||||
if (sm == null) {
|
||||
throw new ISE("No sink has been processed for identifier[%s]", identifier);
|
||||
}
|
||||
File persistedDir = sm.getPersistedFileDir();
|
||||
if (persistedDir == null) {
|
||||
throw new ISE("Persisted directory for identifier[%s] is null in sink metadata", identifier);
|
||||
}
|
||||
totalHydrantsMerged += sm.getNumHydrants();
|
||||
|
||||
}
|
||||
// retrieve sink from disk:
|
||||
Sink sinkForIdentifier;
|
||||
try {
|
||||
sinkForIdentifier = getSinkForIdentifierPath(identifier, persistedDir);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new ISE(e, "Failed to retrieve sinks for identifier[%s]", identifier);
|
||||
}
|
||||
|
||||
log.info("Push complete: total sinks merged[%d], total hydrants merged[%d]",
|
||||
identifiers.size(), totalHydrantsMerged);
|
||||
// push it:
|
||||
final DataSegment dataSegment = mergeAndPush(
|
||||
identifier,
|
||||
sinkForIdentifier
|
||||
);
|
||||
|
||||
return Futures.immediateFuture(new SegmentsAndCommitMetadata(dataSegments, null));
|
||||
// record it:
|
||||
if (dataSegment != null) {
|
||||
dataSegments.add(dataSegment);
|
||||
} else {
|
||||
log.warn("mergeAndPush[%s] returned null, skipping.", identifier);
|
||||
}
|
||||
}
|
||||
log.info("Push done: total sinks merged[%d], total hydrants merged[%d]",
|
||||
identifiers.size(), totalHydrantsMerged
|
||||
);
|
||||
return new SegmentsAndCommitMetadata(dataSegments, commitMetadata);
|
||||
},
|
||||
pushExecutor // push it in the background, pushAndClear in BaseAppenderatorDriver guarantees
|
||||
// that segments are dropped before next add row
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -762,11 +870,37 @@ public class BatchAppenderator implements Appenderator
|
|||
|
||||
log.debug("Shutting down...");
|
||||
|
||||
clear(false);
|
||||
try {
|
||||
|
||||
unlockBasePersistDirectory();
|
||||
log.debug("Shutdown & wait for persistExecutor");
|
||||
if (persistExecutor != null) {
|
||||
persistExecutor.shutdown();
|
||||
if (!persistExecutor.awaitTermination(365, TimeUnit.DAYS)) {
|
||||
log.warn("persistExecutor not terminated");
|
||||
}
|
||||
persistExecutor = null;
|
||||
}
|
||||
|
||||
log.debug("Shutdown & wait for pushExecutor");
|
||||
if (pushExecutor != null) {
|
||||
pushExecutor.shutdown();
|
||||
if (!pushExecutor.awaitTermination(365, TimeUnit.DAYS)) {
|
||||
log.warn("pushExecutor not terminated");
|
||||
}
|
||||
pushExecutor = null;
|
||||
}
|
||||
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new ISE("Failed to wait & shutdown executors during close()");
|
||||
}
|
||||
|
||||
log.debug("Waited for and shutdown executors...");
|
||||
|
||||
// cleanup:
|
||||
clear(sinks, false);
|
||||
unlockBasePersistDirectory();
|
||||
List<File> persistedIdentifiers = getPersistedidentifierPaths();
|
||||
if (persistedIdentifiers != null) {
|
||||
for (File identifier : persistedIdentifiers) {
|
||||
|
@ -778,9 +912,6 @@ public class BatchAppenderator implements Appenderator
|
|||
sinksMetadata.clear();
|
||||
}
|
||||
|
||||
/**
|
||||
Nothing to do since there are no executors
|
||||
*/
|
||||
@Override
|
||||
public void closeNow()
|
||||
{
|
||||
|
@ -788,8 +919,8 @@ public class BatchAppenderator implements Appenderator
|
|||
log.debug("Appenderator already closed, skipping closeNow() call.");
|
||||
return;
|
||||
}
|
||||
|
||||
log.debug("Shutting down immediately...");
|
||||
shutdownExecutors();
|
||||
}
|
||||
|
||||
private void lockBasePersistDirectory()
|
||||
|
@ -904,10 +1035,17 @@ public class BatchAppenderator implements Appenderator
|
|||
return retVal;
|
||||
}
|
||||
|
||||
private void resetSinkMetadata()
|
||||
{
|
||||
rowsCurrentlyInMemory = 0;
|
||||
bytesCurrentlyInMemory = 0;
|
||||
metrics.setSinkCount(0);
|
||||
}
|
||||
|
||||
// This function does not remove the sink from its tracking Map (sinks), the caller is responsible for that
|
||||
// this is because the Map is not synchronized and removing elements from a map while traversing it
|
||||
// throws a concurrent access exception
|
||||
private void clearSinkMetadata(
|
||||
// throws concurrent access exception
|
||||
private void clearSinkMemoryCountersAndDiskStoredData(
|
||||
final SegmentIdWithShardSpec identifier,
|
||||
final Sink sink,
|
||||
final boolean removeOnDiskData
|
||||
|
@ -926,7 +1064,7 @@ public class BatchAppenderator implements Appenderator
|
|||
bytesCurrentlyInMemory -= calculateMemoryUsedByHydrant();
|
||||
}
|
||||
}
|
||||
// totalRows are not decremented when removing the sink from memory, sink was just persisted and it
|
||||
// totalRows are not decremented when removing the sink from memory, sink was just persisted, and it
|
||||
// still "lives" but it is in hibernation. It will be revived later just before push.
|
||||
}
|
||||
|
||||
|
@ -992,7 +1130,7 @@ public class BatchAppenderator implements Appenderator
|
|||
int numRows = indexToPersist.getIndex().size();
|
||||
|
||||
// since the sink may have been persisted before it may have lost its
|
||||
// hydrant count, we remember that value in the sinks metadata so we have
|
||||
// hydrant count, we remember that value in the sinks' metadata, so we have
|
||||
// to pull it from there....
|
||||
SinkMetadata sm = sinksMetadata.get(identifier);
|
||||
if (sm == null) {
|
||||
|
@ -1009,7 +1147,7 @@ public class BatchAppenderator implements Appenderator
|
|||
sm.setPersistedFileDir(persistDir);
|
||||
|
||||
log.info(
|
||||
"Persisted in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).",
|
||||
"About to persist in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).",
|
||||
indexToPersist.getSegmentId(),
|
||||
indexToPersist.getCount(),
|
||||
(System.nanoTime() - startTime) / 1000000,
|
||||
|
@ -1072,7 +1210,7 @@ public class BatchAppenderator implements Appenderator
|
|||
* This class is used for information that needs to be kept related to Sinks as
|
||||
* they are persisted and removed from memory at every incremental persist.
|
||||
* The information is used for sanity checks and as information required
|
||||
* for functionality, depending in the field that is used. More info about the
|
||||
* for functionality, depending on the field that is used. More info about the
|
||||
* fields is annotated as comments in the class
|
||||
*/
|
||||
private static class SinkMetadata
|
||||
|
|
|
@ -58,7 +58,7 @@ public class DefaultOfflineAppenderatorFactory implements AppenderatorFactory
|
|||
public Appenderator build(DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics)
|
||||
{
|
||||
final RowIngestionMeters rowIngestionMeters = new NoopRowIngestionMeters();
|
||||
return Appenderators.createOffline(
|
||||
return Appenderators.createClosedSegmentsOffline(
|
||||
schema.getDataSource(),
|
||||
schema,
|
||||
config,
|
||||
|
@ -73,8 +73,7 @@ public class DefaultOfflineAppenderatorFactory implements AppenderatorFactory
|
|||
false,
|
||||
config.isReportParseExceptions() ? 0 : Integer.MAX_VALUE,
|
||||
0
|
||||
),
|
||||
false
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -89,8 +89,41 @@ public class DummyForInjectionAppenderatorsManager implements AppenderatorsManag
|
|||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler,
|
||||
boolean useLegacyBatchProcessing
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
throw new UOE(ERROR_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Appenderator createOpenSegmentsOfflineAppenderatorForTask(
|
||||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
throw new UOE(ERROR_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Appenderator createClosedSegmentsOfflineAppenderatorForTask(
|
||||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
throw new UOE(ERROR_MSG);
|
||||
|
|
|
@ -121,8 +121,7 @@ public class PeonAppenderatorsManager implements AppenderatorsManager
|
|||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler,
|
||||
boolean useLegacyBatchProcessing
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
// CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators
|
||||
|
@ -139,13 +138,79 @@ public class PeonAppenderatorsManager implements AppenderatorsManager
|
|||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler,
|
||||
useLegacyBatchProcessing
|
||||
parseExceptionHandler
|
||||
);
|
||||
return batchAppenderator;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Appenderator createOpenSegmentsOfflineAppenderatorForTask(
|
||||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
// CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators
|
||||
if (realtimeAppenderator != null) {
|
||||
throw new ISE("A realtime appenderator was already created for this peon's task.");
|
||||
} else {
|
||||
batchAppenderator = Appenderators.createOpenSegmentsOffline(
|
||||
taskId,
|
||||
schema,
|
||||
config,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler
|
||||
);
|
||||
return batchAppenderator;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Appenderator createClosedSegmentsOfflineAppenderatorForTask(
|
||||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
// CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators
|
||||
if (realtimeAppenderator != null) {
|
||||
throw new ISE("A realtime appenderator was already created for this peon's task.");
|
||||
} else {
|
||||
batchAppenderator = Appenderators.createClosedSegmentsOffline(
|
||||
taskId,
|
||||
schema,
|
||||
config,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler
|
||||
);
|
||||
return batchAppenderator;
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public void removeAppenderatorsForTask(String taskId, String dataSource)
|
||||
{
|
||||
|
|
|
@ -207,8 +207,7 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager
|
|||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler,
|
||||
boolean useLegacyBatchProcessing
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
synchronized (this) {
|
||||
|
@ -227,8 +226,81 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager
|
|||
indexIO,
|
||||
wrapIndexMerger(indexMerger),
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler,
|
||||
useLegacyBatchProcessing
|
||||
parseExceptionHandler
|
||||
);
|
||||
datasourceBundle.addAppenderator(taskId, appenderator);
|
||||
return appenderator;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Appenderator createOpenSegmentsOfflineAppenderatorForTask(
|
||||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
synchronized (this) {
|
||||
DatasourceBundle datasourceBundle = datasourceBundles.computeIfAbsent(
|
||||
schema.getDataSource(),
|
||||
DatasourceBundle::new
|
||||
);
|
||||
|
||||
Appenderator appenderator = Appenderators.createOpenSegmentsOffline(
|
||||
taskId,
|
||||
schema,
|
||||
rewriteAppenderatorConfigMemoryLimits(config),
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
wrapIndexMerger(indexMerger),
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler
|
||||
);
|
||||
datasourceBundle.addAppenderator(taskId, appenderator);
|
||||
return appenderator;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Appenderator createClosedSegmentsOfflineAppenderatorForTask(
|
||||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
IndexMerger indexMerger,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
{
|
||||
synchronized (this) {
|
||||
DatasourceBundle datasourceBundle = datasourceBundles.computeIfAbsent(
|
||||
schema.getDataSource(),
|
||||
DatasourceBundle::new
|
||||
);
|
||||
|
||||
Appenderator appenderator = Appenderators.createClosedSegmentsOffline(
|
||||
taskId,
|
||||
schema,
|
||||
rewriteAppenderatorConfigMemoryLimits(config),
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
wrapIndexMerger(indexMerger),
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler
|
||||
);
|
||||
datasourceBundle.addAppenderator(taskId, appenderator);
|
||||
return appenderator;
|
||||
|
|
|
@ -62,7 +62,7 @@ import java.util.Map;
|
|||
import java.util.Objects;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
|
||||
public class BatchAppenderatorTester implements AutoCloseable
|
||||
public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable
|
||||
{
|
||||
public static final String DATASOURCE = "foo";
|
||||
|
||||
|
@ -75,14 +75,14 @@ public class BatchAppenderatorTester implements AutoCloseable
|
|||
|
||||
private final List<DataSegment> pushedSegments = new CopyOnWriteArrayList<>();
|
||||
|
||||
public BatchAppenderatorTester(
|
||||
public ClosedSegmensSinksBatchAppenderatorTester(
|
||||
final int maxRowsInMemory
|
||||
)
|
||||
{
|
||||
this(maxRowsInMemory, -1, null, false);
|
||||
}
|
||||
|
||||
public BatchAppenderatorTester(
|
||||
public ClosedSegmensSinksBatchAppenderatorTester(
|
||||
final int maxRowsInMemory,
|
||||
final boolean enablePushFailure
|
||||
)
|
||||
|
@ -90,7 +90,7 @@ public class BatchAppenderatorTester implements AutoCloseable
|
|||
this(maxRowsInMemory, -1, null, enablePushFailure);
|
||||
}
|
||||
|
||||
public BatchAppenderatorTester(
|
||||
public ClosedSegmensSinksBatchAppenderatorTester(
|
||||
final int maxRowsInMemory,
|
||||
final long maxSizeInBytes,
|
||||
final boolean enablePushFailure
|
||||
|
@ -99,7 +99,7 @@ public class BatchAppenderatorTester implements AutoCloseable
|
|||
this(maxRowsInMemory, maxSizeInBytes, null, enablePushFailure);
|
||||
}
|
||||
|
||||
public BatchAppenderatorTester(
|
||||
public ClosedSegmensSinksBatchAppenderatorTester(
|
||||
final int maxRowsInMemory,
|
||||
final long maxSizeInBytes,
|
||||
final File basePersistDirectory,
|
||||
|
@ -112,12 +112,11 @@ public class BatchAppenderatorTester implements AutoCloseable
|
|||
basePersistDirectory,
|
||||
enablePushFailure,
|
||||
new SimpleRowIngestionMeters(),
|
||||
false,
|
||||
false
|
||||
);
|
||||
}
|
||||
|
||||
public BatchAppenderatorTester(
|
||||
public ClosedSegmensSinksBatchAppenderatorTester(
|
||||
final int maxRowsInMemory,
|
||||
final long maxSizeInBytes,
|
||||
@Nullable final File basePersistDirectory,
|
||||
|
@ -126,18 +125,17 @@ public class BatchAppenderatorTester implements AutoCloseable
|
|||
)
|
||||
{
|
||||
this(maxRowsInMemory, maxSizeInBytes, basePersistDirectory, enablePushFailure, rowIngestionMeters,
|
||||
false, false
|
||||
false
|
||||
);
|
||||
}
|
||||
|
||||
public BatchAppenderatorTester(
|
||||
public ClosedSegmensSinksBatchAppenderatorTester(
|
||||
final int maxRowsInMemory,
|
||||
final long maxSizeInBytes,
|
||||
@Nullable final File basePersistDirectory,
|
||||
final boolean enablePushFailure,
|
||||
final RowIngestionMeters rowIngestionMeters,
|
||||
final boolean skipBytesInMemoryOverheadCheck,
|
||||
final boolean useLegacyBatchProcessing
|
||||
final boolean skipBytesInMemoryOverheadCheck
|
||||
)
|
||||
{
|
||||
objectMapper = new DefaultObjectMapper();
|
||||
|
@ -248,8 +246,7 @@ public class BatchAppenderatorTester implements AutoCloseable
|
|||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0),
|
||||
useLegacyBatchProcessing
|
||||
new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -302,7 +299,7 @@ public class BatchAppenderatorTester implements AutoCloseable
|
|||
}
|
||||
|
||||
|
||||
private static class TestIndexTuningConfig implements AppenderatorConfig
|
||||
static class TestIndexTuningConfig implements AppenderatorConfig
|
||||
{
|
||||
private final AppendableIndexSpec appendableIndexSpec;
|
||||
private final int maxRowsInMemory;
|
|
@ -51,7 +51,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class BatchAppenderatorDriverTest extends EasyMockSupport
|
||||
public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupport
|
||||
{
|
||||
private static final String DATA_SOURCE = "foo";
|
||||
private static final String VERSION = "abc123";
|
||||
|
@ -77,7 +77,7 @@ public class BatchAppenderatorDriverTest extends EasyMockSupport
|
|||
);
|
||||
|
||||
private SegmentAllocator allocator;
|
||||
private BatchAppenderatorTester appenderatorTester;
|
||||
private ClosedSegmensSinksBatchAppenderatorTester appenderatorTester;
|
||||
private BatchAppenderatorDriver driver;
|
||||
private DataSegmentKiller dataSegmentKiller;
|
||||
|
||||
|
@ -88,7 +88,7 @@ public class BatchAppenderatorDriverTest extends EasyMockSupport
|
|||
@Before
|
||||
public void setup()
|
||||
{
|
||||
appenderatorTester = new BatchAppenderatorTester(MAX_ROWS_IN_MEMORY);
|
||||
appenderatorTester = new ClosedSegmensSinksBatchAppenderatorTester(MAX_ROWS_IN_MEMORY);
|
||||
allocator = new TestSegmentAllocator(DATA_SOURCE, Granularities.HOUR);
|
||||
dataSegmentKiller = createStrictMock(DataSegmentKiller.class);
|
||||
driver = new BatchAppenderatorDriver(
|
||||
|
@ -142,7 +142,7 @@ public class BatchAppenderatorDriverTest extends EasyMockSupport
|
|||
Assert.assertNull(published.getCommitMetadata());
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test(timeout = 5000L)
|
||||
public void testIncrementalPush() throws Exception
|
||||
{
|
||||
Assert.assertNull(driver.startJob(null));
|
|
@ -22,6 +22,7 @@ package org.apache.druid.segment.realtime.appenderator;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.MapBasedInputRow;
|
||||
|
@ -42,7 +43,7 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
||||
public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private static final List<SegmentIdWithShardSpec> IDENTIFIERS = ImmutableList.of(
|
||||
createSegmentId("2000/2001", "A", 0), // should be in seg_0
|
||||
|
@ -53,14 +54,14 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testSimpleIngestion() throws Exception
|
||||
{
|
||||
try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) {
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, true)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
// startJob
|
||||
Assert.assertNull(appenderator.startJob());
|
||||
|
||||
// getDataSource
|
||||
Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource());
|
||||
Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource());
|
||||
|
||||
// add #1
|
||||
Assert.assertEquals(
|
||||
|
@ -82,7 +83,6 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
appenderator.getSegments().stream().sorted().collect(Collectors.toList())
|
||||
);
|
||||
|
||||
|
||||
// add #3, this hits max rows in memory:
|
||||
Assert.assertEquals(
|
||||
2,
|
||||
|
@ -90,7 +90,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
.getNumRowsInSegment()
|
||||
);
|
||||
|
||||
// since we just added three rows and the max rows in memory is three, all the segments (sinks etc)
|
||||
// since we just added three rows and the max rows in memory is three, all the segments (sinks etc.)
|
||||
// above should be cleared now
|
||||
Assert.assertEquals(
|
||||
Collections.emptyList(),
|
||||
|
@ -104,7 +104,6 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
.getNumRowsInSegment()
|
||||
);
|
||||
|
||||
|
||||
// push all
|
||||
final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push(
|
||||
appenderator.getSegments(),
|
||||
|
@ -136,14 +135,14 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testPeriodGranularityNonUTCIngestion() throws Exception
|
||||
{
|
||||
try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(1, true)) {
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(1, true)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
// startJob
|
||||
Assert.assertNull(appenderator.startJob());
|
||||
|
||||
// getDataSource
|
||||
Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource());
|
||||
Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource());
|
||||
|
||||
// Create a segment identifier with a non-utc interval
|
||||
SegmentIdWithShardSpec segmentIdWithNonUTCTime =
|
||||
|
@ -194,110 +193,26 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleIngestionWithFallbackCodePath() throws Exception
|
||||
{
|
||||
try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(
|
||||
3,
|
||||
-1,
|
||||
null,
|
||||
true,
|
||||
new SimpleRowIngestionMeters(),
|
||||
true,
|
||||
true
|
||||
)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
// startJob
|
||||
Assert.assertNull(appenderator.startJob());
|
||||
|
||||
// getDataSource
|
||||
Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource());
|
||||
|
||||
// add #1
|
||||
Assert.assertEquals(
|
||||
1,
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null)
|
||||
.getNumRowsInSegment()
|
||||
);
|
||||
|
||||
// add #2
|
||||
Assert.assertEquals(
|
||||
1,
|
||||
appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 2), null)
|
||||
.getNumRowsInSegment()
|
||||
);
|
||||
|
||||
// getSegments
|
||||
Assert.assertEquals(
|
||||
IDENTIFIERS.subList(0, 2),
|
||||
appenderator.getSegments().stream().sorted().collect(Collectors.toList())
|
||||
);
|
||||
|
||||
|
||||
// add #3, this hits max rows in memory:
|
||||
Assert.assertEquals(
|
||||
2,
|
||||
appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "sux", 1), null)
|
||||
.getNumRowsInSegment()
|
||||
);
|
||||
|
||||
// since we just added three rows and the max rows in memory is three BUT we are using
|
||||
// the old, fallback, code path that does not remove sinks, the segments should still be there
|
||||
Assert.assertEquals(
|
||||
2,
|
||||
appenderator.getSegments().size()
|
||||
);
|
||||
|
||||
// add #4, this will add one more temporary segment:
|
||||
Assert.assertEquals(
|
||||
1,
|
||||
appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "qux", 4), null)
|
||||
.getNumRowsInSegment()
|
||||
);
|
||||
|
||||
|
||||
// push all
|
||||
final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push(
|
||||
appenderator.getSegments(),
|
||||
null,
|
||||
false
|
||||
).get();
|
||||
Assert.assertEquals(
|
||||
IDENTIFIERS.subList(0, 3),
|
||||
Lists.transform(
|
||||
segmentsAndCommitMetadata.getSegments(),
|
||||
SegmentIdWithShardSpec::fromDataSegment
|
||||
).stream().sorted().collect(Collectors.toList())
|
||||
);
|
||||
Assert.assertEquals(
|
||||
tester.getPushedSegments().stream().sorted().collect(Collectors.toList()),
|
||||
segmentsAndCommitMetadata.getSegments().stream().sorted().collect(Collectors.toList())
|
||||
);
|
||||
|
||||
appenderator.close();
|
||||
Assert.assertTrue(appenderator.getSegments().isEmpty());
|
||||
}
|
||||
}
|
||||
@Test
|
||||
public void testMaxBytesInMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception
|
||||
{
|
||||
try (
|
||||
final BatchAppenderatorTester tester = new BatchAppenderatorTester(
|
||||
final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(
|
||||
100,
|
||||
1024,
|
||||
null,
|
||||
true,
|
||||
new SimpleRowIngestionMeters(),
|
||||
true,
|
||||
false
|
||||
true
|
||||
)
|
||||
) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
appenderator.startJob();
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null);
|
||||
//expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 54 (dimsKeySize) = 182 + 1 byte when null handling is enabled
|
||||
//expectedSizeInBytes =
|
||||
// 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 54 (dimsKeySize) =
|
||||
// 182 + 1 byte when null handling is enabled
|
||||
int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0;
|
||||
Assert.assertEquals(
|
||||
182 + nullHandlingOverhead,
|
||||
|
@ -317,14 +232,13 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadCheckConfig() throws Exception
|
||||
{
|
||||
try (
|
||||
final BatchAppenderatorTester tester = new BatchAppenderatorTester(
|
||||
final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(
|
||||
100,
|
||||
1024,
|
||||
null,
|
||||
true,
|
||||
new SimpleRowIngestionMeters(),
|
||||
true,
|
||||
false
|
||||
true
|
||||
)
|
||||
) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
@ -348,7 +262,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testMaxBytesInMemory() throws Exception
|
||||
{
|
||||
try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(100, 15000, true)) {
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(100, 15000, true)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
appenderator.startJob();
|
||||
|
@ -372,7 +286,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
for (int i = 0; i < 53; i++) {
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null);
|
||||
}
|
||||
sinkSizeOverhead = BatchAppenderator.ROUGH_OVERHEAD_PER_SINK;
|
||||
|
||||
// currHydrant size is 0 since we just persist all indexes to disk.
|
||||
currentInMemoryIndexSize = 0;
|
||||
// We are now over maxSizeInBytes after the add. Hence, we do a persist.
|
||||
|
@ -381,7 +295,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
currentInMemoryIndexSize,
|
||||
((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0))
|
||||
);
|
||||
// no sinks no hydrants after a persist so we should have zero bytes currently in memory
|
||||
// no sinks no hydrants after a persist, so we should have zero bytes currently in memory
|
||||
Assert.assertEquals(
|
||||
currentInMemoryIndexSize,
|
||||
((BatchAppenderator) appenderator).getBytesCurrentlyInMemory()
|
||||
|
@ -404,6 +318,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
for (int i = 0; i < 53; i++) {
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null);
|
||||
}
|
||||
|
||||
// currHydrant size is 0 since we just persist all indexes to disk.
|
||||
currentInMemoryIndexSize = 0;
|
||||
// We are now over maxSizeInBytes after the add. Hence, we do a persist.
|
||||
|
@ -416,17 +331,18 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
currentInMemoryIndexSize,
|
||||
((BatchAppenderator) appenderator).getBytesCurrentlyInMemory()
|
||||
);
|
||||
appenderator.persistAll(null).get();
|
||||
appenderator.close();
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory());
|
||||
}
|
||||
}
|
||||
|
||||
@Test(expected = RuntimeException.class)
|
||||
@Test(expected = RuntimeException.class, timeout = 5000L)
|
||||
public void testTaskFailAsPersistCannotFreeAnyMoreMemory() throws Exception
|
||||
{
|
||||
try (final BatchAppenderatorTester tester =
|
||||
new BatchAppenderatorTester(100, 5180, true)) {
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester =
|
||||
new ClosedSegmensSinksBatchAppenderatorTester(100, 5180, true)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
appenderator.startJob();
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null);
|
||||
|
@ -437,27 +353,27 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception
|
||||
{
|
||||
try (
|
||||
final BatchAppenderatorTester tester = new BatchAppenderatorTester(
|
||||
final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(
|
||||
100,
|
||||
10,
|
||||
null,
|
||||
true,
|
||||
new SimpleRowIngestionMeters(),
|
||||
true,
|
||||
false
|
||||
true
|
||||
)
|
||||
) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
appenderator.startJob();
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null);
|
||||
|
||||
// Expected 0 since we persisted after the add
|
||||
Assert.assertEquals(
|
||||
0,
|
||||
((BatchAppenderator) appenderator).getBytesCurrentlyInMemory()
|
||||
);
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null);
|
||||
// Expected 0 since we persisted after the add
|
||||
|
||||
Assert.assertEquals(
|
||||
0,
|
||||
((BatchAppenderator) appenderator).getBytesCurrentlyInMemory()
|
||||
|
@ -468,8 +384,8 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testTaskCleanupInMemoryCounterAfterCloseWithRowInMemory() throws Exception
|
||||
{
|
||||
try (final BatchAppenderatorTester tester =
|
||||
new BatchAppenderatorTester(100, 10000, true)) {
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester =
|
||||
new ClosedSegmensSinksBatchAppenderatorTester(100, 10000, true)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
appenderator.startJob();
|
||||
|
@ -495,8 +411,8 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testMaxBytesInMemoryInMultipleSinks() throws Exception
|
||||
{
|
||||
try (final BatchAppenderatorTester tester =
|
||||
new BatchAppenderatorTester(1000, 28748, true)) {
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester =
|
||||
new ClosedSegmensSinksBatchAppenderatorTester(1000, 28748, true)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
appenderator.startJob();
|
||||
|
@ -529,6 +445,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null);
|
||||
appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar_" + i, 1), null);
|
||||
}
|
||||
|
||||
// sinks + currHydrant size is 0 since we just persist all indexes to disk.
|
||||
currentInMemoryIndexSize = 0;
|
||||
// We are now over maxSizeInBytes after the add. Hence, we do a persist.
|
||||
|
@ -586,6 +503,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null);
|
||||
appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar_" + i, 1), null);
|
||||
}
|
||||
|
||||
// currHydrant size is 0 since we just persist all indexes to disk.
|
||||
currentInMemoryIndexSize = 0;
|
||||
// We are now over maxSizeInBytes after the add. Hence, we do a persist.
|
||||
|
@ -614,8 +532,8 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testIgnoreMaxBytesInMemory() throws Exception
|
||||
{
|
||||
try (final BatchAppenderatorTester tester =
|
||||
new BatchAppenderatorTester(100, -1, true)) {
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester =
|
||||
new ClosedSegmensSinksBatchAppenderatorTester(100, -1, true)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
|
@ -631,7 +549,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 1), null);
|
||||
|
||||
// we added two rows only and we told that maxSizeInBytes should be ignored, so it should not have been
|
||||
// we added two rows only, and we told that maxSizeInBytes should be ignored, so it should not have been
|
||||
// persisted:
|
||||
int sinkSizeOverhead = 2 * BatchAppenderator.ROUGH_OVERHEAD_PER_SINK;
|
||||
Assert.assertEquals(
|
||||
|
@ -647,7 +565,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testMaxRowsInMemory() throws Exception
|
||||
{
|
||||
try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) {
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, true)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
|
@ -663,14 +581,15 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bat", 1), null);
|
||||
// persist expected ^ (3) rows added
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "baz", 1), null);
|
||||
Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 1), null);
|
||||
Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bob", 1), null);
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
// persist expected ^ (3) rows added
|
||||
//appenderator.persistAll(null);
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
appenderator.close();
|
||||
}
|
||||
|
@ -679,7 +598,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testAllHydrantsAreRecovered() throws Exception
|
||||
{
|
||||
try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(1, false)) {
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(1, false)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
|
@ -690,7 +609,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo3", 1), null);
|
||||
|
||||
// Since maxRowsInMemory is one there ought to be three hydrants stored and recovered
|
||||
// just before push, internally the code has a sanity check to make sure that this works..if it does not it throws
|
||||
// just before push, internally the code has a sanity check to make sure that this works. If it does not it throws
|
||||
// an exception
|
||||
final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push(
|
||||
appenderator.getSegments(),
|
||||
|
@ -712,7 +631,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testTotalRowsPerSegment() throws Exception
|
||||
{
|
||||
try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) {
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, true)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
|
@ -772,22 +691,25 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testRestoreFromDisk() throws Exception
|
||||
{
|
||||
final BatchAppenderatorTester tester = new BatchAppenderatorTester(2, true);
|
||||
final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(2, true);
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
appenderator.startJob();
|
||||
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null);
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 2), null);
|
||||
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
|
||||
appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "baz", 3), null);
|
||||
appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 4), null);
|
||||
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
|
||||
appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "bob", 5), null);
|
||||
Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
appenderator.persistAll(null).get();
|
||||
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
|
||||
List<File> segmentPaths = ((BatchAppenderator) appenderator).getPersistedidentifierPaths();
|
||||
|
@ -808,7 +730,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testCleanupFromDiskAfterClose() throws Exception
|
||||
{
|
||||
final BatchAppenderatorTester tester = new BatchAppenderatorTester(2, true);
|
||||
final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(2, true);
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
appenderator.startJob();
|
||||
|
@ -845,10 +767,10 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
|
||||
@Test(timeout = 60_000L)
|
||||
@Test(timeout = 5000L)
|
||||
public void testTotalRowCount() throws Exception
|
||||
{
|
||||
try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) {
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, true)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
|
||||
Assert.assertEquals(0, appenderator.getTotalRowCount());
|
||||
|
@ -889,10 +811,10 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
public void testVerifyRowIngestionMetrics() throws Exception
|
||||
{
|
||||
final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters();
|
||||
try (final BatchAppenderatorTester tester =
|
||||
new BatchAppenderatorTester(5,
|
||||
10000L,
|
||||
null, false, rowIngestionMeters
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester =
|
||||
new ClosedSegmensSinksBatchAppenderatorTester(5,
|
||||
10000L,
|
||||
null, false, rowIngestionMeters
|
||||
)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
appenderator.startJob();
|
||||
|
@ -908,14 +830,14 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test(timeout = 10000L)
|
||||
public void testPushContract() throws Exception
|
||||
{
|
||||
final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters();
|
||||
try (final BatchAppenderatorTester tester =
|
||||
new BatchAppenderatorTester(1,
|
||||
50000L,
|
||||
null, false, rowIngestionMeters
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester =
|
||||
new ClosedSegmensSinksBatchAppenderatorTester(1,
|
||||
50000L,
|
||||
null, false, rowIngestionMeters
|
||||
)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
appenderator.startJob();
|
||||
|
@ -957,11 +879,62 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
}
|
||||
}
|
||||
|
||||
@Test(timeout = 5000L)
|
||||
public void testCloseContract() throws Exception
|
||||
{
|
||||
final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters();
|
||||
try (final ClosedSegmensSinksBatchAppenderatorTester tester =
|
||||
new ClosedSegmensSinksBatchAppenderatorTester(1,
|
||||
50000L,
|
||||
null, false, rowIngestionMeters
|
||||
)) {
|
||||
final Appenderator appenderator = tester.getAppenderator();
|
||||
appenderator.startJob();
|
||||
|
||||
// each one of these adds will trigger a persist since maxRowsInMemory is set to one above
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 1), null);
|
||||
appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar2", 1), null);
|
||||
|
||||
// push only a single segment
|
||||
ListenableFuture<SegmentsAndCommitMetadata> firstFuture = appenderator.push(
|
||||
Collections.singletonList(IDENTIFIERS.get(0)),
|
||||
null,
|
||||
false
|
||||
);
|
||||
|
||||
// push remaining segments:
|
||||
appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar3", 1), null);
|
||||
ListenableFuture<SegmentsAndCommitMetadata> secondFuture = appenderator.push(
|
||||
Collections.singletonList(IDENTIFIERS.get(1)),
|
||||
null,
|
||||
false
|
||||
);
|
||||
|
||||
// close should wait for all pushes and persists to end:
|
||||
appenderator.close();
|
||||
|
||||
Assert.assertTrue(!firstFuture.isCancelled());
|
||||
Assert.assertTrue(!secondFuture.isCancelled());
|
||||
|
||||
Assert.assertTrue(firstFuture.isDone());
|
||||
Assert.assertTrue(secondFuture.isDone());
|
||||
|
||||
final SegmentsAndCommitMetadata segmentsAndCommitMetadataForFirstFuture = firstFuture.get();
|
||||
final SegmentsAndCommitMetadata segmentsAndCommitMetadataForSecondFuture = secondFuture.get();
|
||||
|
||||
// all segments must have been pushed:
|
||||
Assert.assertEquals(segmentsAndCommitMetadataForFirstFuture.getSegments().size(), 1);
|
||||
Assert.assertEquals(segmentsAndCommitMetadataForSecondFuture.getSegments().size(), 1);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
private static SegmentIdWithShardSpec createNonUTCSegmentId(String interval, String version, int partitionNum)
|
||||
{
|
||||
return new SegmentIdWithShardSpec(
|
||||
BatchAppenderatorTester.DATASOURCE,
|
||||
ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE,
|
||||
new Interval(interval, ISOChronology.getInstance(DateTimes.inferTzFromString("Asia/Seoul"))),
|
||||
version,
|
||||
new LinearShardSpec(partitionNum)
|
||||
|
@ -972,7 +945,7 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
private static SegmentIdWithShardSpec createSegmentId(String interval, String version, int partitionNum)
|
||||
{
|
||||
return new SegmentIdWithShardSpec(
|
||||
BatchAppenderatorTester.DATASOURCE,
|
||||
ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE,
|
||||
Intervals.of(interval),
|
||||
version,
|
||||
new LinearShardSpec(partitionNum)
|
||||
|
@ -994,6 +967,5 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
@ -168,13 +168,13 @@ public class DefaultOfflineAppenderatorFactoryTest
|
|||
"A",
|
||||
new LinearShardSpec(0)
|
||||
);
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
|
||||
appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "bar", 1), null);
|
||||
Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory());
|
||||
appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "baz", 1), null);
|
||||
Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory());
|
||||
appenderator.close();
|
||||
Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory());
|
||||
Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory());
|
||||
}
|
||||
finally {
|
||||
appenderator.close();
|
||||
|
|
|
@ -0,0 +1,293 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.realtime.appenderator;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.JSONParseSpec;
|
||||
import org.apache.druid.data.input.impl.MapInputRowParser;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.FileUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.java.util.emitter.core.NoopEmitter;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.IndexMerger;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.ParseExceptionHandler;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
import org.apache.druid.segment.incremental.SimpleRowIngestionMeters;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.TuningConfig;
|
||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
import org.apache.druid.segment.loading.DataSegmentPusher;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
|
||||
public class OpenAndClosedSegmentsAppenderatorTester implements AutoCloseable
|
||||
{
|
||||
public static final String DATASOURCE = "foo";
|
||||
|
||||
private final DataSchema schema;
|
||||
private final FireDepartmentMetrics metrics;
|
||||
private final DataSegmentPusher dataSegmentPusher;
|
||||
private final ObjectMapper objectMapper;
|
||||
private final Appenderator appenderator;
|
||||
private final IndexIO indexIO;
|
||||
private final IndexMerger indexMerger;
|
||||
private final ServiceEmitter emitter;
|
||||
private final AppenderatorConfig tuningConfig;
|
||||
|
||||
|
||||
private final List<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),
|
||||
new DimensionsSpec(null, null, null),
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
),
|
||||
Map.class
|
||||
);
|
||||
schema = new DataSchema(
|
||||
DATASOURCE,
|
||||
parserMap,
|
||||
new AggregatorFactory[]{
|
||||
new CountAggregatorFactory("count"),
|
||||
new LongSumAggregatorFactory("met", "met")
|
||||
},
|
||||
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null),
|
||||
null,
|
||||
objectMapper
|
||||
);
|
||||
tuningConfig =
|
||||
new ClosedSegmensSinksBatchAppenderatorTester.TestIndexTuningConfig(
|
||||
TuningConfig.DEFAULT_APPENDABLE_INDEX,
|
||||
maxRowsInMemory,
|
||||
maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes,
|
||||
skipBytesInMemoryOverheadCheck,
|
||||
new IndexSpec(),
|
||||
0,
|
||||
false,
|
||||
0L,
|
||||
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
|
||||
IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE,
|
||||
basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory
|
||||
);
|
||||
|
||||
metrics = new FireDepartmentMetrics();
|
||||
|
||||
indexIO = new IndexIO(
|
||||
objectMapper,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
indexMerger = new IndexMergerV9(objectMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
||||
emitter = new ServiceEmitter(
|
||||
"test",
|
||||
"test",
|
||||
new NoopEmitter()
|
||||
);
|
||||
emitter.start();
|
||||
EmittingLogger.registerEmitter(emitter);
|
||||
dataSegmentPusher = new DataSegmentPusher()
|
||||
{
|
||||
private boolean mustFail = true;
|
||||
|
||||
@Deprecated
|
||||
@Override
|
||||
public String getPathForHadoop(String dataSource)
|
||||
{
|
||||
return getPathForHadoop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPathForHadoop()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException
|
||||
{
|
||||
if (enablePushFailure && mustFail) {
|
||||
mustFail = false;
|
||||
throw new IOException("Push failure test");
|
||||
} else if (enablePushFailure) {
|
||||
mustFail = true;
|
||||
}
|
||||
pushedSegments.add(segment);
|
||||
return segment;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<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));
|
||||
} else {
|
||||
appenderator = Appenderators.createClosedSegmentsOffline(
|
||||
schema.getDataSource(),
|
||||
schema,
|
||||
tuningConfig,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0));
|
||||
}
|
||||
}
|
||||
|
||||
private long getDefaultMaxBytesInMemory()
|
||||
{
|
||||
return (Runtime.getRuntime().totalMemory()) / 3;
|
||||
}
|
||||
|
||||
public DataSchema getSchema()
|
||||
{
|
||||
return schema;
|
||||
}
|
||||
|
||||
public AppenderatorConfig getTuningConfig()
|
||||
{
|
||||
return tuningConfig;
|
||||
}
|
||||
|
||||
public FireDepartmentMetrics getMetrics()
|
||||
{
|
||||
return metrics;
|
||||
}
|
||||
|
||||
public DataSegmentPusher getDataSegmentPusher()
|
||||
{
|
||||
return dataSegmentPusher;
|
||||
}
|
||||
|
||||
public ObjectMapper getObjectMapper()
|
||||
{
|
||||
return objectMapper;
|
||||
}
|
||||
|
||||
public Appenderator getAppenderator()
|
||||
{
|
||||
return appenderator;
|
||||
}
|
||||
|
||||
public List<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");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,205 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.realtime.appenderator;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.MapBasedInputRow;
|
||||
import org.apache.druid.indexing.overlord.SegmentPublishResult;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.segment.loading.DataSegmentKiller;
|
||||
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver.SegmentsForSequence;
|
||||
import org.apache.druid.segment.realtime.appenderator.SegmentWithState.SegmentState;
|
||||
import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriverTest.TestSegmentAllocator;
|
||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.EasyMockSupport;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class OpenAndClosedSegmentsBatchAppenderatorDriverTest extends EasyMockSupport
|
||||
{
|
||||
private static final String DATA_SOURCE = "foo";
|
||||
private static final String VERSION = "abc123";
|
||||
private static final int MAX_ROWS_IN_MEMORY = 100;
|
||||
private static final long TIMEOUT = 1000;
|
||||
|
||||
private static final List<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 TestUsedSegmentChecker(openAndClosedSegmentsAppenderatorTester.getPushedSegments()),
|
||||
dataSegmentKiller
|
||||
);
|
||||
|
||||
EasyMock.replay(dataSegmentKiller);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception
|
||||
{
|
||||
EasyMock.verify(dataSegmentKiller);
|
||||
|
||||
driver.clear();
|
||||
driver.close();
|
||||
}
|
||||
|
||||
@Test (timeout = 2000L)
|
||||
public void testSimple() throws Exception
|
||||
{
|
||||
Assert.assertNull(driver.startJob(null));
|
||||
|
||||
for (InputRow row : ROWS) {
|
||||
Assert.assertTrue(driver.add(row, "dummy").isOk());
|
||||
}
|
||||
|
||||
checkSegmentStates(2, SegmentState.APPENDING);
|
||||
|
||||
driver.pushAllAndClear(TIMEOUT);
|
||||
|
||||
checkSegmentStates(2, SegmentState.PUSHED_AND_DROPPED);
|
||||
|
||||
final SegmentsAndCommitMetadata published =
|
||||
driver.publishAll(null, null, makeOkPublisher(), Function.identity()).get(TIMEOUT, TimeUnit.MILLISECONDS);
|
||||
|
||||
Assert.assertEquals(
|
||||
ImmutableSet.of(
|
||||
new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000/PT1H"), VERSION, new NumberedShardSpec(0, 0)),
|
||||
new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000T01/PT1H"), VERSION, new NumberedShardSpec(0, 0))
|
||||
),
|
||||
published.getSegments()
|
||||
.stream()
|
||||
.map(SegmentIdWithShardSpec::fromDataSegment)
|
||||
.collect(Collectors.toSet())
|
||||
);
|
||||
|
||||
Assert.assertNull(published.getCommitMetadata());
|
||||
}
|
||||
|
||||
@Test(timeout = 5000L)
|
||||
public void testIncrementalPush() throws Exception
|
||||
{
|
||||
Assert.assertNull(driver.startJob(null));
|
||||
|
||||
int i = 0;
|
||||
for (InputRow row : ROWS) {
|
||||
Assert.assertTrue(driver.add(row, "dummy").isOk());
|
||||
|
||||
checkSegmentStates(1, SegmentState.APPENDING);
|
||||
checkSegmentStates(i, SegmentState.PUSHED_AND_DROPPED);
|
||||
|
||||
driver.pushAllAndClear(TIMEOUT);
|
||||
checkSegmentStates(0, SegmentState.APPENDING);
|
||||
checkSegmentStates(++i, SegmentState.PUSHED_AND_DROPPED);
|
||||
}
|
||||
|
||||
final SegmentsAndCommitMetadata published =
|
||||
driver.publishAll(null, null, makeOkPublisher(), Function.identity()).get(TIMEOUT, TimeUnit.MILLISECONDS);
|
||||
|
||||
Assert.assertEquals(
|
||||
ImmutableSet.of(
|
||||
new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000/PT1H"), VERSION, new NumberedShardSpec(0, 0)),
|
||||
new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000T01/PT1H"), VERSION, new NumberedShardSpec(0, 0)),
|
||||
new SegmentIdWithShardSpec(DATA_SOURCE, Intervals.of("2000T01/PT1H"), VERSION, new NumberedShardSpec(1, 0))
|
||||
),
|
||||
published.getSegments()
|
||||
.stream()
|
||||
.map(SegmentIdWithShardSpec::fromDataSegment)
|
||||
.collect(Collectors.toSet())
|
||||
);
|
||||
|
||||
Assert.assertNull(published.getCommitMetadata());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRestart()
|
||||
{
|
||||
Assert.assertNull(driver.startJob(null));
|
||||
driver.close();
|
||||
openAndClosedSegmentsAppenderatorTester.getAppenderator().close();
|
||||
|
||||
Assert.assertNull(driver.startJob(null));
|
||||
}
|
||||
|
||||
private void checkSegmentStates(int expectedNumSegmentsInState, SegmentState expectedState)
|
||||
{
|
||||
final SegmentsForSequence segmentsForSequence = driver.getSegments().get("dummy");
|
||||
Assert.assertNotNull(segmentsForSequence);
|
||||
final List<SegmentWithState> segmentWithStates = segmentsForSequence
|
||||
.allSegmentStateStream()
|
||||
.filter(segmentWithState -> segmentWithState.getState() == expectedState)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
Assert.assertEquals(expectedNumSegmentsInState, segmentWithStates.size());
|
||||
}
|
||||
|
||||
static TransactionalSegmentPublisher makeOkPublisher()
|
||||
{
|
||||
return (segmentsToBeOverwritten, segmentsToBeDropped, segmentsToPublish, commitMetadata) -> SegmentPublishResult.ok(ImmutableSet.of());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,229 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.realtime.appenderator;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.MapBasedInputRow;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class OpenAndClosedSegmentsBatchAppenderatorTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private static final List<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
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -44,6 +44,7 @@ import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactor
|
|||
import org.apache.druid.server.metrics.NoopServiceEmitter;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
@ -67,30 +68,42 @@ public class UnifiedIndexerAppenderatorsManagerTest
|
|||
() -> new DefaultQueryRunnerFactoryConglomerate(ImmutableMap.of())
|
||||
);
|
||||
|
||||
private final Appenderator appenderator = manager.createOfflineAppenderatorForTask(
|
||||
"taskId",
|
||||
new DataSchema(
|
||||
"myDataSource",
|
||||
new TimestampSpec("__time", "millis", null),
|
||||
null,
|
||||
null,
|
||||
new UniformGranularitySpec(Granularities.HOUR, Granularities.HOUR, false, Collections.emptyList()),
|
||||
null
|
||||
),
|
||||
EasyMock.createMock(AppenderatorConfig.class),
|
||||
new FireDepartmentMetrics(),
|
||||
new NoopDataSegmentPusher(),
|
||||
TestHelper.makeJsonMapper(),
|
||||
TestHelper.getTestIndexIO(),
|
||||
TestHelper.getTestIndexMergerV9(OnHeapMemorySegmentWriteOutMediumFactory.instance()),
|
||||
new NoopRowIngestionMeters(),
|
||||
new ParseExceptionHandler(new NoopRowIngestionMeters(), false, 0, 0),
|
||||
false
|
||||
);
|
||||
private AppenderatorConfig appenderatorConfig;
|
||||
private Appenderator appenderator;
|
||||
|
||||
@Before
|
||||
public void setup()
|
||||
{
|
||||
appenderatorConfig = EasyMock.createMock(AppenderatorConfig.class);
|
||||
EasyMock.expect(appenderatorConfig.getMaxPendingPersists()).andReturn(0);
|
||||
EasyMock.expect(appenderatorConfig.isSkipBytesInMemoryOverheadCheck()).andReturn(false);
|
||||
EasyMock.replay(appenderatorConfig);
|
||||
appenderator = manager.createClosedSegmentsOfflineAppenderatorForTask(
|
||||
"taskId",
|
||||
new DataSchema(
|
||||
"myDataSource",
|
||||
new TimestampSpec("__time", "millis", null),
|
||||
null,
|
||||
null,
|
||||
new UniformGranularitySpec(Granularities.HOUR, Granularities.HOUR, false, Collections.emptyList()),
|
||||
null
|
||||
),
|
||||
appenderatorConfig,
|
||||
new FireDepartmentMetrics(),
|
||||
new NoopDataSegmentPusher(),
|
||||
TestHelper.makeJsonMapper(),
|
||||
TestHelper.getTestIndexIO(),
|
||||
TestHelper.getTestIndexMergerV9(OnHeapMemorySegmentWriteOutMediumFactory.instance()),
|
||||
new NoopRowIngestionMeters(),
|
||||
new ParseExceptionHandler(new NoopRowIngestionMeters(), false, 0, 0)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_getBundle_knownDataSource()
|
||||
{
|
||||
|
||||
|
||||
final UnifiedIndexerAppenderatorsManager.DatasourceBundle bundle = manager.getBundle(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(appenderator.getDataSource())
|
||||
|
|
Loading…
Reference in New Issue