From aea77541bc2d4f93e60fb6ea1e30acd6b5861def Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 16 Apr 2021 09:17:52 -0700 Subject: [PATCH 01/47] Bound memory in native batch ingest create segments --- indexing-service/pom.xml | 5 - .../appenderator/BatchAppenderatorTest.java | 122 +- .../appenderator/BatchAppenderatorTester.java | 23 +- .../SeekableStreamIndexTaskTestBase.java | 4 +- .../realtime/appenderator/Appenderator.java | 7 +- .../realtime/appenderator/Appenderators.java | 7 +- .../appenderator/BatchAppenderator.java | 1526 +++++++++++++++++ ...torImpl.java => RealtimeAppenderator.java} | 6 +- .../UnifiedIndexerAppenderatorsManager.java | 4 +- .../appenderator/AppenderatorPlumberTest.java | 5 +- ...DefaultOfflineAppenderatorFactoryTest.java | 12 +- ...est.java => RealtimeAppenderatorTest.java} | 164 +- .../StreamAppenderatorDriverFailTest.java | 6 - 13 files changed, 1668 insertions(+), 223 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java rename server/src/main/java/org/apache/druid/segment/realtime/appenderator/{AppenderatorImpl.java => RealtimeAppenderator.java} (99%) rename server/src/test/java/org/apache/druid/segment/realtime/appenderator/{AppenderatorTest.java => RealtimeAppenderatorTest.java} (87%) diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 5c1d55aa7e48..7903856db922 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -62,11 +62,6 @@ druid-hll ${project.parent.version} - - org.apache.commons - commons-collections4 - 4.2 - io.dropwizard.metrics metrics-core diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java index e4417639f01f..4e6609b29d55 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java @@ -37,6 +37,7 @@ import org.junit.Assert; import org.junit.Test; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -49,11 +50,9 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest ); @Test - public void testSimpleIngestionWithIndexesNotMapped() throws Exception + public void testSimpleIngestion() throws Exception { - try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(2, - false, - false)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); boolean thrown; @@ -71,32 +70,34 @@ public void testSimpleIngestionWithIndexesNotMapped() throws Exception ); Assert.assertEquals( - 2, - appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 2), null) + 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 one more to hit max rows in memory: Assert.assertEquals( - 1, - appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 4), null) + 2, + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "sux", 1), null) .getNumRowsInSegment() ); - // getSegments - Assert.assertEquals(IDENTIFIERS.subList(0, 2), + // 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(), 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); + // add one more: + Assert.assertEquals( + 1, + appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "qux", 4), null) + .getNumRowsInSegment() + ); + // push all final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push( @@ -105,7 +106,7 @@ public void testSimpleIngestionWithIndexesNotMapped() throws Exception false ).get(); Assert.assertEquals( - IDENTIFIERS.subList(0, 2), + IDENTIFIERS.subList(0, 3), Lists.transform( segmentsAndCommitMetadata.getSegments(), new Function() @@ -126,83 +127,6 @@ public SegmentIdWithShardSpec apply(DataSegment input) } } - @Test - public void testSimpleIngestionWithIndexesMapped() throws Exception - { - try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(2, - false, - true)) { - final Appenderator appenderator = tester.getAppenderator(); - boolean thrown; - - // startJob - Assert.assertEquals(null, appenderator.startJob()); - - // getDataSource - Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource()); - - // add - Assert.assertEquals( - 1, - appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null) - .getNumRowsInSegment() - ); - - Assert.assertEquals( - 2, - appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 2), null) - .getNumRowsInSegment() - ); - - Assert.assertEquals( - 1, - appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 4), null) - .getNumRowsInSegment() - ); - - // getSegments - Assert.assertEquals(IDENTIFIERS.subList(0, 2), - appenderator.getSegments().stream().sorted().collect(Collectors.toList())); - - // getRowCount - Assert.assertEquals(2, appenderator.getRowCount(IDENTIFIERS.get(0))); - Assert.assertEquals(1, appenderator.getRowCount(IDENTIFIERS.get(1))); - thrown = false; - try { - appenderator.getRowCount(IDENTIFIERS.get(2)); - } - catch (IllegalStateException e) { - thrown = true; - } - Assert.assertTrue(thrown); - - // push all - final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push( - appenderator.getSegments(), - null, - false - ).get(); - Assert.assertEquals( - IDENTIFIERS.subList(0, 2), - Lists.transform( - segmentsAndCommitMetadata.getSegments(), - new Function() - { - @Override - public SegmentIdWithShardSpec apply(DataSegment input) - { - return SegmentIdWithShardSpec.fromDataSegment(input); - } - } - ).stream().sorted().collect(Collectors.toList()) - ); - Assert.assertEquals(tester.getPushedSegments().stream().sorted().collect(Collectors.toList()), - segmentsAndCommitMetadata.getSegments().stream().sorted().collect(Collectors.toList())); - - appenderator.clear(); - Assert.assertTrue(appenderator.getSegments().isEmpty()); - } - } private static SegmentIdWithShardSpec createSegmentId(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java index 4058aff938e5..d3bae2f7ae7a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java @@ -74,21 +74,26 @@ public class BatchAppenderatorTester implements AutoCloseable private final List pushedSegments = new CopyOnWriteArrayList<>(); + public BatchAppenderatorTester( + final int maxRowsInMemory + ) + { + this(maxRowsInMemory, -1, null, false); + } + public BatchAppenderatorTester( final int maxRowsInMemory, - final boolean enablePushFailure, - boolean batchMemoryMappedIndex + final boolean enablePushFailure ) { - this(maxRowsInMemory, -1, null, enablePushFailure, batchMemoryMappedIndex); + this(maxRowsInMemory, -1, null, enablePushFailure); } public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final File basePersistDirectory, - final boolean enablePushFailure, - boolean batchMemoryMappedIndex + final boolean enablePushFailure ) { this( @@ -97,8 +102,7 @@ public BatchAppenderatorTester( basePersistDirectory, enablePushFailure, new SimpleRowIngestionMeters(), - false, - batchMemoryMappedIndex + false ); } @@ -108,8 +112,7 @@ public BatchAppenderatorTester( final File basePersistDirectory, final boolean enablePushFailure, final RowIngestionMeters rowIngestionMeters, - final boolean skipBytesInMemoryOverheadCheck, - boolean batchMemoryMappedIndex + final boolean skipBytesInMemoryOverheadCheck ) { objectMapper = new DefaultObjectMapper(); @@ -234,7 +237,7 @@ public Map makeLoadSpec(URI uri) indexMerger, rowIngestionMeters, new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - batchMemoryMappedIndex + false ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index e2f226a42769..a15a0e5e0fc0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -79,7 +79,7 @@ import org.apache.druid.segment.column.DictionaryEncodedColumn; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; +import org.apache.druid.segment.realtime.appenderator.RealtimeAppenderator; import org.apache.druid.timeline.DataSegment; import org.apache.druid.utils.CompressionUtils; import org.assertj.core.api.Assertions; @@ -461,7 +461,7 @@ protected long countEvents(final Task task) protected void unlockAppenderatorBasePersistDirForTask(SeekableStreamIndexTask task) throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { - Method unlockBasePersistDir = ((AppenderatorImpl) task.getAppenderator()) + Method unlockBasePersistDir = ((RealtimeAppenderator) task.getAppenderator()) .getClass() .getDeclaredMethod("unlockBasePersistDirectory"); unlockBasePersistDir.setAccessible(true); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java index 1d7da70837a7..a0793020c8ba 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java @@ -221,7 +221,10 @@ ListenableFuture push( * files. In this case, the code will avoid memory mapping them thus ameliorating the occurance * of OOMs. */ - boolean isRealTime(); + default boolean isRealTime() + { + return false; + } /** * Result of {@link Appenderator#add} containing following information @@ -235,7 +238,7 @@ class AppenderatorAddResult private final int numRowsInSegment; private final boolean isPersistRequired; - AppenderatorAddResult( + public AppenderatorAddResult( SegmentIdWithShardSpec identifier, int numRowsInSegment, boolean isPersistRequired diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index ff8799daeba4..607c613aa9eb 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -63,7 +63,7 @@ public static Appenderator createRealtime( ParseExceptionHandler parseExceptionHandler ) { - return new AppenderatorImpl( + return new RealtimeAppenderator( id, schema, config, @@ -108,7 +108,7 @@ public static Appenderator createOffline( boolean batchMemoryMappedIndex ) { - return new AppenderatorImpl( + return new BatchAppenderator( id, schema, config, @@ -121,8 +121,7 @@ public static Appenderator createOffline( indexMerger, null, rowIngestionMeters, - parseExceptionHandler, - batchMemoryMappedIndex // This is a task config (default false) to fallback to "old" code in case of bug with the new memory optimization code + parseExceptionHandler ); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java new file mode 100644 index 000000000000..58010cc7fc15 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -0,0 +1,1526 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Stopwatch; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.commons.lang.mutable.MutableLong; +import org.apache.druid.client.cache.Cache; +import org.apache.druid.data.input.Committer; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.incremental.IncrementalIndexAddResult; +import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.ParseExceptionHandler; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.FireHydrant; +import org.apache.druid.segment.realtime.plumber.Sink; +import org.apache.druid.server.coordination.DataSegmentAnnouncer; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; + +public class BatchAppenderator implements Appenderator +{ + public static final int ROUGH_OVERHEAD_PER_SINK = 5000; + // Rough estimate of memory footprint of empty FireHydrant based on actual heap dumps + public static final int ROUGH_OVERHEAD_PER_HYDRANT = 1000; + + private static final EmittingLogger log = new EmittingLogger(BatchAppenderator.class); + private static final int WARN_DELAY = 1000; + private static final String IDENTIFIER_FILE_NAME = "identifier.json"; + + private final String myId; + private final DataSchema schema; + private final AppenderatorConfig tuningConfig; + private final FireDepartmentMetrics metrics; + private final DataSegmentPusher dataSegmentPusher; + private final ObjectMapper objectMapper; + private final DataSegmentAnnouncer segmentAnnouncer; + private final IndexIO indexIO; + private final IndexMerger indexMerger; + private final Cache cache; + /** + * This map needs to be concurrent because it's accessed and mutated from multiple threads: both the thread from where + * this Appenderator is used (and methods like {@link #add(SegmentIdWithShardSpec, InputRow, Supplier, boolean)} are + * called) and from {@link #persistExecutor}. It could also be accessed (but not mutated) potentially in the context + * of any thread from {@link #drop}. + */ + private final ConcurrentMap sinks = new ConcurrentHashMap<>(); + //private final Set droppingSinks = Sets.newConcurrentHashSet(); + private final long maxBytesTuningConfig; + private final boolean skipBytesInMemoryOverheadCheck; + + private final QuerySegmentWalker texasRanger; + // This variable updated in add(), persist(), and drop() + private final AtomicInteger rowsCurrentlyInMemory = new AtomicInteger(); + private final AtomicInteger totalRows = new AtomicInteger(); + private final AtomicLong bytesCurrentlyInMemory = new AtomicLong(); + private final RowIngestionMeters rowIngestionMeters; + private final ParseExceptionHandler parseExceptionHandler; + // Synchronize persisting commitMetadata so that multiple persist threads (if present) + // and abandon threads do not step over each other + private final Lock commitLock = new ReentrantLock(); + + private final AtomicBoolean closed = new AtomicBoolean(false); + + private volatile ListeningExecutorService persistExecutor = null; + private volatile ListeningExecutorService pushExecutor = null; + // use intermediate executor so that deadlock conditions can be prevented + // where persist and push Executor try to put tasks in each other queues + // thus creating circular dependency + private volatile ListeningExecutorService intermediateTempExecutor = null; + private volatile long nextFlush; + private volatile FileLock basePersistDirLock = null; + private volatile FileChannel basePersistDirLockChannel = null; + + private volatile Throwable persistError; + + /** + * This constructor allows the caller to provide its own SinkQuerySegmentWalker. + *

+ * The sinkTimeline is set to the sink timeline of the provided SinkQuerySegmentWalker. + * If the SinkQuerySegmentWalker is null, a new sink timeline is initialized. + *

+ * It is used by UnifiedIndexerAppenderatorsManager which allows queries on data associated with multiple + * Appenderators. + */ + BatchAppenderator( + String id, + DataSchema schema, + AppenderatorConfig tuningConfig, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + DataSegmentAnnouncer segmentAnnouncer, + @Nullable SinkQuerySegmentWalker sinkQuerySegmentWalker, + IndexIO indexIO, + IndexMerger indexMerger, + Cache cache, + RowIngestionMeters rowIngestionMeters, + ParseExceptionHandler parseExceptionHandler + ) + { + Preconditions.checkArgument( + sinkQuerySegmentWalker == null, + "Batch appenderator does not use a versioned timeline" + ); + + this.myId = id; + this.schema = Preconditions.checkNotNull(schema, "schema"); + this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); + this.metrics = Preconditions.checkNotNull(metrics, "metrics"); + this.dataSegmentPusher = Preconditions.checkNotNull(dataSegmentPusher, "dataSegmentPusher"); + this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper"); + this.segmentAnnouncer = Preconditions.checkNotNull(segmentAnnouncer, "segmentAnnouncer"); + this.indexIO = Preconditions.checkNotNull(indexIO, "indexIO"); + this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger"); + this.cache = cache; + this.texasRanger = sinkQuerySegmentWalker; + this.rowIngestionMeters = Preconditions.checkNotNull(rowIngestionMeters, "rowIngestionMeters"); + this.parseExceptionHandler = Preconditions.checkNotNull(parseExceptionHandler, "parseExceptionHandler"); + + + maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault(); + skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck(); + } + + @Override + public String getId() + { + return myId; + } + + @Override + public String getDataSource() + { + return schema.getDataSource(); + } + + @Override + public Object startJob() + { + tuningConfig.getBasePersistDirectory().mkdirs(); + lockBasePersistDirectory(); + final Object retVal = bootstrapSinksFromDisk(); + initializeExecutors(); + resetNextFlush(); + return retVal; + } + + private void throwPersistErrorIfExists() + { + if (persistError != null) { + throw new RE(persistError, "Error while persisting"); + } + } + + @Override + public AppenderatorAddResult add( + final SegmentIdWithShardSpec identifier, + final InputRow row, + @Nullable final Supplier committerSupplier, + final boolean allowIncrementalPersists + ) throws IndexSizeExceededException, SegmentNotWritableException + { + throwPersistErrorIfExists(); + Preconditions.checkArgument( + committerSupplier == null, + "Batch appenderator does not need a committer!" + ); + + if (!identifier.getDataSource().equals(schema.getDataSource())) { + throw new IAE( + "Expected dataSource[%s] but was asked to insert row for dataSource[%s]?!", + schema.getDataSource(), + identifier.getDataSource() + ); + } + + final Sink sink = getOrCreateSink(identifier); + metrics.reportMessageMaxTimestamp(row.getTimestampFromEpoch()); + final int sinkRowsInMemoryBeforeAdd = sink.getNumRowsInMemory(); + final int sinkRowsInMemoryAfterAdd; + final long bytesInMemoryBeforeAdd = sink.getBytesInMemory(); + final long bytesInMemoryAfterAdd; + final IncrementalIndexAddResult addResult; + + try { + addResult = sink.add(row, !allowIncrementalPersists); + sinkRowsInMemoryAfterAdd = addResult.getRowCount(); + bytesInMemoryAfterAdd = addResult.getBytesInMemory(); + } + catch (IndexSizeExceededException e) { + // Uh oh, we can't do anything about this! We can't persist (commit metadata would be out of sync) and we + // can't add the row (it just failed). This should never actually happen, though, because we check + // sink.canAddRow after returning from add. + log.error(e, "Sink for segment[%s] was unexpectedly full!", identifier); + throw e; + } + + if (sinkRowsInMemoryAfterAdd < 0) { + throw new SegmentNotWritableException("Attempt to add row to swapped-out sink for segment[%s].", identifier); + } + + if (addResult.isRowAdded()) { + rowIngestionMeters.incrementProcessed(); + } else if (addResult.hasParseException()) { + parseExceptionHandler.handle(addResult.getParseException()); + } + + final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd; + rowsCurrentlyInMemory.addAndGet(numAddedRows); + bytesCurrentlyInMemory.addAndGet(bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd); + totalRows.addAndGet(numAddedRows); + + boolean isPersistRequired = false; + boolean persist = false; + List persistReasons = new ArrayList<>(); + + if (!sink.canAppendRow()) { + persist = true; + persistReasons.add("No more rows can be appended to sink"); + } + if (System.currentTimeMillis() > nextFlush) { + persist = true; + persistReasons.add(StringUtils.format( + "current time[%d] is greater than nextFlush[%d]", + System.currentTimeMillis(), + nextFlush + )); + } + if (rowsCurrentlyInMemory.get() >= tuningConfig.getMaxRowsInMemory()) { + persist = true; + persistReasons.add(StringUtils.format( + "rowsCurrentlyInMemory[%d] is greater than maxRowsInMemory[%d]", + rowsCurrentlyInMemory.get(), + tuningConfig.getMaxRowsInMemory() + )); + } + if (bytesCurrentlyInMemory.get() >= maxBytesTuningConfig) { + persist = true; + persistReasons.add(StringUtils.format( + "bytesCurrentlyInMemory[%d] is greater than maxBytesInMemory[%d]", + bytesCurrentlyInMemory.get(), + maxBytesTuningConfig + )); + } + if (persist) { + if (allowIncrementalPersists) { + // persistAll clears rowsCurrentlyInMemory, no need to update it. + log.info("Flushing in-memory data to disk because %s.", String.join(",", persistReasons)); + + long bytesToBePersisted = 0L; + for (Map.Entry entry : sinks.entrySet()) { + final Sink sinkEntry = entry.getValue(); + if (sinkEntry != null) { + bytesToBePersisted += sinkEntry.getBytesInMemory(); + if (sinkEntry.swappable()) { + // After swapping the sink, we use memory mapped segment instead (but only for real time appenderators!). + // However, the memory mapped segment still consumes memory. + // These memory mapped segments are held in memory throughout the ingestion phase and permanently add to the bytesCurrentlyInMemory + int memoryStillInUse = calculateMMappedHydrantMemoryInUsed(sink.getCurrHydrant()); + bytesCurrentlyInMemory.addAndGet(memoryStillInUse); + } + } + } + + if (!skipBytesInMemoryOverheadCheck + && bytesCurrentlyInMemory.get() - bytesToBePersisted > maxBytesTuningConfig) { + // We are still over maxBytesTuningConfig even after persisting. + // This means that we ran out of all available memory to ingest (due to overheads created as part of ingestion) + final String alertMessage = StringUtils.format( + "Task has exceeded safe estimated heap usage limits, failing " + + "(numSinks: [%d] numHydrantsAcrossAllSinks: [%d] totalRows: [%d])" + + "(bytesCurrentlyInMemory: [%d] - bytesToBePersisted: [%d] > maxBytesTuningConfig: [%d])", + sinks.size(), + sinks.values().stream().mapToInt(Iterables::size).sum(), + getTotalRowCount(), + bytesCurrentlyInMemory.get(), + bytesToBePersisted, + maxBytesTuningConfig + ); + final String errorMessage = StringUtils.format( + "%s.\nThis can occur when the overhead from too many intermediary segment persists becomes to " + + "great to have enough space to process additional input rows. This check, along with metering the overhead " + + "of these objects to factor into the 'maxBytesInMemory' computation, can be disabled by setting " + + "'skipBytesInMemoryOverheadCheck' to 'true' (note that doing so might allow the task to naturally encounter " + + "a 'java.lang.OutOfMemoryError'). Alternatively, 'maxBytesInMemory' can be increased which will cause an " + + "increase in heap footprint, but will allow for more intermediary segment persists to occur before " + + "reaching this condition.", + alertMessage + ); + log.makeAlert(alertMessage) + .addData("dataSource", schema.getDataSource()) + .emit(); + throw new RuntimeException(errorMessage); + } + + +// Futures.addCallback( +// persistAll(null), +// new FutureCallback() +// { +// @Override +// public void onSuccess(@Nullable Object result) +// { +// // do nothing +// } +// +// @Override +// public void onFailure(Throwable t) +// { +// persistError = t; +// } +// } +// ); + + persistAllAndClear(); + + } else { + isPersistRequired = true; + } + } + return new AppenderatorAddResult(identifier, sink.getNumRows(), isPersistRequired); + } + + @Override + public List getSegments() + { + return ImmutableList.copyOf(sinks.keySet()); + } + + @Override + public int getRowCount(final SegmentIdWithShardSpec identifier) + { + final Sink sink = sinks.get(identifier); + + if (sink == null) { + throw new ISE("No such sink: %s", identifier); + } else { + return sink.getNumRows(); + } + } + + @Override + public int getTotalRowCount() + { + return totalRows.get(); + } + + @VisibleForTesting + int getRowsInMemory() + { + return rowsCurrentlyInMemory.get(); + } + + @VisibleForTesting + long getBytesCurrentlyInMemory() + { + return bytesCurrentlyInMemory.get(); + } + + @VisibleForTesting + long getBytesInMemory(SegmentIdWithShardSpec identifier) + { + final Sink sink = sinks.get(identifier); + + if (sink == null) { + throw new ISE("No such sink: %s", identifier); + } else { + return sink.getBytesInMemory(); + } + } + + private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) + { + Sink retVal = sinks.get(identifier); + + if (retVal == null) { + retVal = new Sink( + identifier.getInterval(), + schema, + identifier.getShardSpec(), + identifier.getVersion(), + tuningConfig.getAppendableIndexSpec(), + tuningConfig.getMaxRowsInMemory(), + maxBytesTuningConfig, + null + ); + bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed(retVal)); + + try { + segmentAnnouncer.announceSegment(retVal.getSegment()); + } + catch (IOException e) { + log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource()) + .addData("interval", retVal.getInterval()) + .emit(); + } + + sinks.put(identifier, retVal); + metrics.setSinkCount(sinks.size()); + } + + return retVal; + } + + @Override + public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) + { + if (texasRanger == null) { + throw new IllegalStateException("Don't query me, bro."); + } + + return texasRanger.getQueryRunnerForIntervals(query, intervals); + } + + @Override + public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) + { + if (texasRanger == null) { + throw new IllegalStateException("Don't query me, bro."); + } + + return texasRanger.getQueryRunnerForSegments(query, specs); + } + + @Override + public void clear() throws InterruptedException + { + clear(true); + } + + private void clear(boolean removeOnDiskData) throws InterruptedException + { + // Drop commit metadata, then abandon all segments. + log.info("Clearing all sinks & hydrants, removing data on disk: [%s]", removeOnDiskData); + try { + throwPersistErrorIfExists(); + // Drop everything. + final List> futures = new ArrayList<>(); + for (Map.Entry entry : sinks.entrySet()) { + futures.add(abandonSegment(entry.getKey(), entry.getValue(), removeOnDiskData)); + } + // Await dropping. + Futures.allAsList(futures).get(); + } + catch (ExecutionException e) { + throw new RuntimeException(e); + } + } + + @Override + public ListenableFuture drop(final SegmentIdWithShardSpec identifier) + { + final Sink sink = sinks.get(identifier); + if (sink != null) { + return abandonSegment(identifier, sink, true); + } else { + return Futures.immediateFuture(null); + } + } + + private SegmentsAndCommitMetadata persistAllAndClear() + { + final ListenableFuture toPersist = Futures.transform( + persistAll(null), + (Function) future -> future + ); + + // make sure sinks are cleared before push is called + final SegmentsAndCommitMetadata commitMetadata; + try { + commitMetadata = (SegmentsAndCommitMetadata) toPersist.get(); + clear(false); + return commitMetadata; + } + catch (Throwable t) { + persistError = t; + } + return null; + } + + @Override + public ListenableFuture persistAll(@Nullable final Committer committer) + { + throwPersistErrorIfExists(); + final List> indexesToPersist = new ArrayList<>(); + int numPersistedRows = 0; + long bytesPersisted = 0L; + MutableLong totalHydrantsCount = new MutableLong(); + MutableLong totalHydrantsPersisted = new MutableLong(); + final long totalSinks = sinks.size(); + for (Map.Entry entry : sinks.entrySet()) { + final SegmentIdWithShardSpec identifier = entry.getKey(); + final Sink sink = entry.getValue(); + if (sink == null) { + throw new ISE("No sink for identifier: %s", identifier); + } + final List hydrants = Lists.newArrayList(sink); + totalHydrantsCount.add(hydrants.size()); + numPersistedRows += sink.getNumRowsInMemory(); + bytesPersisted += sink.getBytesInMemory(); + + final int limit = sink.isWritable() ? hydrants.size() - 1 : hydrants.size(); + + // gather hydrants that have not been persisted: + for (FireHydrant hydrant : hydrants.subList(0, limit)) { + if (!hydrant.hasSwapped()) { + log.debug("Hydrant[%s] hasn't persisted yet, persisting. Segment[%s]", hydrant, identifier); + indexesToPersist.add(Pair.of(hydrant, identifier)); + totalHydrantsPersisted.add(1); + } + } + + if (sink.swappable()) { + // It is swappable. Get the old one to persist it and create a new one: + indexesToPersist.add(Pair.of(sink.swap(), identifier)); + totalHydrantsPersisted.add(1); + } + } + log.debug("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); + + final Stopwatch runExecStopwatch = Stopwatch.createStarted(); + final Stopwatch persistStopwatch = Stopwatch.createStarted(); + AtomicLong totalPersistedRows = new AtomicLong(numPersistedRows); + final ListenableFuture future = persistExecutor.submit( + new Callable() + { + @Override + public Object call() + { + try { + for (Pair pair : indexesToPersist) { + metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs)); + } + + log.info( + "Flushed in-memory data for segments: %s", + indexesToPersist.stream() + .map(itp -> itp.rhs.asSegmentId().toString()) + .distinct() + .collect(Collectors.joining(", ")) + ); + log.info( + "Persisted stats: processed rows: [%d], persisted rows[%d], sinks: [%d], total fireHydrants (across sinks): [%d], persisted fireHydrants (across sinks): [%d]", + rowIngestionMeters.getProcessed(), + totalPersistedRows.get(), + totalSinks, + totalHydrantsCount.longValue(), + totalHydrantsPersisted.longValue() + ); + + // return null if committer is null + return null; + } + catch (Exception e) { + metrics.incrementFailedPersists(); + throw e; + } + finally { + metrics.incrementNumPersists(); + metrics.incrementPersistTimeMillis(persistStopwatch.elapsed(TimeUnit.MILLISECONDS)); + persistStopwatch.stop(); + } + } + } + ); + + final long startDelay = runExecStopwatch.elapsed(TimeUnit.MILLISECONDS); + metrics.incrementPersistBackPressureMillis(startDelay); + if (startDelay > WARN_DELAY) { + log.warn("Ingestion was throttled for [%,d] millis because persists were pending.", startDelay); + } + runExecStopwatch.stop(); + resetNextFlush(); + + // NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes. + rowsCurrentlyInMemory.addAndGet(-numPersistedRows); + bytesCurrentlyInMemory.addAndGet(-bytesPersisted); + + log.info("Persisted rows[%,d] and bytes[%,d]", numPersistedRows, bytesPersisted); + + return future; + } + + @Override + public ListenableFuture push( + final Collection identifiers, + @Nullable final Committer committer, + final boolean useUniquePath + ) + { + + if (committer != null) { + throw new ISE("There should be no committer for batch ingestion"); + } + + // Any sinks not persisted so far will be persisted before push: + final SegmentsAndCommitMetadata commitMetadata = persistAllAndClear(); + + final ListenableFuture pushFuture = pushExecutor.submit( + new Callable() + { + @Override + public SegmentsAndCommitMetadata call() + { + log.info("Preparing to push..."); + + final List dataSegments = new ArrayList<>(); + List persistedIdentifiers = getPersistedidentifierPaths(); + for (File identifier : persistedIdentifiers) { + Pair sinkAndHydrants = getIdentifierAndSinkForPersistedFile(identifier); + final DataSegment dataSegment = mergeAndPush( + sinkAndHydrants.lhs, + sinkAndHydrants.rhs, + useUniquePath + ); + if (dataSegment != null) { + dataSegments.add(dataSegment); + } else { + log.warn("mergeAndPush[%s] returned null, skipping.", sinkAndHydrants.lhs); + } + } + + log.info("Push complete..."); + + return new SegmentsAndCommitMetadata(dataSegments, commitMetadata); + } + }); + + return pushFuture; + } + + /** + * Insert a barrier into the merge-and-push queue. When this future resolves, all pending pushes will have finished. + * This is useful if we're going to do something that would otherwise potentially break currently in-progress + * pushes. + */ + private ListenableFuture pushBarrier() + { + return intermediateTempExecutor.submit( + (Runnable) () -> pushExecutor.submit(() -> { + }) + ); + } + + /** + * Merge segment, push to deep storage. Should only be used on segments that have been fully persisted. Must only + * be run in the single-threaded pushExecutor. + * + * @param identifier sink identifier + * @param sink sink to push + * @param useUniquePath true if the segment should be written to a path with a unique identifier + * @return segment descriptor, or null if the sink is no longer valid + */ + @Nullable + private DataSegment mergeAndPush( + final SegmentIdWithShardSpec identifier, + final Sink sink, + final boolean useUniquePath + ) + { + + // Use a descriptor file to indicate that pushing has completed. + final File persistDir = computePersistDir(identifier); + final File mergedTarget = new File(persistDir, "merged"); + final File descriptorFile = computeDescriptorFile(identifier); + + // Sanity checks + for (FireHydrant hydrant : sink) { + if (sink.isWritable()) { + throw new ISE("Expected sink to be no longer writable before mergeAndPush for segment[%s].", identifier); + } + + synchronized (hydrant) { + if (!hydrant.hasSwapped()) { + throw new ISE("Expected sink to be fully persisted before mergeAndPush for segment[%s].", identifier); + } + } + } + + try { + if (descriptorFile.exists()) { + // Already pushed. + + if (useUniquePath) { + // Don't reuse the descriptor, because the caller asked for a unique path. Leave the old one as-is, since + // it might serve some unknown purpose. + log.debug( + "Segment[%s] already pushed, but we want a unique path, so will push again with a new path.", + identifier + ); + } else { + log.info("Segment[%s] already pushed, skipping.", identifier); + return objectMapper.readValue(descriptorFile, DataSegment.class); + } + } + + removeDirectory(mergedTarget); + + if (mergedTarget.exists()) { + throw new ISE("Merged target[%s] exists after removing?!", mergedTarget); + } + + final File mergedFile; + final long mergeFinishTime; + final long startTime = System.nanoTime(); + List indexes = new ArrayList<>(); + Closer closer = Closer.create(); + try { + for (FireHydrant fireHydrant : sink) { + Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); + final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex(); + log.debug("Segment[%s] adding hydrant[%s]", identifier, fireHydrant); + indexes.add(queryableIndex); + closer.register(segmentAndCloseable.rhs); + } + + mergedFile = indexMerger.mergeQueryableIndex( + indexes, + schema.getGranularitySpec().isRollup(), + schema.getAggregators(), + schema.getDimensionsSpec(), + mergedTarget, + tuningConfig.getIndexSpec(), + tuningConfig.getSegmentWriteOutMediumFactory(), + tuningConfig.getMaxColumnsToMerge() + ); + + mergeFinishTime = System.nanoTime(); + + log.debug("Segment[%s] built in %,dms.", identifier, (mergeFinishTime - startTime) / 1000000); + } + catch (Throwable t) { + throw closer.rethrow(t); + } + finally { + closer.close(); + } + + // Retry pushing segments because uploading to deep storage might fail especially for cloud storage types + final DataSegment segment = RetryUtils.retry( + // The appenderator is currently being used for the local indexing task and the Kafka indexing task. For the + // Kafka indexing task, pushers must use unique file paths in deep storage in order to maintain exactly-once + // semantics. + () -> dataSegmentPusher.push( + mergedFile, + sink.getSegment() + .withDimensions(IndexMerger.getMergedDimensionsFromQueryableIndexes( + indexes, + schema.getDimensionsSpec() + )), + useUniquePath + ), + exception -> exception instanceof Exception, + 5 + ); + + // Drop the queryable indexes behind the hydrants... they are not needed anymore and their + // mapped file references + // can generate OOMs during merge if enough of them are held back... + for (FireHydrant fireHydrant : sink) { + fireHydrant.swapSegment(null); + } + + final long pushFinishTime = System.nanoTime(); + + objectMapper.writeValue(descriptorFile, segment); + + log.info( + "Segment[%s] of %,d bytes " + + "built from %d incremental persist(s) in %,dms; " + + "pushed to deep storage in %,dms. " + + "Load spec is: %s", + identifier, + segment.getSize(), + indexes.size(), + (mergeFinishTime - startTime) / 1000000, + (pushFinishTime - mergeFinishTime) / 1000000, + objectMapper.writeValueAsString(segment.getLoadSpec()) + ); + + return segment; + } + catch (Exception e) { + metrics.incrementFailedHandoffs(); + log.warn(e, "Failed to push merged index for segment[%s].", identifier); + throw new RuntimeException(e); + } + } + + @Override + public void close() + { + if (!closed.compareAndSet(false, true)) { + log.debug("Appenderator already closed, skipping close() call."); + return; + } + + log.debug("Shutting down..."); + + final List> futures = new ArrayList<>(); + for (Map.Entry entry : sinks.entrySet()) { + futures.add(abandonSegment(entry.getKey(), entry.getValue(), false)); + } + + try { + Futures.allAsList(futures).get(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + log.warn(e, "Interrupted during close()"); + } + catch (ExecutionException e) { + log.warn(e, "Unable to abandon existing segments during close()"); + } + + try { + shutdownExecutors(); + Preconditions.checkState( + persistExecutor == null || persistExecutor.awaitTermination(365, TimeUnit.DAYS), + "persistExecutor not terminated" + ); + Preconditions.checkState( + pushExecutor == null || pushExecutor.awaitTermination(365, TimeUnit.DAYS), + "pushExecutor not terminated" + ); + Preconditions.checkState( + intermediateTempExecutor == null || intermediateTempExecutor.awaitTermination(365, TimeUnit.DAYS), + "intermediateTempExecutor not terminated" + ); + persistExecutor = null; + pushExecutor = null; + intermediateTempExecutor = null; + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ISE("Failed to shutdown executors during close()"); + } + + // Only unlock if executors actually shut down. + unlockBasePersistDirectory(); + } + + /** + * Unannounce the segments and wait for outstanding persists to finish. + * Do not unlock base persist dir as we are not waiting for push executor to shut down + * relying on current JVM to shutdown to not cause any locking problem if the task is restored. + * In case when task is restored and current task is still active because of push executor (which it shouldn't be + * since push executor starts daemon threads) then the locking should fail and new task should fail to start. + * This also means that this method should only be called when task is shutting down. + */ + @Override + public void closeNow() + { + if (!closed.compareAndSet(false, true)) { + log.debug("Appenderator already closed, skipping closeNow() call."); + return; + } + + log.debug("Shutting down immediately..."); + for (Map.Entry entry : sinks.entrySet()) { + try { + segmentAnnouncer.unannounceSegment(entry.getValue().getSegment()); + } + catch (Exception e) { + log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) + .addData("identifier", entry.getKey().toString()) + .emit(); + } + } + try { + shutdownExecutors(); + // We don't wait for pushExecutor to be terminated. See Javadoc for more details. + Preconditions.checkState( + persistExecutor == null || persistExecutor.awaitTermination(365, TimeUnit.DAYS), + "persistExecutor not terminated" + ); + Preconditions.checkState( + intermediateTempExecutor == null || intermediateTempExecutor.awaitTermination(365, TimeUnit.DAYS), + "intermediateTempExecutor not terminated" + ); + persistExecutor = null; + intermediateTempExecutor = null; + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ISE("Failed to shutdown executors during close()"); + } + } + + private void lockBasePersistDirectory() + { + if (basePersistDirLock == null) { + try { + basePersistDirLockChannel = FileChannel.open( + computeLockFile().toPath(), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE + ); + + basePersistDirLock = basePersistDirLockChannel.tryLock(); + if (basePersistDirLock == null) { + throw new ISE("Cannot acquire lock on basePersistDir: %s", computeLockFile()); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + private void unlockBasePersistDirectory() + { + try { + if (basePersistDirLock != null) { + basePersistDirLock.release(); + basePersistDirLockChannel.close(); + basePersistDirLock = null; + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + private void initializeExecutors() + { + final int maxPendingPersists = tuningConfig.getMaxPendingPersists(); + + if (persistExecutor == null) { + // use a blocking single threaded executor to throttle the firehose when write to disk is slow + persistExecutor = MoreExecutors.listeningDecorator( + Execs.newBlockingSingleThreaded( + "[" + StringUtils.encodeForFormat(myId) + "]-appenderator-persist", + maxPendingPersists + ) + ); + } + + if (pushExecutor == null) { + // use a blocking single threaded executor to throttle the firehose when write to disk is slow + pushExecutor = MoreExecutors.listeningDecorator( + Execs.newBlockingSingleThreaded("[" + StringUtils.encodeForFormat(myId) + "]-appenderator-merge", 1) + ); + } + + if (intermediateTempExecutor == null) { + // use single threaded executor with SynchronousQueue so that all abandon operations occur sequentially + intermediateTempExecutor = MoreExecutors.listeningDecorator( + Execs.newBlockingSingleThreaded("[" + StringUtils.encodeForFormat(myId) + "]-appenderator-abandon", 0) + ); + } + } + + private void shutdownExecutors() + { + if (persistExecutor != null) { + persistExecutor.shutdownNow(); + } + + if (pushExecutor != null) { + pushExecutor.shutdownNow(); + } + + if (intermediateTempExecutor != null) { + intermediateTempExecutor.shutdownNow(); + } + } + + private void resetNextFlush() + { + nextFlush = DateTimes.nowUtc().plus(tuningConfig.getIntermediatePersistPeriod()).getMillis(); + } + + /** + * Populate "sinks" and "sinkTimeline" with committed segments, and announce them with the segmentAnnouncer. + * + * @return persisted commit metadata + */ + private Object bootstrapSinksFromDisk() + { + Preconditions.checkState(sinks.isEmpty(), "Already bootstrapped?!"); + + final File baseDir = tuningConfig.getBasePersistDirectory(); + if (!baseDir.exists()) { + return null; + } + + final File[] files = baseDir.listFiles(); + if (files == null) { + return null; + } + + + final Committed committed; + File commitFile = null; + try { + commitLock.lock(); + commitFile = computeCommitFile(); + if (commitFile.exists()) { + committed = objectMapper.readValue(commitFile, Committed.class); + } else { + committed = Committed.nil(); + } + } + catch (Exception e) { + throw new ISE(e, "Failed to read commitFile: %s", commitFile); + } + finally { + commitLock.unlock(); + } + + int rowsSoFar = 0; + + if (committed.equals(Committed.nil())) { + log.debug("No previously committed metadata."); + } else { + log.info( + "Loading partially-persisted segments[%s] from[%s] with commit metadata: %s", + String.join(", ", committed.getHydrants().keySet()), + baseDir, + committed.getMetadata() + ); + } + + for (File sinkDir : files) { + final File identifierFile = new File(sinkDir, IDENTIFIER_FILE_NAME); + if (!identifierFile.isFile()) { + // No identifier in this sinkDir; it must not actually be a sink directory. Skip it. + continue; + } + + try { + final SegmentIdWithShardSpec identifier = objectMapper.readValue( + new File(sinkDir, "identifier.json"), + SegmentIdWithShardSpec.class + ); + + final int committedHydrants = committed.getCommittedHydrants(identifier.toString()); + + if (committedHydrants <= 0) { + log.info("Removing uncommitted segment at [%s].", sinkDir); + FileUtils.deleteDirectory(sinkDir); + continue; + } + + // To avoid reading and listing of "merged" dir and other special files + final File[] sinkFiles = sinkDir.listFiles( + (dir, fileName) -> !(Ints.tryParse(fileName) == null) + ); + + Arrays.sort( + sinkFiles, + (o1, o2) -> Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName())) + ); + + List hydrants = new ArrayList<>(); + for (File hydrantDir : sinkFiles) { + final int hydrantNumber = Integer.parseInt(hydrantDir.getName()); + + if (hydrantNumber >= committedHydrants) { + log.info("Removing uncommitted partial segment at [%s]", hydrantDir); + FileUtils.deleteDirectory(hydrantDir); + } else { + log.debug("Loading previously persisted partial segment at [%s]", hydrantDir); + if (hydrantNumber != hydrants.size()) { + throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir); + } + + hydrants.add( + new FireHydrant( + new QueryableIndexSegment(indexIO.loadIndex(hydrantDir), identifier.asSegmentId()), + hydrantNumber + ) + ); + } + } + + // Make sure we loaded enough hydrants. + if (committedHydrants != hydrants.size()) { + throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir); + } + + Sink currSink = new Sink( + identifier.getInterval(), + schema, + identifier.getShardSpec(), + identifier.getVersion(), + tuningConfig.getAppendableIndexSpec(), + tuningConfig.getMaxRowsInMemory(), + maxBytesTuningConfig, + null, + hydrants + ); + rowsSoFar += currSink.getNumRows(); + sinks.put(identifier, currSink); + segmentAnnouncer.announceSegment(currSink.getSegment()); + } + catch (IOException e) { + log.makeAlert(e, "Problem loading sink[%s] from disk.", schema.getDataSource()) + .addData("sinkDir", sinkDir) + .emit(); + } + } + + // Make sure we loaded all committed sinks. + final Set loadedSinks = Sets.newHashSet( + Iterables.transform(sinks.keySet(), SegmentIdWithShardSpec::toString) + ); + final Set missingSinks = Sets.difference(committed.getHydrants().keySet(), loadedSinks); + if (!missingSinks.isEmpty()) { + throw new ISE("Missing committed sinks [%s]", Joiner.on(", ").join(missingSinks)); + } + + totalRows.set(rowsSoFar); + return committed.getMetadata(); + } + + + private List getPersistedidentifierPaths() + { + + ArrayList retVal = new ArrayList<>(); + + final File baseDir = tuningConfig.getBasePersistDirectory(); + if (!baseDir.exists()) { + return null; + } + + final File[] files = baseDir.listFiles(); + if (files == null) { + return null; + } + + for (File sinkDir : files) { + final File identifierFile = new File(sinkDir, IDENTIFIER_FILE_NAME); + if (!identifierFile.isFile()) { + // No identifier in this sinkDir; it must not actually be a sink directory. Skip it. + continue; + } + retVal.add(sinkDir); + } + + return retVal; + } + + Pair getIdentifierAndSinkForPersistedFile(File identifierPath) + { + + try { + final SegmentIdWithShardSpec identifier = objectMapper.readValue( + new File(identifierPath, "identifier.json"), + SegmentIdWithShardSpec.class + ); + + // To avoid reading and listing of "merged" dir and other special files + final File[] sinkFiles = identifierPath.listFiles( + (dir, fileName) -> !(Ints.tryParse(fileName) == null) + ); + + Arrays.sort( + sinkFiles, + (o1, o2) -> Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName())) + ); + + List hydrants = new ArrayList<>(); + for (File hydrantDir : sinkFiles) { + final int hydrantNumber = Integer.parseInt(hydrantDir.getName()); + + log.debug("Loading previously persisted partial segment at [%s]", hydrantDir); + if (hydrantNumber != hydrants.size()) { + throw new ISE("Missing hydrant [%,d] in identifier [%s].", hydrants.size(), identifier); + } + + hydrants.add( + new FireHydrant( + new QueryableIndexSegment(indexIO.loadIndex(hydrantDir), identifier.asSegmentId()), + hydrantNumber + ) + ); + } + + Sink currSink = new Sink( + identifier.getInterval(), + schema, + identifier.getShardSpec(), + identifier.getVersion(), + tuningConfig.getAppendableIndexSpec(), + tuningConfig.getMaxRowsInMemory(), + maxBytesTuningConfig, + null, + hydrants + ); + currSink.finishWriting(); // this sink is not writable + //sinks.put(identifier, currSink); + return new Pair<>(identifier, currSink); + } + catch (IOException e) { + log.makeAlert(e, "Problem loading sink[%s] from disk.", schema.getDataSource()) + .addData("identifier path", identifierPath) + .emit(); + } + return null; + } + + private ListenableFuture abandonSegment( + final SegmentIdWithShardSpec identifier, + final Sink sink, + final boolean removeOnDiskData + ) + { + // Ensure no future writes will be made to this sink. + if (sink.finishWriting()) { + // Decrement this sink's rows from the counters. we only count active sinks so that we don't double decrement, + // i.e. those that haven't been persisted for *InMemory counters, or pushed to deep storage for the total counter. + rowsCurrentlyInMemory.addAndGet(-sink.getNumRowsInMemory()); + bytesCurrentlyInMemory.addAndGet(-sink.getBytesInMemory()); + bytesCurrentlyInMemory.addAndGet(-calculateSinkMemoryInUsed(sink)); + for (FireHydrant hydrant : sink) { + // Decrement memory used by all Memory Mapped Hydrant + if (!hydrant.equals(sink.getCurrHydrant())) { + bytesCurrentlyInMemory.addAndGet(-calculateMMappedHydrantMemoryInUsed(hydrant)); + } + } + totalRows.addAndGet(-sink.getNumRows()); + } + + // Mark this identifier as dropping, so no future push tasks will pick it up. + //droppingSinks.add(identifier); + + // Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread. + return Futures.transform( + pushBarrier(), + new Function() + { + @Nullable + @Override + public Void apply(@Nullable Object input) + { + if (!sinks.remove(identifier, sink)) { + log.error("Sink for segment[%s] no longer valid, not abandoning.", identifier); + return null; + } + + metrics.setSinkCount(sinks.size()); + + if (removeOnDiskData) { + // Remove this segment from the committed list. This must be done from the persist thread. + log.debug("Removing commit metadata for segment[%s].", identifier); + try { + commitLock.lock(); + final Committed oldCommit = readCommit(); + if (oldCommit != null) { + writeCommit(oldCommit.without(identifier.toString())); + } + } + catch (Exception e) { + log.makeAlert(e, "Failed to update committed segments[%s]", schema.getDataSource()) + .addData("identifier", identifier.toString()) + .emit(); + throw new RuntimeException(e); + } + finally { + commitLock.unlock(); + } + } + + // Unannounce the segment. + try { + segmentAnnouncer.unannounceSegment(sink.getSegment()); + } + catch (Exception e) { + log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) + .addData("identifier", identifier.toString()) + .emit(); + } + + //droppingSinks.remove(identifier); + for (FireHydrant hydrant : sink) { + if (cache != null) { + cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant)); + } + hydrant.swapSegment(null); + } + + if (removeOnDiskData) { + removeDirectory(computePersistDir(identifier)); + } + + log.info("Dropped segment[%s].", identifier); + + return null; + } + }, + // use persistExecutor to make sure that all the pending persists completes before + // starting to abandon segments + persistExecutor + ); + } + + private Committed readCommit() throws IOException + { + final File commitFile = computeCommitFile(); + if (commitFile.exists()) { + // merge current hydrants with existing hydrants + return objectMapper.readValue(commitFile, Committed.class); + } else { + return null; + } + } + + private void writeCommit(Committed newCommit) throws IOException + { + final File commitFile = computeCommitFile(); + objectMapper.writeValue(commitFile, newCommit); + } + + private File computeCommitFile() + { + return new File(tuningConfig.getBasePersistDirectory(), "commit.json"); + } + + private File computeLockFile() + { + return new File(tuningConfig.getBasePersistDirectory(), ".lock"); + } + + private File computePersistDir(SegmentIdWithShardSpec identifier) + { + return new File(tuningConfig.getBasePersistDirectory(), identifier.toString()); + } + + private File computeIdentifierFile(SegmentIdWithShardSpec identifier) + { + return new File(computePersistDir(identifier), IDENTIFIER_FILE_NAME); + } + + private File computeDescriptorFile(SegmentIdWithShardSpec identifier) + { + return new File(computePersistDir(identifier), "descriptor.json"); + } + + private File createPersistDirIfNeeded(SegmentIdWithShardSpec identifier) throws IOException + { + final File persistDir = computePersistDir(identifier); + org.apache.commons.io.FileUtils.forceMkdir(persistDir); + + objectMapper.writeValue(computeIdentifierFile(identifier), identifier); + + return persistDir; + } + + /** + * Persists the given hydrant and returns the number of rows persisted. Must only be called in the single-threaded + * persistExecutor. + * + * @param indexToPersist hydrant to persist + * @param identifier the segment this hydrant is going to be part of + * @return the number of rows persisted + */ + private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec identifier) + { + synchronized (indexToPersist) { + if (indexToPersist.hasSwapped()) { + log.info( + "Segment[%s] hydrant[%s] already swapped. Ignoring request to persist.", + identifier, + indexToPersist + ); + return 0; + } + + log.debug("Segment[%s], persisting Hydrant[%s]", identifier, indexToPersist); + + try { + final long startTime = System.nanoTime(); + int numRows = indexToPersist.getIndex().size(); + + final File persistDir = createPersistDirIfNeeded(identifier); + indexMerger.persist( + indexToPersist.getIndex(), + identifier.getInterval(), + new File(persistDir, String.valueOf(indexToPersist.getCount())), + tuningConfig.getIndexSpecForIntermediatePersists(), + tuningConfig.getSegmentWriteOutMediumFactory() + ); + + log.info( + "Flushed in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).", + indexToPersist.getSegmentId(), + indexToPersist.getCount(), + (System.nanoTime() - startTime) / 1000000, + numRows + ); + + indexToPersist.swapSegment(null); + + return numRows; + } + catch (IOException e) { + log.makeAlert("Incremental persist failed") + .addData("segment", identifier.toString()) + .addData("dataSource", schema.getDataSource()) + .addData("count", indexToPersist.getCount()) + .emit(); + + throw new RuntimeException(e); + } + } + } + + private void removeDirectory(final File target) + { + if (target.exists()) { + try { + FileUtils.deleteDirectory(target); + } + catch (Exception e) { + log.makeAlert(e, "Failed to remove directory[%s]", schema.getDataSource()) + .addData("file", target) + .emit(); + } + } + } + + private int calculateMMappedHydrantMemoryInUsed(FireHydrant hydrant) + { + if (skipBytesInMemoryOverheadCheck) { + return 0; + } + // These calculations are approximated from actual heap dumps. + // Memory footprint includes count integer in FireHydrant, shorts in ReferenceCountingSegment, + // Objects in SimpleQueryableIndex (such as SmooshedFileMapper, each ColumnHolder in column map, etc.) + int total; + total = Integer.BYTES + (4 * Short.BYTES) + ROUGH_OVERHEAD_PER_HYDRANT; + return total; + } + + private int calculateSinkMemoryInUsed(Sink sink) + { + if (skipBytesInMemoryOverheadCheck) { + return 0; + } + // Rough estimate of memory footprint of empty Sink based on actual heap dumps + return ROUGH_OVERHEAD_PER_SINK; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/RealtimeAppenderator.java similarity index 99% rename from server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java rename to server/src/main/java/org/apache/druid/segment/realtime/appenderator/RealtimeAppenderator.java index e6cd9c52aa92..4da8da8f3aa3 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/RealtimeAppenderator.java @@ -104,7 +104,7 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; -public class AppenderatorImpl implements Appenderator +public class RealtimeAppenderator implements Appenderator { // Rough estimate of memory footprint of a ColumnHolder based on actual heap dumps public static final int ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER = 1000; @@ -115,7 +115,7 @@ public class AppenderatorImpl implements Appenderator // Rough estimate of memory footprint of empty FireHydrant based on actual heap dumps public static final int ROUGH_OVERHEAD_PER_HYDRANT = 1000; - private static final EmittingLogger log = new EmittingLogger(AppenderatorImpl.class); + private static final EmittingLogger log = new EmittingLogger(RealtimeAppenderator.class); private static final int WARN_DELAY = 1000; private static final String IDENTIFIER_FILE_NAME = "identifier.json"; @@ -188,7 +188,7 @@ public class AppenderatorImpl implements Appenderator * It is used by UnifiedIndexerAppenderatorsManager which allows queries on data associated with multiple * Appenderators. */ - AppenderatorImpl( + RealtimeAppenderator( String id, DataSchema schema, AppenderatorConfig tuningConfig, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index 3780c3735753..902771de3951 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -176,7 +176,7 @@ public Appenderator createRealtimeAppenderatorForTask( DatasourceBundle::new ); - Appenderator appenderator = new AppenderatorImpl( + Appenderator appenderator = new RealtimeAppenderator( taskId, schema, rewriteAppenderatorConfigMemoryLimits(config), @@ -494,7 +494,7 @@ private IndexMerger wrapIndexMerger(IndexMerger baseMerger) /** * This wrapper around IndexMerger limits concurrent calls to the merge/persist methods used by - * {@link AppenderatorImpl} with a shared executor service. Merge/persist methods that are not used by + * {@link RealtimeAppenderator} with a shared executor service. Merge/persist methods that are not used by * AppenderatorImpl will throw an exception if called. */ public static class LimitedPoolIndexMerger implements IndexMerger diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java index 78210d3bc191..aef5a9c81b3a 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java @@ -103,8 +103,9 @@ public void testSimpleIngestion() throws Exception // getDataSource Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource()); - InputRow[] rows = new InputRow[] {AppenderatorTest.ir("2000", "foo", 1), - AppenderatorTest.ir("2000", "bar", 2), AppenderatorTest.ir("2000", "qux", 4)}; + InputRow[] rows = new InputRow[] { + RealtimeAppenderatorTest.ir("2000", "foo", 1), + RealtimeAppenderatorTest.ir("2000", "bar", 2), RealtimeAppenderatorTest.ir("2000", "qux", 4)}; // add Assert.assertEquals(1, plumber.add(rows[0], null).getRowCount()); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index 491f4b26d702..debda154204b 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -171,13 +171,13 @@ public int columnCacheSizeBytes() "A", new LinearShardSpec(0) ); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); - appenderator.add(identifier, AppenderatorTest.ir("2000", "bar", 1), Suppliers.ofInstance(Committers.nil())); - Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); - appenderator.add(identifier, AppenderatorTest.ir("2000", "baz", 1), Suppliers.ofInstance(Committers.nil())); - Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + appenderator.add(identifier, RealtimeAppenderatorTest.ir("2000", "bar", 1), Suppliers.ofInstance(Committers.nil())); + Assert.assertEquals(1, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + appenderator.add(identifier, RealtimeAppenderatorTest.ir("2000", "baz", 1), Suppliers.ofInstance(Committers.nil())); + Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.close(); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); } finally { appenderator.close(); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/RealtimeAppenderatorTest.java similarity index 87% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/RealtimeAppenderatorTest.java index 0c1285d06ed6..4fcd8d4ea96a 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/RealtimeAppenderatorTest.java @@ -61,7 +61,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; -public class AppenderatorTest extends InitializedNullHandlingTest +public class RealtimeAppenderatorTest extends InitializedNullHandlingTest { private static final List IDENTIFIERS = ImmutableList.of( si("2000/2001", "A", 0), @@ -193,15 +193,15 @@ public void run() int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; Assert.assertEquals( 182 + nullHandlingOverhead, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier); Assert.assertEquals( 182 + nullHandlingOverhead, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); appenderator.close(); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); } } @@ -243,14 +243,14 @@ public void run() appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier); //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 54 (dimsKeySize) = 182 int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; - Assert.assertEquals(182 + nullHandlingOverhead, ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()); + Assert.assertEquals(182 + nullHandlingOverhead, ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier); Assert.assertEquals( 364 + 2 * nullHandlingOverhead, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); appenderator.close(); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); } } @@ -285,38 +285,38 @@ public void run() //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; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = 1 * AppenderatorImpl.ROUGH_OVERHEAD_PER_SINK; + int sinkSizeOverhead = 1 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_SINK; // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); // We do multiple more adds to the same sink to cause persist. for (int i = 0; i < 53; i++) { appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar_" + i, 1), committerSupplier); } - sinkSizeOverhead = 1 * AppenderatorImpl.ROUGH_OVERHEAD_PER_SINK; + sinkSizeOverhead = 1 * RealtimeAppenderator.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. // currHydrant in the sink has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); // Mapped index size is the memory still needed after we persisted indexes. Note that the segments have // 1 dimension columns, 2 metric column, 1 time column. - int mappedIndexSize = 1012 + (2 * AppenderatorImpl.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + - AppenderatorImpl.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + - AppenderatorImpl.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER; + int mappedIndexSize = 1012 + (2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + + RealtimeAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + + RealtimeAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER; Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); // Add a single row after persisted @@ -325,11 +325,11 @@ public void run() currentInMemoryIndexSize = 182 + nullHandlingOverhead; Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); // We do multiple more adds to the same sink to cause persist. @@ -342,21 +342,21 @@ public void run() // currHydrant in the sink has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); // Mapped index size is the memory still needed after we persisted indexes. Note that the segments have // 1 dimension columns, 2 metric column, 1 time column. However, we have two indexes now from the two pervious // persists. - mappedIndexSize = 2 * (1012 + (2 * AppenderatorImpl.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + - AppenderatorImpl.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + - AppenderatorImpl.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER); + mappedIndexSize = 2 * (1012 + (2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + + RealtimeAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + + RealtimeAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER); Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); appenderator.close(); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory()); } } @@ -429,13 +429,13 @@ public void run() // Expected 0 since we persisted after the add Assert.assertEquals( 0, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier); // Expected 0 since we persisted after the add Assert.assertEquals( 0, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); } } @@ -471,17 +471,17 @@ public void run() // Still under maxSizeInBytes after the add. Hence, we do not persist yet int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = 1 * AppenderatorImpl.ROUGH_OVERHEAD_PER_SINK; + int sinkSizeOverhead = 1 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_SINK; Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); // Close with row still in memory (no persist) appenderator.close(); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory()); } } @@ -518,19 +518,19 @@ public void run() //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; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = 2 * AppenderatorImpl.ROUGH_OVERHEAD_PER_SINK; + int sinkSizeOverhead = 2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_SINK; // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); Assert.assertEquals( (2 * currentInMemoryIndexSize) + sinkSizeOverhead, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); // We do multiple more adds to the same sink to cause persist. @@ -538,27 +538,27 @@ public void run() appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar_" + i, 1), committerSupplier); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar_" + i, 1), committerSupplier); } - sinkSizeOverhead = 2 * AppenderatorImpl.ROUGH_OVERHEAD_PER_SINK; + sinkSizeOverhead = 2 * RealtimeAppenderator.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. // currHydrant in the sink has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); // Mapped index size is the memory still needed after we persisted indexes. Note that the segments have // 1 dimension columns, 2 metric column, 1 time column. - int mappedIndexSize = 2 * (1012 + (2 * AppenderatorImpl.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + - AppenderatorImpl.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + - AppenderatorImpl.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER); + int mappedIndexSize = 2 * (1012 + (2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + + RealtimeAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + + RealtimeAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER); Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); // Add a single row after persisted to sink 0 @@ -567,29 +567,29 @@ public void run() currentInMemoryIndexSize = 182 + nullHandlingOverhead; Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( 0, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); // Now add a single row to sink 1 appenderator.add(IDENTIFIERS.get(1), ir("2000", "bob", 1), committerSupplier); Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); Assert.assertEquals( (2 * currentInMemoryIndexSize) + sinkSizeOverhead + mappedIndexSize, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); // We do multiple more adds to the both sink to cause persist. @@ -603,25 +603,25 @@ public void run() // currHydrant in the sink has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( currentInMemoryIndexSize, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); // Mapped index size is the memory still needed after we persisted indexes. Note that the segments have // 1 dimension columns, 2 metric column, 1 time column. However, we have two indexes now from the two pervious // persists. - mappedIndexSize = 2 * (2 * (1012 + (2 * AppenderatorImpl.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + - AppenderatorImpl.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + - AppenderatorImpl.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER)); + mappedIndexSize = 2 * (2 * (1012 + (2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + + RealtimeAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + + RealtimeAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER)); Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); appenderator.close(); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory()); } } @@ -650,26 +650,26 @@ public void run() }; }; - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.startJob(); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier); //we still calculate the size even when ignoring it to make persist decision int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; Assert.assertEquals( 182 + nullHandlingOverhead, - ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); - Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(1, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier); - int sinkSizeOverhead = 2 * AppenderatorImpl.ROUGH_OVERHEAD_PER_SINK; + int sinkSizeOverhead = 2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_SINK; Assert.assertEquals( (364 + 2 * nullHandlingOverhead) + sinkSizeOverhead, - ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory() + ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() ); - Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.close(); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); } } @@ -703,23 +703,23 @@ public void run() } }; - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.startJob(); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier); - Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(1, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier); - Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier); - Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "baz", 1), committerSupplier); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 1), committerSupplier); - Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(1, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 1), committerSupplier); - Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.persistAll(committerSupplier.get()); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.close(); } } @@ -749,23 +749,23 @@ public void run() }; }; - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.startJob(); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier, false); - Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(1, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier, false); - Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier, false); - Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "baz", 1), committerSupplier, false); - Assert.assertEquals(3, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(3, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 1), committerSupplier, false); - Assert.assertEquals(4, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(4, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 1), committerSupplier, false); - Assert.assertEquals(5, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(5, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.persistAll(committerSupplier.get()); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); appenderator.close(); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java index 408aa97e400b..4f9cd3c34158 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java @@ -545,11 +545,5 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable Date: Fri, 21 May 2021 17:00:40 -0700 Subject: [PATCH 02/47] Move BatchAppenderatorDriverTest to indexing service... note that we had to put the sink back in sinks in mergeandpush since the persistent data needs to be dropped and the sink is required for that --- .../BatchAppenderatorDriverTest.java | 54 ++++++++++++++-- .../appenderator/TestUsedSegmentChecker.java | 61 +++++++++++++++++++ .../appenderator/BaseAppenderatorDriver.java | 7 +-- .../appenderator/BatchAppenderator.java | 22 +------ 4 files changed, 117 insertions(+), 27 deletions(-) rename {server/src/test/java/org/apache/druid/segment/realtime => indexing-service/src/test/java/org/apache/druid/indexing}/appenderator/BatchAppenderatorDriverTest.java (77%) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/appenderator/TestUsedSegmentChecker.java diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorDriverTest.java similarity index 77% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorDriverTest.java index 024bfa053913..b319f1f5b85f 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorDriverTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.appenderator; +package org.apache.druid.indexing.appenderator; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -29,21 +29,31 @@ 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.java.util.common.granularity.Granularity; import org.apache.druid.segment.loading.DataSegmentKiller; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver.SegmentsForSequence; +import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; +import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.segment.realtime.appenderator.SegmentWithState; import org.apache.druid.segment.realtime.appenderator.SegmentWithState.SegmentState; -import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriverTest.TestSegmentAllocator; +import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; +import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; +import org.joda.time.DateTime; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Collectors; @@ -73,7 +83,7 @@ public class BatchAppenderatorDriverTest extends EasyMockSupport ); private SegmentAllocator allocator; - private AppenderatorTester appenderatorTester; + private BatchAppenderatorTester appenderatorTester; private BatchAppenderatorDriver driver; private DataSegmentKiller dataSegmentKiller; @@ -84,7 +94,7 @@ public class BatchAppenderatorDriverTest extends EasyMockSupport @Before public void setup() { - appenderatorTester = new AppenderatorTester(MAX_ROWS_IN_MEMORY); + appenderatorTester = new BatchAppenderatorTester(MAX_ROWS_IN_MEMORY); allocator = new TestSegmentAllocator(DATA_SOURCE, Granularities.HOUR); dataSegmentKiller = createStrictMock(DataSegmentKiller.class); driver = new BatchAppenderatorDriver( @@ -199,4 +209,40 @@ static TransactionalSegmentPublisher makeOkPublisher() { return (segmentsToBeOverwritten, segmentsToBeDropped, segmentsToPublish, commitMetadata) -> SegmentPublishResult.ok(ImmutableSet.of()); } + + static class TestSegmentAllocator implements SegmentAllocator + { + private final String dataSource; + private final Granularity granularity; + private final Map counters = new HashMap<>(); + + public TestSegmentAllocator(String dataSource, Granularity granularity) + { + this.dataSource = dataSource; + this.granularity = granularity; + } + + @Override + public SegmentIdWithShardSpec allocate( + final InputRow row, + final String sequenceName, + final String previousSegmentId, + final boolean skipSegmentLineageCheck + ) + { + synchronized (counters) { + DateTime dateTimeTruncated = granularity.bucketStart(row.getTimestamp()); + final long timestampTruncated = dateTimeTruncated.getMillis(); + counters.putIfAbsent(timestampTruncated, new AtomicInteger()); + final int partitionNum = counters.get(timestampTruncated).getAndIncrement(); + return new SegmentIdWithShardSpec( + dataSource, + granularity.bucket(dateTimeTruncated), + VERSION, + new NumberedShardSpec(partitionNum, 0) + ); + } + } + } + } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/TestUsedSegmentChecker.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/TestUsedSegmentChecker.java new file mode 100644 index 000000000000..5a89a81cfff6 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/TestUsedSegmentChecker.java @@ -0,0 +1,61 @@ +/* + * 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.appenderator; + +import com.google.common.collect.Ordering; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineObjectHolder; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.PartitionChunk; + +import java.util.HashSet; +import java.util.Set; + +public class TestUsedSegmentChecker implements UsedSegmentChecker +{ + private final BatchAppenderatorTester appenderatorTester; + + public TestUsedSegmentChecker(BatchAppenderatorTester appenderatorTester) + { + this.appenderatorTester = appenderatorTester; + } + + @Override + public Set findUsedSegments(Set identifiers) + { + final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + VersionedIntervalTimeline.addSegments(timeline, appenderatorTester.getPushedSegments().iterator()); + + final Set retVal = new HashSet<>(); + for (SegmentIdWithShardSpec identifier : identifiers) { + for (TimelineObjectHolder holder : timeline.lookup(identifier.getInterval())) { + for (PartitionChunk chunk : holder.getObject()) { + if (identifiers.contains(SegmentIdWithShardSpec.fromDataSegment(chunk.getObject()))) { + retVal.add(chunk.getObject()); + } + } + } + } + + return retVal; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java index 1a96f4e13adc..c0d646820d66 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java @@ -172,7 +172,7 @@ List getAllSegments() /** * Allocated segments for a sequence */ - static class SegmentsForSequence + public static class SegmentsForSequence { // Interval Start millis -> List of Segments for this interval // there might be multiple segments for a start interval, for example one segment @@ -215,7 +215,7 @@ SegmentsOfInterval get(long timestamp) return intervalToSegmentStates.get(timestamp); } - Stream allSegmentStateStream() + public Stream allSegmentStateStream() { return intervalToSegmentStates .values() @@ -260,8 +260,7 @@ Stream getAllSegmentsOfInterval() ); } - @VisibleForTesting - Map getSegments() + public Map getSegments() { return segments; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 58010cc7fc15..8d44afbde397 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -373,25 +373,6 @@ public AppenderatorAddResult add( throw new RuntimeException(errorMessage); } - -// Futures.addCallback( -// persistAll(null), -// new FutureCallback() -// { -// @Override -// public void onSuccess(@Nullable Object result) -// { -// // do nothing -// } -// -// @Override -// public void onFailure(Throwable t) -// { -// persistError = t; -// } -// } -// ); - persistAllAndClear(); } else { @@ -837,6 +818,9 @@ private DataSegment mergeAndPush( // Drop the queryable indexes behind the hydrants... they are not needed anymore and their // mapped file references // can generate OOMs during merge if enough of them are held back... + // we need to put the sink back so downstream code (i.e. drop segment) can process + // agfixme: Maybe we ought to keep the sink references all along... + sinks.put(identifier,sink); for (FireHydrant fireHydrant : sink) { fireHydrant.swapSegment(null); } From e3946fc14c019ec4d9bef6843dd2818dcc39f06b Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Mon, 24 May 2021 13:12:06 -0700 Subject: [PATCH 03/47] Remove sinks from memory and clean up intermediate persists dirs manually after sink has been merged --- .../realtime/appenderator/BatchAppenderator.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 8d44afbde397..1e2bfd912bf0 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -818,16 +818,18 @@ private DataSegment mergeAndPush( // Drop the queryable indexes behind the hydrants... they are not needed anymore and their // mapped file references // can generate OOMs during merge if enough of them are held back... - // we need to put the sink back so downstream code (i.e. drop segment) can process - // agfixme: Maybe we ought to keep the sink references all along... - sinks.put(identifier,sink); + // agfixme: Since we cannot keep sinks due to memory growth then we have to add the sink metadata table and keep it up to date + //sinks.put(identifier,sink); for (FireHydrant fireHydrant : sink) { fireHydrant.swapSegment(null); } + // cleanup, sink no longer needed + removeDirectory(computePersistDir(identifier)); + final long pushFinishTime = System.nanoTime(); - objectMapper.writeValue(descriptorFile, segment); + //objectMapper.writeValue(descriptorFile, segment); log.info( "Segment[%s] of %,d bytes " @@ -1477,6 +1479,7 @@ private void removeDirectory(final File target) if (target.exists()) { try { FileUtils.deleteDirectory(target); + log.info("Removed directory [%s]", target); } catch (Exception e) { log.makeAlert(e, "Failed to remove directory[%s]", schema.getDataSource()) From 4e8d57335ba8ba149156190b283e708048a5fdd4 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Mon, 24 May 2021 17:54:42 -0700 Subject: [PATCH 04/47] Changed name from RealtimeAppenderator to StreamAppenderator --- .../SeekableStreamIndexTaskTestBase.java | 4 +- .../realtime/appenderator/Appenderators.java | 2 +- ...enderator.java => StreamAppenderator.java} | 6 +- .../UnifiedIndexerAppenderatorsManager.java | 4 +- .../appenderator/AppenderatorPlumberTest.java | 4 +- ...DefaultOfflineAppenderatorFactoryTest.java | 12 +- ...rTest.java => StreamAppenderatorTest.java} | 164 +++++++++--------- 7 files changed, 98 insertions(+), 98 deletions(-) rename server/src/main/java/org/apache/druid/segment/realtime/appenderator/{RealtimeAppenderator.java => StreamAppenderator.java} (99%) rename server/src/test/java/org/apache/druid/segment/realtime/appenderator/{RealtimeAppenderatorTest.java => StreamAppenderatorTest.java} (87%) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index a15a0e5e0fc0..607001326871 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -79,7 +79,7 @@ import org.apache.druid.segment.column.DictionaryEncodedColumn; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.appenderator.RealtimeAppenderator; +import org.apache.druid.segment.realtime.appenderator.StreamAppenderator; import org.apache.druid.timeline.DataSegment; import org.apache.druid.utils.CompressionUtils; import org.assertj.core.api.Assertions; @@ -461,7 +461,7 @@ protected long countEvents(final Task task) protected void unlockAppenderatorBasePersistDirForTask(SeekableStreamIndexTask task) throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { - Method unlockBasePersistDir = ((RealtimeAppenderator) task.getAppenderator()) + Method unlockBasePersistDir = ((StreamAppenderator) task.getAppenderator()) .getClass() .getDeclaredMethod("unlockBasePersistDirectory"); unlockBasePersistDir.setAccessible(true); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 607c613aa9eb..56cb2f409efc 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -63,7 +63,7 @@ public static Appenderator createRealtime( ParseExceptionHandler parseExceptionHandler ) { - return new RealtimeAppenderator( + return new StreamAppenderator( id, schema, config, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/RealtimeAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java similarity index 99% rename from server/src/main/java/org/apache/druid/segment/realtime/appenderator/RealtimeAppenderator.java rename to server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index 4da8da8f3aa3..ce5ea255649b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/RealtimeAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -104,7 +104,7 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; -public class RealtimeAppenderator implements Appenderator +public class StreamAppenderator implements Appenderator { // Rough estimate of memory footprint of a ColumnHolder based on actual heap dumps public static final int ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER = 1000; @@ -115,7 +115,7 @@ public class RealtimeAppenderator implements Appenderator // Rough estimate of memory footprint of empty FireHydrant based on actual heap dumps public static final int ROUGH_OVERHEAD_PER_HYDRANT = 1000; - private static final EmittingLogger log = new EmittingLogger(RealtimeAppenderator.class); + private static final EmittingLogger log = new EmittingLogger(StreamAppenderator.class); private static final int WARN_DELAY = 1000; private static final String IDENTIFIER_FILE_NAME = "identifier.json"; @@ -188,7 +188,7 @@ public class RealtimeAppenderator implements Appenderator * It is used by UnifiedIndexerAppenderatorsManager which allows queries on data associated with multiple * Appenderators. */ - RealtimeAppenderator( + StreamAppenderator( String id, DataSchema schema, AppenderatorConfig tuningConfig, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index 902771de3951..d91d6d387b62 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -176,7 +176,7 @@ public Appenderator createRealtimeAppenderatorForTask( DatasourceBundle::new ); - Appenderator appenderator = new RealtimeAppenderator( + Appenderator appenderator = new StreamAppenderator( taskId, schema, rewriteAppenderatorConfigMemoryLimits(config), @@ -494,7 +494,7 @@ private IndexMerger wrapIndexMerger(IndexMerger baseMerger) /** * This wrapper around IndexMerger limits concurrent calls to the merge/persist methods used by - * {@link RealtimeAppenderator} with a shared executor service. Merge/persist methods that are not used by + * {@link StreamAppenderator} with a shared executor service. Merge/persist methods that are not used by * AppenderatorImpl will throw an exception if called. */ public static class LimitedPoolIndexMerger implements IndexMerger diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java index aef5a9c81b3a..24b697bd80c6 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java @@ -104,8 +104,8 @@ public void testSimpleIngestion() throws Exception Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource()); InputRow[] rows = new InputRow[] { - RealtimeAppenderatorTest.ir("2000", "foo", 1), - RealtimeAppenderatorTest.ir("2000", "bar", 2), RealtimeAppenderatorTest.ir("2000", "qux", 4)}; + StreamAppenderatorTest.ir("2000", "foo", 1), + StreamAppenderatorTest.ir("2000", "bar", 2), StreamAppenderatorTest.ir("2000", "qux", 4)}; // add Assert.assertEquals(1, plumber.add(rows[0], null).getRowCount()); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index debda154204b..85829fe07bf9 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -171,13 +171,13 @@ public int columnCacheSizeBytes() "A", new LinearShardSpec(0) ); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); - appenderator.add(identifier, RealtimeAppenderatorTest.ir("2000", "bar", 1), Suppliers.ofInstance(Committers.nil())); - Assert.assertEquals(1, ((RealtimeAppenderator) appenderator).getRowsInMemory()); - appenderator.add(identifier, RealtimeAppenderatorTest.ir("2000", "baz", 1), Suppliers.ofInstance(Committers.nil())); - Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); + appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "bar", 1), Suppliers.ofInstance(Committers.nil())); + Assert.assertEquals(1, ((StreamAppenderator) appenderator).getRowsInMemory()); + appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "baz", 1), Suppliers.ofInstance(Committers.nil())); + Assert.assertEquals(2, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.close(); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); } finally { appenderator.close(); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/RealtimeAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java similarity index 87% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/RealtimeAppenderatorTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java index 4fcd8d4ea96a..3dd0a69912a9 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/RealtimeAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java @@ -61,7 +61,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; -public class RealtimeAppenderatorTest extends InitializedNullHandlingTest +public class StreamAppenderatorTest extends InitializedNullHandlingTest { private static final List IDENTIFIERS = ImmutableList.of( si("2000/2001", "A", 0), @@ -193,15 +193,15 @@ public void run() int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; Assert.assertEquals( 182 + nullHandlingOverhead, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier); Assert.assertEquals( 182 + nullHandlingOverhead, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); appenderator.close(); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); } } @@ -243,14 +243,14 @@ public void run() appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier); //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 54 (dimsKeySize) = 182 int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; - Assert.assertEquals(182 + nullHandlingOverhead, ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory()); + Assert.assertEquals(182 + nullHandlingOverhead, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier); Assert.assertEquals( 364 + 2 * nullHandlingOverhead, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); appenderator.close(); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); } } @@ -285,38 +285,38 @@ public void run() //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; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = 1 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_SINK; + int sinkSizeOverhead = 1 * StreamAppenderator.ROUGH_OVERHEAD_PER_SINK; // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); // We do multiple more adds to the same sink to cause persist. for (int i = 0; i < 53; i++) { appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar_" + i, 1), committerSupplier); } - sinkSizeOverhead = 1 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_SINK; + sinkSizeOverhead = 1 * StreamAppenderator.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. // currHydrant in the sink has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); // Mapped index size is the memory still needed after we persisted indexes. Note that the segments have // 1 dimension columns, 2 metric column, 1 time column. - int mappedIndexSize = 1012 + (2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + - RealtimeAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + - RealtimeAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER; + int mappedIndexSize = 1012 + (2 * StreamAppenderator.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + + StreamAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + + StreamAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER; Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); // Add a single row after persisted @@ -325,11 +325,11 @@ public void run() currentInMemoryIndexSize = 182 + nullHandlingOverhead; Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); // We do multiple more adds to the same sink to cause persist. @@ -342,21 +342,21 @@ public void run() // currHydrant in the sink has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); // Mapped index size is the memory still needed after we persisted indexes. Note that the segments have // 1 dimension columns, 2 metric column, 1 time column. However, we have two indexes now from the two pervious // persists. - mappedIndexSize = 2 * (1012 + (2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + - RealtimeAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + - RealtimeAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER); + mappedIndexSize = 2 * (1012 + (2 * StreamAppenderator.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + + StreamAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + + StreamAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER); Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); appenderator.close(); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory()); } } @@ -429,13 +429,13 @@ public void run() // Expected 0 since we persisted after the add Assert.assertEquals( 0, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier); // Expected 0 since we persisted after the add Assert.assertEquals( 0, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); } } @@ -471,17 +471,17 @@ public void run() // Still under maxSizeInBytes after the add. Hence, we do not persist yet int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = 1 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_SINK; + int sinkSizeOverhead = 1 * StreamAppenderator.ROUGH_OVERHEAD_PER_SINK; Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); // Close with row still in memory (no persist) appenderator.close(); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory()); } } @@ -518,19 +518,19 @@ public void run() //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; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = 2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_SINK; + int sinkSizeOverhead = 2 * StreamAppenderator.ROUGH_OVERHEAD_PER_SINK; // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); Assert.assertEquals( (2 * currentInMemoryIndexSize) + sinkSizeOverhead, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); // We do multiple more adds to the same sink to cause persist. @@ -538,27 +538,27 @@ public void run() appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar_" + i, 1), committerSupplier); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar_" + i, 1), committerSupplier); } - sinkSizeOverhead = 2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_SINK; + sinkSizeOverhead = 2 * StreamAppenderator.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. // currHydrant in the sink has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); // Mapped index size is the memory still needed after we persisted indexes. Note that the segments have // 1 dimension columns, 2 metric column, 1 time column. - int mappedIndexSize = 2 * (1012 + (2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + - RealtimeAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + - RealtimeAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER); + int mappedIndexSize = 2 * (1012 + (2 * StreamAppenderator.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + + StreamAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + + StreamAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER); Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); // Add a single row after persisted to sink 0 @@ -567,29 +567,29 @@ public void run() currentInMemoryIndexSize = 182 + nullHandlingOverhead; Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( 0, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); // Now add a single row to sink 1 appenderator.add(IDENTIFIERS.get(1), ir("2000", "bob", 1), committerSupplier); Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); Assert.assertEquals( (2 * currentInMemoryIndexSize) + sinkSizeOverhead + mappedIndexSize, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); // We do multiple more adds to the both sink to cause persist. @@ -603,25 +603,25 @@ public void run() // currHydrant in the sink has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( currentInMemoryIndexSize, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); // Mapped index size is the memory still needed after we persisted indexes. Note that the segments have // 1 dimension columns, 2 metric column, 1 time column. However, we have two indexes now from the two pervious // persists. - mappedIndexSize = 2 * (2 * (1012 + (2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + - RealtimeAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + - RealtimeAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER)); + mappedIndexSize = 2 * (2 * (1012 + (2 * StreamAppenderator.ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + + StreamAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + + StreamAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER)); Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); appenderator.close(); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory()); } } @@ -650,26 +650,26 @@ public void run() }; }; - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.startJob(); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier); //we still calculate the size even when ignoring it to make persist decision int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; Assert.assertEquals( 182 + nullHandlingOverhead, - ((RealtimeAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); - Assert.assertEquals(1, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(1, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier); - int sinkSizeOverhead = 2 * RealtimeAppenderator.ROUGH_OVERHEAD_PER_SINK; + int sinkSizeOverhead = 2 * StreamAppenderator.ROUGH_OVERHEAD_PER_SINK; Assert.assertEquals( (364 + 2 * nullHandlingOverhead) + sinkSizeOverhead, - ((RealtimeAppenderator) appenderator).getBytesCurrentlyInMemory() + ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); - Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.close(); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); } } @@ -703,23 +703,23 @@ public void run() } }; - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.startJob(); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier); - Assert.assertEquals(1, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(1, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier); - Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier); - Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "baz", 1), committerSupplier); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 1), committerSupplier); - Assert.assertEquals(1, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(1, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 1), committerSupplier); - Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.persistAll(committerSupplier.get()); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.close(); } } @@ -749,23 +749,23 @@ public void run() }; }; - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.startJob(); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier, false); - Assert.assertEquals(1, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(1, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier, false); - Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier, false); - Assert.assertEquals(2, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(2, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "baz", 1), committerSupplier, false); - Assert.assertEquals(3, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(3, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 1), committerSupplier, false); - Assert.assertEquals(4, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(4, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 1), committerSupplier, false); - Assert.assertEquals(5, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(5, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.persistAll(committerSupplier.get()); - Assert.assertEquals(0, ((RealtimeAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.close(); } } From b20582f1823add404bdaad796634db688e3f0bb8 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Tue, 25 May 2021 13:12:55 -0700 Subject: [PATCH 05/47] Style --- .../druid/indexing/appenderator/BatchAppenderatorTest.java | 6 ++++-- .../segment/realtime/appenderator/BatchAppenderator.java | 5 ++--- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java index 4e6609b29d55..3a9d5fdcc2d2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java @@ -76,8 +76,10 @@ public void testSimpleIngestion() throws Exception ); // getSegments - Assert.assertEquals(IDENTIFIERS.subList(0,2), - appenderator.getSegments().stream().sorted().collect(Collectors.toList())); + Assert.assertEquals( + IDENTIFIERS.subList(0, 2), + appenderator.getSegments().stream().sorted().collect(Collectors.toList()) + ); // add one more to hit max rows in memory: Assert.assertEquals( diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 1e2bfd912bf0..586e51feff8d 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -197,7 +197,6 @@ public class BatchAppenderator implements Appenderator this.rowIngestionMeters = Preconditions.checkNotNull(rowIngestionMeters, "rowIngestionMeters"); this.parseExceptionHandler = Preconditions.checkNotNull(parseExceptionHandler, "parseExceptionHandler"); - maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault(); skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck(); } @@ -407,7 +406,7 @@ public int getTotalRowCount() } @VisibleForTesting - int getRowsInMemory() + public int getRowsInMemory() { return rowsCurrentlyInMemory.get(); } @@ -419,7 +418,7 @@ long getBytesCurrentlyInMemory() } @VisibleForTesting - long getBytesInMemory(SegmentIdWithShardSpec identifier) + public long getBytesInMemory(SegmentIdWithShardSpec identifier) { final Sink sink = sinks.get(identifier); From eec4f393ea97fe9a2199f40db0e21a1671ffa4c8 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Tue, 25 May 2021 19:05:25 -0700 Subject: [PATCH 06/47] Incorporating tests from StreamAppenderatorTest --- .../appenderator/BatchAppenderatorTest.java | 575 +++++++++++++++++- .../appenderator/BatchAppenderatorTester.java | 13 +- .../appenderator/BatchAppenderator.java | 14 +- 3 files changed, 581 insertions(+), 21 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java index 3a9d5fdcc2d2..376708b426ba 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java @@ -23,12 +23,16 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +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.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorTester; +import org.apache.druid.segment.realtime.appenderator.BatchAppenderator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -37,8 +41,10 @@ import org.junit.Assert; import org.junit.Test; +import java.io.File; import java.util.Collections; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; public class BatchAppenderatorTest extends InitializedNullHandlingTest @@ -90,8 +96,10 @@ public void testSimpleIngestion() throws Exception // 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(), - appenderator.getSegments().stream().sorted().collect(Collectors.toList())); + Assert.assertEquals( + Collections.emptyList(), + appenderator.getSegments().stream().sorted().collect(Collectors.toList()) + ); // add one more: Assert.assertEquals( @@ -109,26 +117,561 @@ public void testSimpleIngestion() throws Exception ).get(); Assert.assertEquals( IDENTIFIERS.subList(0, 3), - Lists.transform( - segmentsAndCommitMetadata.getSegments(), - new Function() - { - @Override - public SegmentIdWithShardSpec apply(DataSegment input) - { - return SegmentIdWithShardSpec.fromDataSegment(input); - } - } - ).stream().sorted().collect(Collectors.toList()) - ); - Assert.assertEquals(tester.getPushedSegments().stream().sorted().collect(Collectors.toList()), - segmentsAndCommitMetadata.getSegments().stream().sorted().collect(Collectors.toList())); + Lists.transform( + segmentsAndCommitMetadata.getSegments(), + new Function() + { + @Override + public SegmentIdWithShardSpec apply(DataSegment input) + { + return SegmentIdWithShardSpec.fromDataSegment(input); + } + } + ).stream().sorted().collect(Collectors.toList()) + ); + Assert.assertEquals( + tester.getPushedSegments().stream().sorted().collect(Collectors.toList()), + segmentsAndCommitMetadata.getSegments().stream().sorted().collect(Collectors.toList()) + ); appenderator.clear(); Assert.assertTrue(appenderator.getSegments().isEmpty()); } } + @Test + public void testMaxBytesInMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception + { + try ( + final BatchAppenderatorTester tester = new BatchAppenderatorTester( + 100, + 1024, + null, + true, + new SimpleRowIngestionMeters(), + 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 + int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; + Assert.assertEquals( + 182 + nullHandlingOverhead, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ); + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 1), null); + Assert.assertEquals( + 182 + nullHandlingOverhead, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ); + appenderator.close(); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + } + } + + @Test + public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadCheckConfig() throws Exception + { + try ( + final BatchAppenderatorTester tester = new BatchAppenderatorTester( + 100, + 1024, + null, + true, + new SimpleRowIngestionMeters(), + true + ) + ) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + + 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 + int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; + Assert.assertEquals(182 + nullHandlingOverhead, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory()); + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 1), null); + Assert.assertEquals( + 364 + 2 * nullHandlingOverhead, + ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() + ); + Assert.assertEquals(2, appenderator.getSegments().size()); + appenderator.close(); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + } + } + + @Test + public void testMaxBytesInMemory() throws Exception + { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(100, 15000, true)) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + + appenderator.startJob(); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + // Still under maxSizeInBytes after the add. Hence, we do not persist yet + //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; + int currentInMemoryIndexSize = 182 + nullHandlingOverhead; + int sinkSizeOverhead = 1 * BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ); + Assert.assertEquals( + currentInMemoryIndexSize + sinkSizeOverhead, + ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() + ); + + // We do multiple more adds to the same sink to cause persist. + for (int i = 0; i < 53; i++) { + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null); + } + sinkSizeOverhead = 1 * 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. + // currHydrant in the sink has 0 bytesInMemory since we just did a persist + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ); + // no sinks no hydrants after a persist so we should have zero bytes currently in memory + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() + ); + + // Add a single row after persisted + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bob", 1), null); + // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet + currentInMemoryIndexSize = 182 + nullHandlingOverhead; + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ); + Assert.assertEquals( + currentInMemoryIndexSize + sinkSizeOverhead, + ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() + ); + + // We do multiple more adds to the same sink to cause persist. + 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. + // so no sinks & hydrants should be in memory... + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ); + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() + ); + appenderator.close(); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory()); + } + } + + @Test(expected = RuntimeException.class) + public void testTaskFailAsPersistCannotFreeAnyMoreMemory() throws Exception + { + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(100, 5180, true)) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + appenderator.startJob(); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + } + } + + @Test + public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception + { + try ( + final BatchAppenderatorTester tester = new BatchAppenderatorTester( + 100, + 10, + null, + true, + new SimpleRowIngestionMeters(), + true + ) + ) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + + 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() + ); + } + } + + @Test + public void testTaskCleanupInMemoryCounterAfterCloseWithRowInMemory() throws Exception + { + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(100, 10000, true)) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + + appenderator.startJob(); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + + // Still under maxSizeInBytes after the add. Hence, we do not persist yet + int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; + int currentInMemoryIndexSize = 182 + nullHandlingOverhead; + int sinkSizeOverhead = 1 * BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + Assert.assertEquals( + currentInMemoryIndexSize + sinkSizeOverhead, + ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() + ); + + // Close with row still in memory (no persist) + appenderator.close(); + + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory()); + } + } + + @Test + public void testMaxBytesInMemoryInMultipleSinks() throws Exception + { + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(1000, 28748, true)) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + + appenderator.startJob(); + // next records are 182 bytes + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 1), null); + + // Still under maxSizeInBytes after the add. Hence, we do not persist yet + //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; + int currentInMemoryIndexSize = 182 + nullHandlingOverhead; + int sinkSizeOverhead = 2 * BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ); + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ); + Assert.assertEquals( + (2 * currentInMemoryIndexSize) + sinkSizeOverhead, + ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() + ); + + // We do multiple more adds to the same sink to cause persist. + for (int i = 0; i < 49; i++) { + // these records are 186 bytes + 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. + // currHydrant and the sink has 0 bytesInMemory since we just did a persist + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ); + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ); + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() + ); + + // Add a single row after persisted to sink 0 + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bob", 1), null); + // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet + currentInMemoryIndexSize = 182 + nullHandlingOverhead; + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ); + Assert.assertEquals( + 0, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ); + // only one sink so far: + sinkSizeOverhead = BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + Assert.assertEquals( + currentInMemoryIndexSize + sinkSizeOverhead, + ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() + ); + // Now add a single row to sink 1 + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bob", 1), null); + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ); + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ); + sinkSizeOverhead += BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + Assert.assertEquals( + (2 * currentInMemoryIndexSize) + sinkSizeOverhead, + ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() + ); + + // We do multiple more adds to the both sink to cause persist. + for (int i = 0; i < 49; i++) { + // 186 bytes + 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. + // currHydrant in the sink has 0 bytesInMemory since we just did a persist + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ); + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) + ); + // Mapped index size is the memory still needed after we persisted indexes. Note that the segments have + // 1 dimension columns, 2 metric column, 1 time column. However, we have two indexes now from the two pervious + // persists. + Assert.assertEquals( + currentInMemoryIndexSize, + ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() + ); + appenderator.close(); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory()); + } + } + + @Test + public void testIgnoreMaxBytesInMemory() throws Exception + { + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(100, -1, true)) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.startJob(); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + //we still calculate the size even when ignoring it to make persist decision + int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; + Assert.assertEquals( + 182 + nullHandlingOverhead, + ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) + ); + 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 + // persisted: + int sinkSizeOverhead = 2 * BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + Assert.assertEquals( + (364 + 2 * nullHandlingOverhead) + sinkSizeOverhead, + ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() + ); + Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.close(); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + } + } + + @Test + public void testMaxRowsInMemory() throws Exception + { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.startJob(); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 1), null); + Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 1), null); + Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); + // no persist since last add was for a dup record + 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()); + appenderator.close(); + } + } + + + @Test + public void testRestoreFromDisk() throws Exception + { + final BatchAppenderatorTester tester = new BatchAppenderatorTester(2, true); + final Appenderator appenderator = tester.getAppenderator(); + + appenderator.startJob(); + + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 2), null); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "baz", 3), null); + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 4), null); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + + appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "bob", 5), null); + Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.persistAll(null).get(); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + + List segmentPaths = ((BatchAppenderator) appenderator).getPersistedidentifierPaths(); + Assert.assertEquals(3, segmentPaths.size()); + + + appenderator.push(IDENTIFIERS, null, false).get(); + + segmentPaths = ((BatchAppenderator) appenderator).getPersistedidentifierPaths(); + Assert.assertEquals(0, segmentPaths.size()); + + appenderator.close(); + + } + + @Test + public void testCleanupFromDiskAfterClose() throws Exception + { + final BatchAppenderatorTester tester = new BatchAppenderatorTester(2, true); + final Appenderator appenderator = tester.getAppenderator(); + + appenderator.startJob(); + + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 2), null); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "baz", 3), null); + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 4), null); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + + appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "bob", 5), null); + Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.persistAll(null).get(); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + + List segmentPaths = ((BatchAppenderator) appenderator).getPersistedidentifierPaths(); + Assert.assertEquals(3, segmentPaths.size()); + + appenderator.close(); + + segmentPaths = ((BatchAppenderator) appenderator).getPersistedidentifierPaths(); + Assert.assertEquals(0, segmentPaths.size()); + + } + + + @Test(timeout = 60_000L) + public void testTotalRowCount() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester(3, true)) { + final Appenderator appenderator = tester.getAppenderator(); + + Assert.assertEquals(0, appenderator.getTotalRowCount()); + appenderator.startJob(); + Assert.assertEquals(0, appenderator.getTotalRowCount()); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + Assert.assertEquals(1, appenderator.getTotalRowCount()); + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 1), null); + Assert.assertEquals(2, appenderator.getTotalRowCount()); + + appenderator.persistAll(null).get(); + Assert.assertEquals(2, appenderator.getTotalRowCount()); + appenderator.drop(IDENTIFIERS.get(0)).get(); + Assert.assertEquals(1, appenderator.getTotalRowCount()); + appenderator.drop(IDENTIFIERS.get(1)).get(); + Assert.assertEquals(0, appenderator.getTotalRowCount()); + + appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "bar", 1), null); + Assert.assertEquals(1, appenderator.getTotalRowCount()); + appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "baz", 1), null); + Assert.assertEquals(2, appenderator.getTotalRowCount()); + appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "qux", 1), null); + Assert.assertEquals(3, appenderator.getTotalRowCount()); + appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "bob", 1), null); + Assert.assertEquals(4, appenderator.getTotalRowCount()); + + appenderator.persistAll(null).get(); + Assert.assertEquals(4, appenderator.getTotalRowCount()); + appenderator.drop(IDENTIFIERS.get(2)).get(); + Assert.assertEquals(0, appenderator.getTotalRowCount()); + + appenderator.close(); + Assert.assertEquals(0, appenderator.getTotalRowCount()); + } + } + + @Test + public void testVerifyRowIngestionMetrics() throws Exception + { + final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); + try (final AppenderatorTester tester = + new AppenderatorTester(5, + 10000L, + null, false, rowIngestionMeters + )) { + final Appenderator appenderator = tester.getAppenderator(); + appenderator.startJob(); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", + "foo", "invalid_met" + ), null); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + + Assert.assertEquals(1, rowIngestionMeters.getProcessed()); + Assert.assertEquals(1, rowIngestionMeters.getProcessedWithError()); + Assert.assertEquals(0, rowIngestionMeters.getUnparseable()); + Assert.assertEquals(0, rowIngestionMeters.getThrownAway()); + } + } + + private static SegmentIdWithShardSpec createSegmentId(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java index d3bae2f7ae7a..feccfac5b75c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java @@ -89,6 +89,15 @@ public BatchAppenderatorTester( this(maxRowsInMemory, -1, null, enablePushFailure); } + public BatchAppenderatorTester( + final int maxRowsInMemory, + final long maxSizeInBytes, + final boolean enablePushFailure + ) + { + this(maxRowsInMemory, maxSizeInBytes, null, enablePushFailure); + } + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, @@ -147,7 +156,7 @@ public BatchAppenderatorTester( null, maxRowsInMemory, maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, - false, + skipBytesInMemoryOverheadCheck, null, null, null, @@ -166,7 +175,7 @@ public BatchAppenderatorTester( null, null, null - ).withBasePersistDirectory(createNewBasePersistDirectory()); + ).withBasePersistDirectory(basePersistDirectory != null ? basePersistDirectory : createNewBasePersistDirectory()); metrics = new FireDepartmentMetrics(); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 586e51feff8d..4f354605f914 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -412,7 +412,7 @@ public int getRowsInMemory() } @VisibleForTesting - long getBytesCurrentlyInMemory() + public long getBytesCurrentlyInMemory() { return bytesCurrentlyInMemory.get(); } @@ -423,7 +423,7 @@ public long getBytesInMemory(SegmentIdWithShardSpec identifier) final Sink sink = sinks.get(identifier); if (sink == null) { - throw new ISE("No such sink: %s", identifier); + return 0L; // sinks are removed after a persist } else { return sink.getBytesInMemory(); } @@ -895,6 +895,7 @@ public void close() persistExecutor = null; pushExecutor = null; intermediateTempExecutor = null; + } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -903,6 +904,12 @@ public void close() // Only unlock if executors actually shut down. unlockBasePersistDirectory(); + + // cleanup: + List persistedIdentifiers = getPersistedidentifierPaths(); + for (File identifier : persistedIdentifiers) { + removeDirectory(identifier); + } } /** @@ -1181,7 +1188,8 @@ private Object bootstrapSinksFromDisk() } - private List getPersistedidentifierPaths() + @VisibleForTesting + public List getPersistedidentifierPaths() { ArrayList retVal = new ArrayList<>(); From 30e1ec392ddb83655b7257d402c78124b3829fbd Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 28 May 2021 17:00:13 -0700 Subject: [PATCH 07/47] Keep totalRows and cleanup code --- .../appenderator/BatchAppenderatorTest.java | 6 + .../appenderator/BatchAppenderator.java | 218 +----------------- 2 files changed, 15 insertions(+), 209 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java index 376708b426ba..aa0f2335bbc5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java @@ -587,15 +587,18 @@ public void testCleanupFromDiskAfterClose() throws Exception 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()); + Assert.assertEquals(2, ((BatchAppenderator) appenderator).getTotalRowCount()); 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()); + Assert.assertEquals(4, ((BatchAppenderator) appenderator).getTotalRowCount()); 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()); + Assert.assertEquals(5, ((BatchAppenderator) appenderator).getTotalRowCount()); List segmentPaths = ((BatchAppenderator) appenderator).getPersistedidentifierPaths(); Assert.assertEquals(3, segmentPaths.size()); @@ -605,6 +608,9 @@ public void testCleanupFromDiskAfterClose() throws Exception segmentPaths = ((BatchAppenderator) appenderator).getPersistedidentifierPaths(); Assert.assertEquals(0, segmentPaths.size()); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getTotalRowCount()); + } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 4f354605f914..8bd3a224294c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -218,10 +218,9 @@ public Object startJob() { tuningConfig.getBasePersistDirectory().mkdirs(); lockBasePersistDirectory(); - final Object retVal = bootstrapSinksFromDisk(); initializeExecutors(); resetNextFlush(); - return retVal; + return null; } private void throwPersistErrorIfExists() @@ -497,7 +496,7 @@ private void clear(boolean removeOnDiskData) throws InterruptedException // Drop everything. final List> futures = new ArrayList<>(); for (Map.Entry entry : sinks.entrySet()) { - futures.add(abandonSegment(entry.getKey(), entry.getValue(), removeOnDiskData)); + futures.add(removeSink(entry.getKey(), entry.getValue(), removeOnDiskData)); } // Await dropping. Futures.allAsList(futures).get(); @@ -512,7 +511,7 @@ public ListenableFuture drop(final SegmentIdWithShardSpec identifier) { final Sink sink = sinks.get(identifier); if (sink != null) { - return abandonSegment(identifier, sink, true); + return removeSink(identifier, sink, true); } else { return Futures.immediateFuture(null); } @@ -864,7 +863,7 @@ public void close() final List> futures = new ArrayList<>(); for (Map.Entry entry : sinks.entrySet()) { - futures.add(abandonSegment(entry.getKey(), entry.getValue(), false)); + futures.add(removeSink(entry.getKey(), entry.getValue(), false)); } try { @@ -910,6 +909,8 @@ public void close() for (File identifier : persistedIdentifiers) { removeDirectory(identifier); } + + totalRows.set(0); } /** @@ -1043,151 +1044,6 @@ private void resetNextFlush() nextFlush = DateTimes.nowUtc().plus(tuningConfig.getIntermediatePersistPeriod()).getMillis(); } - /** - * Populate "sinks" and "sinkTimeline" with committed segments, and announce them with the segmentAnnouncer. - * - * @return persisted commit metadata - */ - private Object bootstrapSinksFromDisk() - { - Preconditions.checkState(sinks.isEmpty(), "Already bootstrapped?!"); - - final File baseDir = tuningConfig.getBasePersistDirectory(); - if (!baseDir.exists()) { - return null; - } - - final File[] files = baseDir.listFiles(); - if (files == null) { - return null; - } - - - final Committed committed; - File commitFile = null; - try { - commitLock.lock(); - commitFile = computeCommitFile(); - if (commitFile.exists()) { - committed = objectMapper.readValue(commitFile, Committed.class); - } else { - committed = Committed.nil(); - } - } - catch (Exception e) { - throw new ISE(e, "Failed to read commitFile: %s", commitFile); - } - finally { - commitLock.unlock(); - } - - int rowsSoFar = 0; - - if (committed.equals(Committed.nil())) { - log.debug("No previously committed metadata."); - } else { - log.info( - "Loading partially-persisted segments[%s] from[%s] with commit metadata: %s", - String.join(", ", committed.getHydrants().keySet()), - baseDir, - committed.getMetadata() - ); - } - - for (File sinkDir : files) { - final File identifierFile = new File(sinkDir, IDENTIFIER_FILE_NAME); - if (!identifierFile.isFile()) { - // No identifier in this sinkDir; it must not actually be a sink directory. Skip it. - continue; - } - - try { - final SegmentIdWithShardSpec identifier = objectMapper.readValue( - new File(sinkDir, "identifier.json"), - SegmentIdWithShardSpec.class - ); - - final int committedHydrants = committed.getCommittedHydrants(identifier.toString()); - - if (committedHydrants <= 0) { - log.info("Removing uncommitted segment at [%s].", sinkDir); - FileUtils.deleteDirectory(sinkDir); - continue; - } - - // To avoid reading and listing of "merged" dir and other special files - final File[] sinkFiles = sinkDir.listFiles( - (dir, fileName) -> !(Ints.tryParse(fileName) == null) - ); - - Arrays.sort( - sinkFiles, - (o1, o2) -> Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName())) - ); - - List hydrants = new ArrayList<>(); - for (File hydrantDir : sinkFiles) { - final int hydrantNumber = Integer.parseInt(hydrantDir.getName()); - - if (hydrantNumber >= committedHydrants) { - log.info("Removing uncommitted partial segment at [%s]", hydrantDir); - FileUtils.deleteDirectory(hydrantDir); - } else { - log.debug("Loading previously persisted partial segment at [%s]", hydrantDir); - if (hydrantNumber != hydrants.size()) { - throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir); - } - - hydrants.add( - new FireHydrant( - new QueryableIndexSegment(indexIO.loadIndex(hydrantDir), identifier.asSegmentId()), - hydrantNumber - ) - ); - } - } - - // Make sure we loaded enough hydrants. - if (committedHydrants != hydrants.size()) { - throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir); - } - - Sink currSink = new Sink( - identifier.getInterval(), - schema, - identifier.getShardSpec(), - identifier.getVersion(), - tuningConfig.getAppendableIndexSpec(), - tuningConfig.getMaxRowsInMemory(), - maxBytesTuningConfig, - null, - hydrants - ); - rowsSoFar += currSink.getNumRows(); - sinks.put(identifier, currSink); - segmentAnnouncer.announceSegment(currSink.getSegment()); - } - catch (IOException e) { - log.makeAlert(e, "Problem loading sink[%s] from disk.", schema.getDataSource()) - .addData("sinkDir", sinkDir) - .emit(); - } - } - - // Make sure we loaded all committed sinks. - final Set loadedSinks = Sets.newHashSet( - Iterables.transform(sinks.keySet(), SegmentIdWithShardSpec::toString) - ); - final Set missingSinks = Sets.difference(committed.getHydrants().keySet(), loadedSinks); - if (!missingSinks.isEmpty()) { - throw new ISE("Missing committed sinks [%s]", Joiner.on(", ").join(missingSinks)); - } - - totalRows.set(rowsSoFar); - return committed.getMetadata(); - } - - @VisibleForTesting public List getPersistedidentifierPaths() { @@ -1275,7 +1131,7 @@ Pair getIdentifierAndSinkForPersistedFile(File ide return null; } - private ListenableFuture abandonSegment( + private ListenableFuture removeSink( final SegmentIdWithShardSpec identifier, final Sink sink, final boolean removeOnDiskData @@ -1294,12 +1150,10 @@ private ListenableFuture abandonSegment( bytesCurrentlyInMemory.addAndGet(-calculateMMappedHydrantMemoryInUsed(hydrant)); } } - totalRows.addAndGet(-sink.getNumRows()); + // 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. } - // Mark this identifier as dropping, so no future push tasks will pick it up. - //droppingSinks.add(identifier); - // Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread. return Futures.transform( pushBarrier(), @@ -1316,38 +1170,6 @@ public Void apply(@Nullable Object input) metrics.setSinkCount(sinks.size()); - if (removeOnDiskData) { - // Remove this segment from the committed list. This must be done from the persist thread. - log.debug("Removing commit metadata for segment[%s].", identifier); - try { - commitLock.lock(); - final Committed oldCommit = readCommit(); - if (oldCommit != null) { - writeCommit(oldCommit.without(identifier.toString())); - } - } - catch (Exception e) { - log.makeAlert(e, "Failed to update committed segments[%s]", schema.getDataSource()) - .addData("identifier", identifier.toString()) - .emit(); - throw new RuntimeException(e); - } - finally { - commitLock.unlock(); - } - } - - // Unannounce the segment. - try { - segmentAnnouncer.unannounceSegment(sink.getSegment()); - } - catch (Exception e) { - log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) - .addData("identifier", identifier.toString()) - .emit(); - } - - //droppingSinks.remove(identifier); for (FireHydrant hydrant : sink) { if (cache != null) { cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant)); @@ -1370,28 +1192,6 @@ public Void apply(@Nullable Object input) ); } - private Committed readCommit() throws IOException - { - final File commitFile = computeCommitFile(); - if (commitFile.exists()) { - // merge current hydrants with existing hydrants - return objectMapper.readValue(commitFile, Committed.class); - } else { - return null; - } - } - - private void writeCommit(Committed newCommit) throws IOException - { - final File commitFile = computeCommitFile(); - objectMapper.writeValue(commitFile, newCommit); - } - - private File computeCommitFile() - { - return new File(tuningConfig.getBasePersistDirectory(), "commit.json"); - } - private File computeLockFile() { return new File(tuningConfig.getBasePersistDirectory(), ".lock"); From 00922d73acba542f4b847bd4939f12e530842e97 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 28 May 2021 17:18:36 -0700 Subject: [PATCH 08/47] Added missing dep --- indexing-service/pom.xml | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 7903856db922..022ad8881b44 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -227,7 +227,11 @@ jackson-core-asl provided - + + org.apache.commons + commons-collections4 + provided + junit From 00dd32b0f5fdbe7bac63fc301a2e7fd03bbd7e5b Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 28 May 2021 17:25:47 -0700 Subject: [PATCH 09/47] Fix unit test --- .../DefaultOfflineAppenderatorFactoryTest.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index 85829fe07bf9..58c20a0e5b02 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -171,13 +171,13 @@ public int columnCacheSizeBytes() "A", new LinearShardSpec(0) ); - Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); - appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "bar", 1), Suppliers.ofInstance(Committers.nil())); - Assert.assertEquals(1, ((StreamAppenderator) appenderator).getRowsInMemory()); - appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "baz", 1), Suppliers.ofInstance(Committers.nil())); - Assert.assertEquals(2, ((StreamAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "bar", 1),null); + Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "baz", 1), null); + Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.close(); - Assert.assertEquals(0, ((StreamAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); } finally { appenderator.close(); From 864357dfb0c9902e8478998394dd5bead9373bce Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Tue, 1 Jun 2021 10:23:35 -0700 Subject: [PATCH 10/47] Checkstyle --- .../segment/realtime/appenderator/BatchAppenderator.java | 3 --- .../appenderator/DefaultOfflineAppenderatorFactoryTest.java | 4 +--- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 8bd3a224294c..3abd83ad0b39 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -22,14 +22,12 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; -import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.base.Stopwatch; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; import com.google.common.primitives.Ints; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -84,7 +82,6 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index 58c20a0e5b02..2181517eefc1 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.realtime.appenderator; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Injector; @@ -43,7 +42,6 @@ import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.plumber.Committers; import org.apache.druid.timeline.partition.LinearShardSpec; import org.junit.Assert; import org.junit.Rule; @@ -172,7 +170,7 @@ public int columnCacheSizeBytes() new LinearShardSpec(0) ); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); - appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "bar", 1),null); + appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "bar", 1), null); Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "baz", 1), null); Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); From 2fa2a9d26b9236c075830c7edb9ddbf496b0edf2 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Tue, 1 Jun 2021 13:23:17 -0700 Subject: [PATCH 11/47] allowIncrementalPersists should always be true for batch --- .../realtime/appenderator/BatchAppenderator.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 3abd83ad0b39..3101c3a222b3 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -235,12 +235,19 @@ public AppenderatorAddResult add( final boolean allowIncrementalPersists ) throws IndexSizeExceededException, SegmentNotWritableException { + throwPersistErrorIfExists(); + Preconditions.checkArgument( committerSupplier == null, "Batch appenderator does not need a committer!" ); + Preconditions.checkArgument( + allowIncrementalPersists, + "Batch appenderator should always allow incremental persists!" + ); + if (!identifier.getDataSource().equals(schema.getDataSource())) { throw new IAE( "Expected dataSource[%s] but was asked to insert row for dataSource[%s]?!", @@ -371,7 +378,7 @@ public AppenderatorAddResult add( persistAllAndClear(); } else { - isPersistRequired = true; + throw new ISE("Batch appenderator always persists as needed!"); } } return new AppenderatorAddResult(identifier, sink.getNumRows(), isPersistRequired); @@ -1178,7 +1185,7 @@ public Void apply(@Nullable Object input) removeDirectory(computePersistDir(identifier)); } - log.info("Dropped segment[%s].", identifier); + log.info("Removed sink for segment[%s].", identifier); return null; } From 5b0157beca3cc2589574609b05055be7ef54eb6a Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Tue, 1 Jun 2021 16:20:34 -0700 Subject: [PATCH 12/47] Added sinks metadata --- .../appenderator/BatchAppenderatorTest.java | 60 +++++++++++++++++++ .../appenderator/BatchAppenderator.java | 36 ++++++++++- 2 files changed, 93 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java index aa0f2335bbc5..a06ce0f0292b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java @@ -541,6 +541,66 @@ public void testMaxRowsInMemory() throws Exception } } + @Test + public void testTotalRowsPerSegment() throws Exception + { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.startJob(); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + Appenderator.AppenderatorAddResult addResult0 = + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(1,addResult0.getNumRowsInSegment()); + + Appenderator.AppenderatorAddResult addResult1 = + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 1), null); + Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(1,addResult1.getNumRowsInSegment()); + + addResult1 = // dup! + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 1), null); + Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(1,addResult1.getNumRowsInSegment()); // dup record does not count + // no persist since last add was for a dup record + + addResult1 = + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bat", 1), null); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(2,addResult1.getNumRowsInSegment()); + // persist expected ^ (3) rows added + + // total rows per segment ought to be preserved even when sinks are removed from memory: + addResult1 = + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bat", 1), null); + Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(3,addResult1.getNumRowsInSegment()); + + addResult0 = + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "baz", 1), null); + Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(2,addResult0.getNumRowsInSegment()); + + addResult1 = + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 1), null); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(4,addResult1.getNumRowsInSegment()); + // persist expected ^ (3) rows added + + addResult0 = + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bob", 1), null); + Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); + Assert.assertEquals(3,addResult0.getNumRowsInSegment()); + + appenderator.close(); + + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + } + } + @Test public void testRestoreFromDisk() throws Exception diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 3101c3a222b3..48e7e6031006 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -121,10 +121,39 @@ public class BatchAppenderator implements Appenderator * of any thread from {@link #drop}. */ private final ConcurrentMap sinks = new ConcurrentHashMap<>(); - //private final Set droppingSinks = Sets.newConcurrentHashSet(); private final long maxBytesTuningConfig; private final boolean skipBytesInMemoryOverheadCheck; + /** + * 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. For now, {@link SinkMetadata} only + * contains a single memeber {@link SinkMetadata#numRowsInSegment} but we can add more in the future as needed + */ + private final ConcurrentMap sinksMetadata = new ConcurrentHashMap<>(); + private static class SinkMetadata + { + private int numRowsInSegment; + + public SinkMetadata() + { + this(0); + } + + public SinkMetadata(int numRowsInSegment) + { + this.numRowsInSegment = numRowsInSegment; + } + public void addRows(int numRows) { + numRowsInSegment += numRows; + } + + public int getNumRowsInSegment() + { + return numRowsInSegment; + } + } + + private final QuerySegmentWalker texasRanger; // This variable updated in add(), persist(), and drop() private final AtomicInteger rowsCurrentlyInMemory = new AtomicInteger(); @@ -291,6 +320,7 @@ public AppenderatorAddResult add( rowsCurrentlyInMemory.addAndGet(numAddedRows); bytesCurrentlyInMemory.addAndGet(bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd); totalRows.addAndGet(numAddedRows); + sinksMetadata.computeIfAbsent(identifier,x -> new SinkMetadata()).addRows(numAddedRows); boolean isPersistRequired = false; boolean persist = false; @@ -327,7 +357,7 @@ public AppenderatorAddResult add( if (persist) { if (allowIncrementalPersists) { // persistAll clears rowsCurrentlyInMemory, no need to update it. - log.info("Flushing in-memory data to disk because %s.", String.join(",", persistReasons)); + log.info("Incremental persist to disk because %s.", String.join(",", persistReasons)); long bytesToBePersisted = 0L; for (Map.Entry entry : sinks.entrySet()) { @@ -381,7 +411,7 @@ public AppenderatorAddResult add( throw new ISE("Batch appenderator always persists as needed!"); } } - return new AppenderatorAddResult(identifier, sink.getNumRows(), isPersistRequired); + return new AppenderatorAddResult(identifier, sinksMetadata.get(identifier).numRowsInSegment, isPersistRequired); } @Override From 031276344e3b63dd36ce26f0ae2724c8674b2bd2 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Tue, 1 Jun 2021 16:44:11 -0700 Subject: [PATCH 13/47] clear sinks metadata when closing appenderator --- .../druid/segment/realtime/appenderator/BatchAppenderator.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 48e7e6031006..6976788c9e4b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -945,6 +945,7 @@ public void close() } totalRows.set(0); + sinksMetadata.clear(); } /** From e308d68225112931354fde1f9a6477cde9d38379 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Tue, 1 Jun 2021 18:40:22 -0700 Subject: [PATCH 14/47] Style + minor edits to log msgs --- .../appenderator/BatchAppenderatorTest.java | 16 ++++++++-------- .../realtime/appenderator/BatchAppenderator.java | 14 ++++++++++---- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java index a06ce0f0292b..d17fcfe50a48 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java @@ -554,46 +554,46 @@ public void testTotalRowsPerSegment() throws Exception Appenderator.AppenderatorAddResult addResult0 = appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); - Assert.assertEquals(1,addResult0.getNumRowsInSegment()); + Assert.assertEquals(1, addResult0.getNumRowsInSegment()); Appenderator.AppenderatorAddResult addResult1 = appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 1), null); Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); - Assert.assertEquals(1,addResult1.getNumRowsInSegment()); + Assert.assertEquals(1, addResult1.getNumRowsInSegment()); addResult1 = // dup! appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar", 1), null); Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); - Assert.assertEquals(1,addResult1.getNumRowsInSegment()); // dup record does not count + Assert.assertEquals(1, addResult1.getNumRowsInSegment()); // dup record does not count // no persist since last add was for a dup record addResult1 = appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bat", 1), null); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); - Assert.assertEquals(2,addResult1.getNumRowsInSegment()); + Assert.assertEquals(2, addResult1.getNumRowsInSegment()); // persist expected ^ (3) rows added // total rows per segment ought to be preserved even when sinks are removed from memory: addResult1 = appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bat", 1), null); Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); - Assert.assertEquals(3,addResult1.getNumRowsInSegment()); + Assert.assertEquals(3, addResult1.getNumRowsInSegment()); addResult0 = appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "baz", 1), null); Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); - Assert.assertEquals(2,addResult0.getNumRowsInSegment()); + Assert.assertEquals(2, addResult0.getNumRowsInSegment()); addResult1 = appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "qux", 1), null); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); - Assert.assertEquals(4,addResult1.getNumRowsInSegment()); + Assert.assertEquals(4, addResult1.getNumRowsInSegment()); // persist expected ^ (3) rows added addResult0 = appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bob", 1), null); Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); - Assert.assertEquals(3,addResult0.getNumRowsInSegment()); + Assert.assertEquals(3, addResult0.getNumRowsInSegment()); appenderator.close(); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 6976788c9e4b..d195f65471bc 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -130,6 +130,7 @@ public class BatchAppenderator implements Appenderator * contains a single memeber {@link SinkMetadata#numRowsInSegment} but we can add more in the future as needed */ private final ConcurrentMap sinksMetadata = new ConcurrentHashMap<>(); + private static class SinkMetadata { private int numRowsInSegment; @@ -143,7 +144,9 @@ public SinkMetadata(int numRowsInSegment) { this.numRowsInSegment = numRowsInSegment; } - public void addRows(int numRows) { + + public void addRows(int numRows) + { numRowsInSegment += numRows; } @@ -320,7 +323,7 @@ public AppenderatorAddResult add( rowsCurrentlyInMemory.addAndGet(numAddedRows); bytesCurrentlyInMemory.addAndGet(bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd); totalRows.addAndGet(numAddedRows); - sinksMetadata.computeIfAbsent(identifier,x -> new SinkMetadata()).addRows(numAddedRows); + sinksMetadata.computeIfAbsent(identifier, Void -> new SinkMetadata()).addRows(numAddedRows); boolean isPersistRequired = false; boolean persist = false; @@ -611,6 +614,9 @@ public ListenableFuture persistAll(@Nullable final Committer committer) } log.debug("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); + if (indexesToPersist.isEmpty()) { + log.info("No indexes will be peristed"); + } final Stopwatch runExecStopwatch = Stopwatch.createStarted(); final Stopwatch persistStopwatch = Stopwatch.createStarted(); AtomicLong totalPersistedRows = new AtomicLong(numPersistedRows); @@ -626,7 +632,7 @@ public Object call() } log.info( - "Flushed in-memory data for segments: %s", + "Persisted in-memory data for segments: %s", indexesToPersist.stream() .map(itp -> itp.rhs.asSegmentId().toString()) .distinct() @@ -1293,7 +1299,7 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id ); log.info( - "Flushed in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).", + "Persisted in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).", indexToPersist.getSegmentId(), indexToPersist.getCount(), (System.nanoTime() - startTime) / 1000000, From e55c40edd7eb6f5c7cbaf1b874b286fa3af9c5da Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Wed, 2 Jun 2021 12:40:10 -0700 Subject: [PATCH 15/47] Update sinks metadata & totalRows when dropping a sink (segment) --- .../realtime/appenderator/BatchAppenderator.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index d195f65471bc..9d3daf7df98b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -547,6 +547,16 @@ private void clear(boolean removeOnDiskData) throws InterruptedException public ListenableFuture drop(final SegmentIdWithShardSpec identifier) { final Sink sink = sinks.get(identifier); + SinkMetadata sm = sinksMetadata.remove(identifier); + if (sm != null) { + int originalTotalRows = getTotalRowCount(); + int rowsToDrop = sm.getNumRowsInSegment(); + int totalRowsAfter = originalTotalRows - rowsToDrop; + if (totalRowsAfter < 0) { + log.warn("Total rows[%d] after dropping segment[%s] rows [%d]", totalRowsAfter, identifier, rowsToDrop); + } + totalRows.set(Math.max(originalTotalRows - rowsToDrop,0); + } if (sink != null) { return removeSink(identifier, sink, true); } else { From 4bb61a75dca3388e950cb780f717c8da4598ae5b Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Wed, 2 Jun 2021 13:43:26 -0700 Subject: [PATCH 16/47] Remove max --- .../druid/segment/realtime/appenderator/BatchAppenderator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 9d3daf7df98b..d193070a5622 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -555,7 +555,7 @@ public ListenableFuture drop(final SegmentIdWithShardSpec identifier) if (totalRowsAfter < 0) { log.warn("Total rows[%d] after dropping segment[%s] rows [%d]", totalRowsAfter, identifier, rowsToDrop); } - totalRows.set(Math.max(originalTotalRows - rowsToDrop,0); + totalRows.set(Math.max(totalRowsAfter, 0)); } if (sink != null) { return removeSink(identifier, sink, true); From dddd4e415ad44483ad86fdf1e4c89f4c097fc701 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Wed, 2 Jun 2021 18:34:43 -0700 Subject: [PATCH 17/47] Intelli-j check --- .../druid/segment/realtime/appenderator/BatchAppenderator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index d193070a5622..d719723089b8 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -129,7 +129,7 @@ public class BatchAppenderator implements Appenderator * are being completely removed from memory after each incremental persist. For now, {@link SinkMetadata} only * contains a single memeber {@link SinkMetadata#numRowsInSegment} but we can add more in the future as needed */ - private final ConcurrentMap sinksMetadata = new ConcurrentHashMap<>(); + private final ConcurrentHashMap sinksMetadata = new ConcurrentHashMap<>(); private static class SinkMetadata { From 19bdee042aac2a5258fe7f06552574f64c75dabb Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Thu, 3 Jun 2021 11:53:45 -0700 Subject: [PATCH 18/47] Keep a count of hydrants persisted by sink for sanity check before merge --- .../appenderator/BatchAppenderatorTest.java | 16 +++-- .../appenderator/BatchAppenderator.java | 71 +++++++++++++------ 2 files changed, 60 insertions(+), 27 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java index d17fcfe50a48..e9848bb2c272 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java @@ -50,9 +50,9 @@ public class BatchAppenderatorTest extends InitializedNullHandlingTest { private static final List IDENTIFIERS = ImmutableList.of( - createSegmentId("2000/2001", "A", 0), - createSegmentId("2000/2001", "A", 1), - createSegmentId("2001/2002", "A", 0) + createSegmentId("2000/2001", "A", 0), // should be in seg_0 + createSegmentId("2000/2001", "A", 1), // seg_1 + createSegmentId("2001/2002", "A", 0) // seg 2 ); @Test @@ -68,13 +68,14 @@ public void testSimpleIngestion() throws Exception // getDataSource Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource()); - // add + // 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) @@ -87,7 +88,8 @@ public void testSimpleIngestion() throws Exception appenderator.getSegments().stream().sorted().collect(Collectors.toList()) ); - // add one more to hit max rows in memory: + + // add #3, this hits max rows in memory: Assert.assertEquals( 2, appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "sux", 1), null) @@ -101,7 +103,7 @@ public void testSimpleIngestion() throws Exception appenderator.getSegments().stream().sorted().collect(Collectors.toList()) ); - // add one more: + // add #4, this will add one more temporary segment: Assert.assertEquals( 1, appenderator.add(IDENTIFIERS.get(2), createInputRow("2001", "qux", 4), null) @@ -134,7 +136,7 @@ public SegmentIdWithShardSpec apply(DataSegment input) segmentsAndCommitMetadata.getSegments().stream().sorted().collect(Collectors.toList()) ); - appenderator.clear(); + appenderator.close(); Assert.assertTrue(appenderator.getSegments().isEmpty()); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index d719723089b8..ee7258b42c59 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -33,7 +33,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import org.apache.commons.lang.mutable.MutableLong; +import org.apache.commons.lang.mutable.MutableInt; import org.apache.druid.client.cache.Cache; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputRow; @@ -134,26 +134,38 @@ public class BatchAppenderator implements Appenderator private static class SinkMetadata { private int numRowsInSegment; + private int numHydrants; public SinkMetadata() { - this(0); + this(0,0); } - public SinkMetadata(int numRowsInSegment) + public SinkMetadata(int numRowsInSegment, int numHydrants) { this.numRowsInSegment = numRowsInSegment; + this.numHydrants = numHydrants; } - public void addRows(int numRows) + public void addRows(int num) { - numRowsInSegment += numRows; + numRowsInSegment += num; + } + + public void addHydrants(int num) + { + numHydrants += num; } public int getNumRowsInSegment() { return numRowsInSegment; } + + public int getNumHydrants() + { + return numHydrants; + } } @@ -368,10 +380,8 @@ public AppenderatorAddResult add( if (sinkEntry != null) { bytesToBePersisted += sinkEntry.getBytesInMemory(); if (sinkEntry.swappable()) { - // After swapping the sink, we use memory mapped segment instead (but only for real time appenderators!). - // However, the memory mapped segment still consumes memory. - // These memory mapped segments are held in memory throughout the ingestion phase and permanently add to the bytesCurrentlyInMemory - int memoryStillInUse = calculateMMappedHydrantMemoryInUsed(sink.getCurrHydrant()); + // Code for batch no longer memory maps hydrants but they still take memory... + int memoryStillInUse = calculateMemoryUsedByHydrants(sink.getCurrHydrant()); bytesCurrentlyInMemory.addAndGet(memoryStillInUse); } } @@ -591,8 +601,9 @@ public ListenableFuture persistAll(@Nullable final Committer committer) final List> indexesToPersist = new ArrayList<>(); int numPersistedRows = 0; long bytesPersisted = 0L; - MutableLong totalHydrantsCount = new MutableLong(); - MutableLong totalHydrantsPersisted = new MutableLong(); + MutableInt totalHydrantsCount = new MutableInt(); + MutableInt totalHydrantsPersistedAcrossSinks = new MutableInt(); + SegmentIdWithShardSpec startIdentifier = null; final long totalSinks = sinks.size(); for (Map.Entry entry : sinks.entrySet()) { final SegmentIdWithShardSpec identifier = entry.getKey(); @@ -600,6 +611,9 @@ public ListenableFuture persistAll(@Nullable final Committer committer) if (sink == null) { throw new ISE("No sink for identifier: %s", identifier); } + + int previousHydrantCount = totalHydrantsPersistedAcrossSinks.intValue(); + final List hydrants = Lists.newArrayList(sink); totalHydrantsCount.add(hydrants.size()); numPersistedRows += sink.getNumRowsInMemory(); @@ -612,15 +626,22 @@ public ListenableFuture persistAll(@Nullable final Committer committer) if (!hydrant.hasSwapped()) { log.debug("Hydrant[%s] hasn't persisted yet, persisting. Segment[%s]", hydrant, identifier); indexesToPersist.add(Pair.of(hydrant, identifier)); - totalHydrantsPersisted.add(1); + totalHydrantsPersistedAcrossSinks.add(1); } } if (sink.swappable()) { // It is swappable. Get the old one to persist it and create a new one: indexesToPersist.add(Pair.of(sink.swap(), identifier)); - totalHydrantsPersisted.add(1); + totalHydrantsPersistedAcrossSinks.add(1); } + + // keep track of hydrants for sanity when resurrecting before push + sinksMetadata.computeIfAbsent( + identifier, + Void -> new SinkMetadata() + ).addHydrants(totalHydrantsPersistedAcrossSinks.intValue() - previousHydrantCount); + } log.debug("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); @@ -654,7 +675,7 @@ public Object call() totalPersistedRows.get(), totalSinks, totalHydrantsCount.longValue(), - totalHydrantsPersisted.longValue() + totalHydrantsPersistedAcrossSinks.longValue() ); // return null if committer is null @@ -716,16 +737,16 @@ public SegmentsAndCommitMetadata call() final List dataSegments = new ArrayList<>(); List persistedIdentifiers = getPersistedidentifierPaths(); for (File identifier : persistedIdentifiers) { - Pair sinkAndHydrants = getIdentifierAndSinkForPersistedFile(identifier); + Pair identifiersAndSinks = getIdentifierAndSinkForPersistedFile(identifier); final DataSegment dataSegment = mergeAndPush( - sinkAndHydrants.lhs, - sinkAndHydrants.rhs, + identifiersAndSinks.lhs, + identifiersAndSinks.rhs, useUniquePath ); if (dataSegment != null) { dataSegments.add(dataSegment); } else { - log.warn("mergeAndPush[%s] returned null, skipping.", sinkAndHydrants.lhs); + log.warn("mergeAndPush[%s] returned null, skipping.", identifiersAndSinks.lhs); } } @@ -774,6 +795,7 @@ private DataSegment mergeAndPush( final File descriptorFile = computeDescriptorFile(identifier); // Sanity checks + int numHydrants = 0; for (FireHydrant hydrant : sink) { if (sink.isWritable()) { throw new ISE("Expected sink to be no longer writable before mergeAndPush for segment[%s].", identifier); @@ -784,6 +806,15 @@ private DataSegment mergeAndPush( throw new ISE("Expected sink to be fully persisted before mergeAndPush for segment[%s].", identifier); } } + numHydrants++; + } + + SinkMetadata sm = sinksMetadata.get(identifier); + if (sm == null) { + log.warn("Sink metadata not found just before merge for identifier [%s]", identifier); + } else if (numHydrants != sinksMetadata.get(identifier).getNumHydrants()) { + throw new ISE("Number of restored hydrants[%d] for identifier[%s] does not match expected value[%d]", + numHydrants, identifier, sinksMetadata.get(identifier).getNumHydrants()); } try { @@ -1198,7 +1229,7 @@ private ListenableFuture removeSink( for (FireHydrant hydrant : sink) { // Decrement memory used by all Memory Mapped Hydrant if (!hydrant.equals(sink.getCurrHydrant())) { - bytesCurrentlyInMemory.addAndGet(-calculateMMappedHydrantMemoryInUsed(hydrant)); + bytesCurrentlyInMemory.addAndGet(-calculateMemoryUsedByHydrants(hydrant)); } } // totalRows are not decremented when removing the sink from memory, sink was just persisted and it @@ -1347,7 +1378,7 @@ private void removeDirectory(final File target) } } - private int calculateMMappedHydrantMemoryInUsed(FireHydrant hydrant) + private int calculateMemoryUsedByHydrants(FireHydrant hydrant) { if (skipBytesInMemoryOverheadCheck) { return 0; From f1211f0514628a3cc4bf893478c615a94af8d7be Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Thu, 3 Jun 2021 18:09:02 -0700 Subject: [PATCH 19/47] Move out sanity --- .../segment/realtime/appenderator/BatchAppenderator.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index ee7258b42c59..83bbb9433bfd 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -795,12 +795,12 @@ private DataSegment mergeAndPush( final File descriptorFile = computeDescriptorFile(identifier); // Sanity checks + if (sink.isWritable()) { + throw new ISE("Expected sink to be no longer writable before mergeAndPush for segment[%s].", identifier); + } + int numHydrants = 0; for (FireHydrant hydrant : sink) { - if (sink.isWritable()) { - throw new ISE("Expected sink to be no longer writable before mergeAndPush for segment[%s].", identifier); - } - synchronized (hydrant) { if (!hydrant.hasSwapped()) { throw new ISE("Expected sink to be fully persisted before mergeAndPush for segment[%s].", identifier); From 1f37d5d5e4c77069e68803765140cc51b3ef5109 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 4 Jun 2021 11:53:31 -0700 Subject: [PATCH 20/47] Add previous hydrant count to sink metadata --- .../appenderator/BatchAppenderatorTest.java | 41 +++++++++++++++++++ .../appenderator/BatchAppenderator.java | 40 ++++++++++++++++-- 2 files changed, 78 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java index e9848bb2c272..41f28d544054 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java @@ -543,6 +543,47 @@ public void testMaxRowsInMemory() throws Exception } } + @Test + public void testAllHydrantsAreRecovered() throws Exception + { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(1, false)) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.startJob(); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo2", 1), null); + 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 + // an exception + final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push( + appenderator.getSegments(), + null, + false + ).get(); + Assert.assertEquals( + IDENTIFIERS.subList(0, 1), + Lists.transform( + segmentsAndCommitMetadata.getSegments(), + new Function() + { + @Override + public SegmentIdWithShardSpec apply(DataSegment input) + { + return SegmentIdWithShardSpec.fromDataSegment(input); + } + } + ).stream().sorted().collect(Collectors.toList()) + ); + Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); + appenderator.close(); + } + } + @Test public void testTotalRowsPerSegment() throws Exception { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 83bbb9433bfd..4d43343ac11c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -131,20 +131,41 @@ public class BatchAppenderator implements Appenderator */ private final ConcurrentHashMap sinksMetadata = new ConcurrentHashMap<>(); + /** + * 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 + * fields is annotated as comments in the class + */ private static class SinkMetadata { + /** This is used to maintain the rows in the sink accross persists of the sink + // used for functionality (i.e. to detect whether an incremental push + // is needed {@link AppenderatorDriverAddResult#isPushRequired(Integer, Long)} + **/ private int numRowsInSegment; + /** For sanity check to make sure that all hydrants for a sink are restored from disk at + * push time + */ private int numHydrants; + /** Sinks when they are persisted lose information about the previous hydrant count, + * this variable remembers that so the proper directory can be created when persisting + * hydrants + */ + private int previousHydrantCount; public SinkMetadata() { - this(0,0); + this(0,0,0); } - public SinkMetadata(int numRowsInSegment, int numHydrants) + public SinkMetadata(int numRowsInSegment, int numHydrants, int previousHydrantCount) { this.numRowsInSegment = numRowsInSegment; this.numHydrants = numHydrants; + this.previousHydrantCount = previousHydrantCount; + } public void addRows(int num) @@ -166,6 +187,13 @@ public int getNumHydrants() { return numHydrants; } + + public void addHydrantCount(int num) { + previousHydrantCount += num; + } + public int getHydrantCount() { + return previousHydrantCount; + } } @@ -1330,11 +1358,15 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id final long startTime = System.nanoTime(); 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 + // to pull it from there.... + SinkMetadata sm = sinksMetadata.get(identifier); final File persistDir = createPersistDirIfNeeded(identifier); indexMerger.persist( indexToPersist.getIndex(), identifier.getInterval(), - new File(persistDir, String.valueOf(indexToPersist.getCount())), + new File(persistDir, String.valueOf(sm.getHydrantCount())), tuningConfig.getIndexSpecForIntermediatePersists(), tuningConfig.getSegmentWriteOutMediumFactory() ); @@ -1348,6 +1380,8 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id ); indexToPersist.swapSegment(null); + // remember hydrant count: + sinksMetadata.get(identifier).addHydrantCount(1); return numRows; } From e0c3e1601ebc7aea36aada09c96a1f6688e60b7d Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 4 Jun 2021 13:32:06 -0700 Subject: [PATCH 21/47] Remove redundant field from SinkMetadata --- .../appenderator/BatchAppenderator.java | 20 +++---------------- 1 file changed, 3 insertions(+), 17 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 4d43343ac11c..06e560aa5639 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -157,15 +157,13 @@ private static class SinkMetadata public SinkMetadata() { - this(0,0,0); + this(0, 0, 0); } public SinkMetadata(int numRowsInSegment, int numHydrants, int previousHydrantCount) { this.numRowsInSegment = numRowsInSegment; this.numHydrants = numHydrants; - this.previousHydrantCount = previousHydrantCount; - } public void addRows(int num) @@ -188,12 +186,6 @@ public int getNumHydrants() return numHydrants; } - public void addHydrantCount(int num) { - previousHydrantCount += num; - } - public int getHydrantCount() { - return previousHydrantCount; - } } @@ -664,12 +656,6 @@ public ListenableFuture persistAll(@Nullable final Committer committer) totalHydrantsPersistedAcrossSinks.add(1); } - // keep track of hydrants for sanity when resurrecting before push - sinksMetadata.computeIfAbsent( - identifier, - Void -> new SinkMetadata() - ).addHydrants(totalHydrantsPersistedAcrossSinks.intValue() - previousHydrantCount); - } log.debug("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); @@ -1366,7 +1352,7 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id indexMerger.persist( indexToPersist.getIndex(), identifier.getInterval(), - new File(persistDir, String.valueOf(sm.getHydrantCount())), + new File(persistDir, String.valueOf(sm.getNumHydrants())), tuningConfig.getIndexSpecForIntermediatePersists(), tuningConfig.getSegmentWriteOutMediumFactory() ); @@ -1381,7 +1367,7 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id indexToPersist.swapSegment(null); // remember hydrant count: - sinksMetadata.get(identifier).addHydrantCount(1); + sinksMetadata.get(identifier).addHydrants(1); return numRows; } From 4fb1f47cef0ab5ffa32b1b0dd7e2d85fb080fa21 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 4 Jun 2021 13:57:48 -0700 Subject: [PATCH 22/47] Remove unneeded functions --- .../appenderator/BatchAppenderator.java | 40 ++----------------- 1 file changed, 3 insertions(+), 37 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 06e560aa5639..8e00f3e9638c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -37,7 +37,6 @@ import org.apache.druid.client.cache.Cache; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputRow; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -50,7 +49,6 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; @@ -90,8 +88,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; public class BatchAppenderator implements Appenderator @@ -188,17 +184,12 @@ public int getNumHydrants() } - - private final QuerySegmentWalker texasRanger; // This variable updated in add(), persist(), and drop() private final AtomicInteger rowsCurrentlyInMemory = new AtomicInteger(); private final AtomicInteger totalRows = new AtomicInteger(); private final AtomicLong bytesCurrentlyInMemory = new AtomicLong(); private final RowIngestionMeters rowIngestionMeters; private final ParseExceptionHandler parseExceptionHandler; - // Synchronize persisting commitMetadata so that multiple persist threads (if present) - // and abandon threads do not step over each other - private final Lock commitLock = new ReentrantLock(); private final AtomicBoolean closed = new AtomicBoolean(false); @@ -208,7 +199,6 @@ public int getNumHydrants() // where persist and push Executor try to put tasks in each other queues // thus creating circular dependency private volatile ListeningExecutorService intermediateTempExecutor = null; - private volatile long nextFlush; private volatile FileLock basePersistDirLock = null; private volatile FileChannel basePersistDirLockChannel = null; @@ -254,7 +244,6 @@ public int getNumHydrants() this.indexIO = Preconditions.checkNotNull(indexIO, "indexIO"); this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger"); this.cache = cache; - this.texasRanger = sinkQuerySegmentWalker; this.rowIngestionMeters = Preconditions.checkNotNull(rowIngestionMeters, "rowIngestionMeters"); this.parseExceptionHandler = Preconditions.checkNotNull(parseExceptionHandler, "parseExceptionHandler"); @@ -280,7 +269,6 @@ public Object startJob() tuningConfig.getBasePersistDirectory().mkdirs(); lockBasePersistDirectory(); initializeExecutors(); - resetNextFlush(); return null; } @@ -329,7 +317,7 @@ public AppenderatorAddResult add( final IncrementalIndexAddResult addResult; try { - addResult = sink.add(row, !allowIncrementalPersists); + addResult = sink.add(row, false); // allow incrememtal persis is always true for batch sinkRowsInMemoryAfterAdd = addResult.getRowCount(); bytesInMemoryAfterAdd = addResult.getBytesInMemory(); } @@ -365,14 +353,6 @@ public AppenderatorAddResult add( persist = true; persistReasons.add("No more rows can be appended to sink"); } - if (System.currentTimeMillis() > nextFlush) { - persist = true; - persistReasons.add(StringUtils.format( - "current time[%d] is greater than nextFlush[%d]", - System.currentTimeMillis(), - nextFlush - )); - } if (rowsCurrentlyInMemory.get() >= tuningConfig.getMaxRowsInMemory()) { persist = true; persistReasons.add(StringUtils.format( @@ -531,21 +511,13 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) @Override public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) { - if (texasRanger == null) { - throw new IllegalStateException("Don't query me, bro."); - } - - return texasRanger.getQueryRunnerForIntervals(query, intervals); + throw new UnsupportedOperationException("No query runner for batch appenderator"); } @Override public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) { - if (texasRanger == null) { - throw new IllegalStateException("Don't query me, bro."); - } - - return texasRanger.getQueryRunnerForSegments(query, specs); + throw new UnsupportedOperationException("No query runner for batch appenderator"); } @Override @@ -714,7 +686,6 @@ public Object call() log.warn("Ingestion was throttled for [%,d] millis because persists were pending.", startDelay); } runExecStopwatch.stop(); - resetNextFlush(); // NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes. rowsCurrentlyInMemory.addAndGet(-numPersistedRows); @@ -1135,11 +1106,6 @@ private void shutdownExecutors() } } - private void resetNextFlush() - { - nextFlush = DateTimes.nowUtc().plus(tuningConfig.getIntermediatePersistPeriod()).getMillis(); - } - @VisibleForTesting public List getPersistedidentifierPaths() { From 10264dc3ba2f6dca3411df99bd3d5bd76fa941c9 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 4 Jun 2021 15:01:19 -0700 Subject: [PATCH 23/47] Cleanup unused code --- .../appenderator/BatchAppenderator.java | 104 ++++++++---------- 1 file changed, 47 insertions(+), 57 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 8e00f3e9638c..88eda45fd75b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -370,59 +370,57 @@ public AppenderatorAddResult add( )); } if (persist) { - if (allowIncrementalPersists) { - // persistAll clears rowsCurrentlyInMemory, no need to update it. - log.info("Incremental persist to disk because %s.", String.join(",", persistReasons)); - - long bytesToBePersisted = 0L; - for (Map.Entry entry : sinks.entrySet()) { - final Sink sinkEntry = entry.getValue(); - if (sinkEntry != null) { - bytesToBePersisted += sinkEntry.getBytesInMemory(); - if (sinkEntry.swappable()) { - // Code for batch no longer memory maps hydrants but they still take memory... - int memoryStillInUse = calculateMemoryUsedByHydrants(sink.getCurrHydrant()); - bytesCurrentlyInMemory.addAndGet(memoryStillInUse); - } + // persistAll clears rowsCurrentlyInMemory, no need to update it. + log.info("Incremental persist to disk because %s.", String.join(",", persistReasons)); + + long bytesToBePersisted = 0L; + for (Map.Entry entry : sinks.entrySet()) { + final Sink sinkEntry = entry.getValue(); + if (sinkEntry != null) { + bytesToBePersisted += sinkEntry.getBytesInMemory(); + if (sinkEntry.swappable()) { + // Code for batch no longer memory maps hydrants but they still take memory... + int memoryStillInUse = calculateMemoryUsedByHydrants(sink.getCurrHydrant()); + bytesCurrentlyInMemory.addAndGet(memoryStillInUse); } } + } - if (!skipBytesInMemoryOverheadCheck - && bytesCurrentlyInMemory.get() - bytesToBePersisted > maxBytesTuningConfig) { - // We are still over maxBytesTuningConfig even after persisting. - // This means that we ran out of all available memory to ingest (due to overheads created as part of ingestion) - final String alertMessage = StringUtils.format( - "Task has exceeded safe estimated heap usage limits, failing " - + "(numSinks: [%d] numHydrantsAcrossAllSinks: [%d] totalRows: [%d])" - + "(bytesCurrentlyInMemory: [%d] - bytesToBePersisted: [%d] > maxBytesTuningConfig: [%d])", - sinks.size(), - sinks.values().stream().mapToInt(Iterables::size).sum(), - getTotalRowCount(), - bytesCurrentlyInMemory.get(), - bytesToBePersisted, - maxBytesTuningConfig - ); - final String errorMessage = StringUtils.format( - "%s.\nThis can occur when the overhead from too many intermediary segment persists becomes to " - + "great to have enough space to process additional input rows. This check, along with metering the overhead " - + "of these objects to factor into the 'maxBytesInMemory' computation, can be disabled by setting " - + "'skipBytesInMemoryOverheadCheck' to 'true' (note that doing so might allow the task to naturally encounter " - + "a 'java.lang.OutOfMemoryError'). Alternatively, 'maxBytesInMemory' can be increased which will cause an " - + "increase in heap footprint, but will allow for more intermediary segment persists to occur before " - + "reaching this condition.", - alertMessage - ); - log.makeAlert(alertMessage) - .addData("dataSource", schema.getDataSource()) - .emit(); - throw new RuntimeException(errorMessage); - } + if (!skipBytesInMemoryOverheadCheck + && bytesCurrentlyInMemory.get() - bytesToBePersisted > maxBytesTuningConfig) { + // We are still over maxBytesTuningConfig even after persisting. + // This means that we ran out of all available memory to ingest (due to overheads created as part of ingestion) + final String alertMessage = StringUtils.format( + "Task has exceeded safe estimated heap usage limits, failing " + + "(numSinks: [%d] numHydrantsAcrossAllSinks: [%d] totalRows: [%d])" + + "(bytesCurrentlyInMemory: [%d] - bytesToBePersisted: [%d] > maxBytesTuningConfig: [%d])", + sinks.size(), + sinks.values().stream().mapToInt(Iterables::size).sum(), + getTotalRowCount(), + bytesCurrentlyInMemory.get(), + bytesToBePersisted, + maxBytesTuningConfig + ); + final String errorMessage = StringUtils.format( + "%s.\nThis can occur when the overhead from too many intermediary segment persists becomes to " + + "great to have enough space to process additional input rows. This check, along with metering the overhead " + + "of these objects to factor into the 'maxBytesInMemory' computation, can be disabled by setting " + + "'skipBytesInMemoryOverheadCheck' to 'true' (note that doing so might allow the task to naturally encounter " + + "a 'java.lang.OutOfMemoryError'). Alternatively, 'maxBytesInMemory' can be increased which will cause an " + + "increase in heap footprint, but will allow for more intermediary segment persists to occur before " + + "reaching this condition.", + alertMessage + ); + log.makeAlert(alertMessage) + .addData("dataSource", schema.getDataSource()) + .emit(); + throw new RuntimeException(errorMessage); + } - persistAllAndClear(); + persistAllAndClear(); - } else { - throw new ISE("Batch appenderator always persists as needed!"); - } + } else { + throw new ISE("Batch appenderator always persists as needed!"); } return new AppenderatorAddResult(identifier, sinksMetadata.get(identifier).numRowsInSegment, isPersistRequired); } @@ -436,13 +434,7 @@ public List getSegments() @Override public int getRowCount(final SegmentIdWithShardSpec identifier) { - final Sink sink = sinks.get(identifier); - - if (sink == null) { - throw new ISE("No such sink: %s", identifier); - } else { - return sink.getNumRows(); - } + return sinksMetadata.get(identifier).getNumRowsInSegment(); } @Override @@ -604,8 +596,6 @@ public ListenableFuture persistAll(@Nullable final Committer committer) throw new ISE("No sink for identifier: %s", identifier); } - int previousHydrantCount = totalHydrantsPersistedAcrossSinks.intValue(); - final List hydrants = Lists.newArrayList(sink); totalHydrantsCount.add(hydrants.size()); numPersistedRows += sink.getNumRowsInMemory(); From f74baafd47aa73011b044041f244206e80dc1296 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 4 Jun 2021 15:04:40 -0700 Subject: [PATCH 24/47] Removed unused code --- .../druid/segment/realtime/appenderator/BatchAppenderator.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 88eda45fd75b..7d1110f8ee97 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -419,8 +419,6 @@ public AppenderatorAddResult add( persistAllAndClear(); - } else { - throw new ISE("Batch appenderator always persists as needed!"); } return new AppenderatorAddResult(identifier, sinksMetadata.get(identifier).numRowsInSegment, isPersistRequired); } From 34cffcf1edaab116236973c51bf8bc0f3f3f0060 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 4 Jun 2021 16:28:01 -0700 Subject: [PATCH 25/47] Remove unused field --- .../segment/realtime/appenderator/BatchAppenderator.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 7d1110f8ee97..8d7366cd9488 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -145,11 +145,6 @@ private static class SinkMetadata * push time */ private int numHydrants; - /** Sinks when they are persisted lose information about the previous hydrant count, - * this variable remembers that so the proper directory can be created when persisting - * hydrants - */ - private int previousHydrantCount; public SinkMetadata() { From fac1ca108732a91a1e70f6b1b8eaa98e0525ea85 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 4 Jun 2021 17:11:20 -0700 Subject: [PATCH 26/47] Exclude it from jacoco because it is very hard to get branch coverage --- server/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/server/pom.xml b/server/pom.xml index cf0030d640f8..494f4d2ee763 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -470,6 +470,7 @@ org/apache/druid/metadata/BasicDataSourceExt.class + org/apache/druid/segment/realtime/appenderator/BatchAppenderator.class From af93370bb4a1a1dfd8eddf58dacabc83c8d2754b Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Mon, 7 Jun 2021 10:14:46 -0700 Subject: [PATCH 27/47] Remove segment announcement and some other minor cleanup --- .../appenderator/BatchAppenderator.java | 33 ++++--------------- 1 file changed, 6 insertions(+), 27 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 8d7366cd9488..fb3cb90e3e44 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -106,7 +106,6 @@ public class BatchAppenderator implements Appenderator private final FireDepartmentMetrics metrics; private final DataSegmentPusher dataSegmentPusher; private final ObjectMapper objectMapper; - private final DataSegmentAnnouncer segmentAnnouncer; private final IndexIO indexIO; private final IndexMerger indexMerger; private final Cache cache; @@ -137,21 +136,21 @@ public class BatchAppenderator implements Appenderator private static class SinkMetadata { /** This is used to maintain the rows in the sink accross persists of the sink - // used for functionality (i.e. to detect whether an incremental push - // is needed {@link AppenderatorDriverAddResult#isPushRequired(Integer, Long)} + * used for functionality (i.e. to detect whether an incremental push + * is needed {@link AppenderatorDriverAddResult#isPushRequired(Integer, Long)} **/ private int numRowsInSegment; - /** For sanity check to make sure that all hydrants for a sink are restored from disk at - * push time + /** For sanity check as well as functionality: to make sure that all hydrants for a sink are restored from disk at + * push time and also to remember the fire hydrant "count" when persisting it. */ private int numHydrants; public SinkMetadata() { - this(0, 0, 0); + this(0, 0); } - public SinkMetadata(int numRowsInSegment, int numHydrants, int previousHydrantCount) + public SinkMetadata(int numRowsInSegment, int numHydrants) { this.numRowsInSegment = numRowsInSegment; this.numHydrants = numHydrants; @@ -235,7 +234,6 @@ public int getNumHydrants() this.metrics = Preconditions.checkNotNull(metrics, "metrics"); this.dataSegmentPusher = Preconditions.checkNotNull(dataSegmentPusher, "dataSegmentPusher"); this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper"); - this.segmentAnnouncer = Preconditions.checkNotNull(segmentAnnouncer, "segmentAnnouncer"); this.indexIO = Preconditions.checkNotNull(indexIO, "indexIO"); this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger"); this.cache = cache; @@ -477,15 +475,6 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) ); bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed(retVal)); - try { - segmentAnnouncer.announceSegment(retVal.getSegment()); - } - catch (IOException e) { - log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource()) - .addData("interval", retVal.getInterval()) - .emit(); - } - sinks.put(identifier, retVal); metrics.setSinkCount(sinks.size()); } @@ -980,16 +969,6 @@ public void closeNow() } log.debug("Shutting down immediately..."); - for (Map.Entry entry : sinks.entrySet()) { - try { - segmentAnnouncer.unannounceSegment(entry.getValue().getSegment()); - } - catch (Exception e) { - log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) - .addData("identifier", entry.getKey().toString()) - .emit(); - } - } try { shutdownExecutors(); // We don't wait for pushExecutor to be terminated. See Javadoc for more details. From a019433768745dea40b2671502bb4b1ef3876702 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Mon, 7 Jun 2021 16:45:38 -0700 Subject: [PATCH 28/47] Add fallback flag --- .../appenderator/BatchAppenderatorTest.java | 102 +++++++++++++++++- .../appenderator/BatchAppenderatorTester.java | 6 +- .../realtime/appenderator/Appenderators.java | 21 ++++ 3 files changed, 124 insertions(+), 5 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java index 41f28d544054..f371ed07d776 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java @@ -141,6 +141,99 @@ public SegmentIdWithShardSpec apply(DataSegment input) } } + @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(); + boolean thrown; + + // startJob + Assert.assertEquals(null, appenderator.startJob()); + + // getDataSource + Assert.assertEquals(AppenderatorTester.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(), + new Function() + { + @Override + public SegmentIdWithShardSpec apply(DataSegment input) + { + return SegmentIdWithShardSpec.fromDataSegment(input); + } + } + ).stream().sorted().collect(Collectors.toList()) + ); + Assert.assertEquals( + tester.getPushedSegments().stream().sorted().collect(Collectors.toList()), + segmentsAndCommitMetadata.getSegments().stream().sorted().collect(Collectors.toList()) + ); + + appenderator.close(); + Assert.assertTrue(appenderator.getSegments().isEmpty()); + } + } @Test public void testMaxBytesInMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { @@ -151,7 +244,8 @@ public void testMaxBytesInMemoryWithSkipBytesInMemoryOverheadCheckConfig() throw null, true, new SimpleRowIngestionMeters(), - true + true, + false ) ) { final Appenderator appenderator = tester.getAppenderator(); @@ -184,7 +278,8 @@ public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadChec null, true, new SimpleRowIngestionMeters(), - true + true, + false ) ) { final Appenderator appenderator = tester.getAppenderator(); @@ -306,7 +401,8 @@ public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadChec null, true, new SimpleRowIngestionMeters(), - true + true, + false ) ) { final Appenderator appenderator = tester.getAppenderator(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java index feccfac5b75c..e1fe1fff0fc1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java @@ -111,6 +111,7 @@ public BatchAppenderatorTester( basePersistDirectory, enablePushFailure, new SimpleRowIngestionMeters(), + false, false ); } @@ -121,7 +122,8 @@ public BatchAppenderatorTester( final File basePersistDirectory, final boolean enablePushFailure, final RowIngestionMeters rowIngestionMeters, - final boolean skipBytesInMemoryOverheadCheck + final boolean skipBytesInMemoryOverheadCheck, + final boolean batchMemoryMappedIndex ) { objectMapper = new DefaultObjectMapper(); @@ -246,7 +248,7 @@ public Map makeLoadSpec(URI uri) indexMerger, rowIngestionMeters, new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false + batchMemoryMappedIndex ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 56cb2f409efc..405d78b7c55b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -108,6 +108,27 @@ public static Appenderator createOffline( boolean batchMemoryMappedIndex ) { + if (batchMemoryMappedIndex) { + // fallback to code know 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, + false + ); + } return new BatchAppenderator( id, schema, From a296703d8138385b2607a3d4a95a8531d541eb21 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Mon, 21 Jun 2021 17:48:10 -0700 Subject: [PATCH 29/47] Minor code cleanup --- .../realtime/appenderator/Appenderators.java | 1 - .../realtime/appenderator/BatchAppenderator.java | 16 +--------------- 2 files changed, 1 insertion(+), 16 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 405d78b7c55b..510d67c063d1 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -140,7 +140,6 @@ public static Appenderator createOffline( null, indexIO, indexMerger, - null, rowIngestionMeters, parseExceptionHandler ); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index fb3cb90e3e44..d8db7366af78 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -108,7 +108,6 @@ public class BatchAppenderator implements Appenderator private final ObjectMapper objectMapper; private final IndexIO indexIO; private final IndexMerger indexMerger; - private final Cache cache; /** * This map needs to be concurrent because it's accessed and mutated from multiple threads: both the thread from where * this Appenderator is used (and methods like {@link #add(SegmentIdWithShardSpec, InputRow, Supplier, boolean)} are @@ -121,8 +120,7 @@ public class BatchAppenderator implements Appenderator /** * 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. For now, {@link SinkMetadata} only - * contains a single memeber {@link SinkMetadata#numRowsInSegment} but we can add more in the future as needed + * are being completely removed from memory after each incremental persist. */ private final ConcurrentHashMap sinksMetadata = new ConcurrentHashMap<>(); @@ -218,7 +216,6 @@ public int getNumHydrants() @Nullable SinkQuerySegmentWalker sinkQuerySegmentWalker, IndexIO indexIO, IndexMerger indexMerger, - Cache cache, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler ) @@ -236,7 +233,6 @@ public int getNumHydrants() this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper"); this.indexIO = Preconditions.checkNotNull(indexIO, "indexIO"); this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger"); - this.cache = cache; this.rowIngestionMeters = Preconditions.checkNotNull(rowIngestionMeters, "rowIngestionMeters"); this.parseExceptionHandler = Preconditions.checkNotNull(parseExceptionHandler, "parseExceptionHandler"); @@ -865,8 +861,6 @@ private DataSegment mergeAndPush( final long pushFinishTime = System.nanoTime(); - //objectMapper.writeValue(descriptorFile, segment); - log.info( "Segment[%s] of %,d bytes " + "built from %d incremental persist(s) in %,dms; " @@ -1144,7 +1138,6 @@ Pair getIdentifierAndSinkForPersistedFile(File ide hydrants ); currSink.finishWriting(); // this sink is not writable - //sinks.put(identifier, currSink); return new Pair<>(identifier, currSink); } catch (IOException e) { @@ -1194,13 +1187,6 @@ public Void apply(@Nullable Object input) metrics.setSinkCount(sinks.size()); - for (FireHydrant hydrant : sink) { - if (cache != null) { - cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant)); - } - hydrant.swapSegment(null); - } - if (removeOnDiskData) { removeDirectory(computePersistDir(identifier)); } From a6450418060d98de8ced2623b9aa47301e18e071 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Mon, 21 Jun 2021 18:59:06 -0700 Subject: [PATCH 30/47] Checkstyle --- .../druid/segment/realtime/appenderator/BatchAppenderator.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index d8db7366af78..999df6688267 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -34,7 +34,6 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.commons.lang.mutable.MutableInt; -import org.apache.druid.client.cache.Cache; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.FileUtils; From 2b321f5e00fe4e3f975197fd94537645a82a6a17 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Thu, 24 Jun 2021 23:14:39 -0700 Subject: [PATCH 31/47] Code review changes --- .../BatchAppenderatorDriverTest.java | 25 +- .../appenderator/BatchAppenderatorTest.java | 30 +- .../appenderator/BatchAppenderatorTester.java | 3 +- .../appenderator/TestUsedSegmentChecker.java | 61 --- server/pom.xml | 1 - .../realtime/appenderator/Appenderator.java | 14 +- .../realtime/appenderator/Appenderators.java | 3 +- .../appenderator/BaseAppenderatorDriver.java | 1 + .../appenderator/BatchAppenderator.java | 492 ++++++------------ .../appenderator/StreamAppenderator.java | 13 +- .../StreamAppenderatorDriverTest.java | 2 +- .../appenderator/TestUsedSegmentChecker.java | 9 +- 12 files changed, 193 insertions(+), 461 deletions(-) delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/appenderator/TestUsedSegmentChecker.java diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorDriverTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorDriverTest.java index b319f1f5b85f..31c97b33420d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorDriverTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorDriverTest.java @@ -38,6 +38,7 @@ import org.apache.druid.segment.realtime.appenderator.SegmentWithState; import org.apache.druid.segment.realtime.appenderator.SegmentWithState.SegmentState; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; +import org.apache.druid.segment.realtime.appenderator.TestUsedSegmentChecker; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.easymock.EasyMock; @@ -100,7 +101,7 @@ public void setup() driver = new BatchAppenderatorDriver( appenderatorTester.getAppenderator(), allocator, - new TestUsedSegmentChecker(appenderatorTester), + new TestUsedSegmentChecker(appenderatorTester.getPushedSegments()), dataSegmentKiller ); @@ -230,18 +231,16 @@ public SegmentIdWithShardSpec allocate( final boolean skipSegmentLineageCheck ) { - synchronized (counters) { - DateTime dateTimeTruncated = granularity.bucketStart(row.getTimestamp()); - final long timestampTruncated = dateTimeTruncated.getMillis(); - counters.putIfAbsent(timestampTruncated, new AtomicInteger()); - final int partitionNum = counters.get(timestampTruncated).getAndIncrement(); - return new SegmentIdWithShardSpec( - dataSource, - granularity.bucket(dateTimeTruncated), - VERSION, - new NumberedShardSpec(partitionNum, 0) - ); - } + DateTime dateTimeTruncated = granularity.bucketStart(row.getTimestamp()); + final long timestampTruncated = dateTimeTruncated.getMillis(); + counters.putIfAbsent(timestampTruncated, new AtomicInteger()); + final int partitionNum = counters.get(timestampTruncated).getAndIncrement(); + return new SegmentIdWithShardSpec( + dataSource, + granularity.bucket(dateTimeTruncated), + VERSION, + new NumberedShardSpec(partitionNum, 0) + ); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java index f371ed07d776..2795f57bcf41 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java @@ -44,7 +44,6 @@ import java.io.File; import java.util.Collections; import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; public class BatchAppenderatorTest extends InitializedNullHandlingTest @@ -60,10 +59,9 @@ public void testSimpleIngestion() throws Exception { try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); - boolean thrown; // startJob - Assert.assertEquals(null, appenderator.startJob()); + Assert.assertNull(appenderator.startJob()); // getDataSource Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource()); @@ -154,7 +152,6 @@ public void testSimpleIngestionWithFallbackCodePath() throws Exception true )) { final Appenderator appenderator = tester.getAppenderator(); - boolean thrown; // startJob Assert.assertEquals(null, appenderator.startJob()); @@ -283,7 +280,6 @@ public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadChec ) ) { final Appenderator appenderator = tester.getAppenderator(); - final AtomicInteger eventCount = new AtomicInteger(0); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); @@ -306,7 +302,6 @@ public void testMaxBytesInMemory() throws Exception { try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(100, 15000, true)) { final Appenderator appenderator = tester.getAppenderator(); - final AtomicInteger eventCount = new AtomicInteger(0); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); @@ -385,7 +380,6 @@ public void testTaskFailAsPersistCannotFreeAnyMoreMemory() throws Exception try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(100, 5180, true)) { final Appenderator appenderator = tester.getAppenderator(); - final AtomicInteger eventCount = new AtomicInteger(0); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); } @@ -406,7 +400,6 @@ public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadChec ) ) { final Appenderator appenderator = tester.getAppenderator(); - final AtomicInteger eventCount = new AtomicInteger(0); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); @@ -430,7 +423,6 @@ public void testTaskCleanupInMemoryCounterAfterCloseWithRowInMemory() throws Exc try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(100, 10000, true)) { final Appenderator appenderator = tester.getAppenderator(); - final AtomicInteger eventCount = new AtomicInteger(0); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); @@ -458,7 +450,6 @@ public void testMaxBytesInMemoryInMultipleSinks() throws Exception try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(1000, 28748, true)) { final Appenderator appenderator = tester.getAppenderator(); - final AtomicInteger eventCount = new AtomicInteger(0); appenderator.startJob(); // next records are 182 bytes @@ -578,7 +569,6 @@ public void testIgnoreMaxBytesInMemory() throws Exception try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(100, -1, true)) { final Appenderator appenderator = tester.getAppenderator(); - final AtomicInteger eventCount = new AtomicInteger(0); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.startJob(); @@ -611,7 +601,6 @@ public void testMaxRowsInMemory() throws Exception { try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); - final AtomicInteger eventCount = new AtomicInteger(0); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.startJob(); @@ -644,7 +633,6 @@ public void testAllHydrantsAreRecovered() throws Exception { try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(1, false)) { final Appenderator appenderator = tester.getAppenderator(); - final AtomicInteger eventCount = new AtomicInteger(0); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.startJob(); @@ -685,7 +673,6 @@ public void testTotalRowsPerSegment() throws Exception { try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); - final AtomicInteger eventCount = new AtomicInteger(0); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); appenderator.startJob(); @@ -763,12 +750,14 @@ public void testRestoreFromDisk() throws Exception Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); List segmentPaths = ((BatchAppenderator) appenderator).getPersistedidentifierPaths(); + Assert.assertNotNull(segmentPaths); Assert.assertEquals(3, segmentPaths.size()); appenderator.push(IDENTIFIERS, null, false).get(); segmentPaths = ((BatchAppenderator) appenderator).getPersistedidentifierPaths(); + Assert.assertNotNull(segmentPaths); Assert.assertEquals(0, segmentPaths.size()); appenderator.close(); @@ -786,29 +775,31 @@ public void testCleanupFromDiskAfterClose() throws Exception 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()); - Assert.assertEquals(2, ((BatchAppenderator) appenderator).getTotalRowCount()); + Assert.assertEquals(2, appenderator.getTotalRowCount()); 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()); - Assert.assertEquals(4, ((BatchAppenderator) appenderator).getTotalRowCount()); + Assert.assertEquals(4, appenderator.getTotalRowCount()); 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()); - Assert.assertEquals(5, ((BatchAppenderator) appenderator).getTotalRowCount()); + Assert.assertEquals(5, appenderator.getTotalRowCount()); List segmentPaths = ((BatchAppenderator) appenderator).getPersistedidentifierPaths(); + Assert.assertNotNull(segmentPaths); Assert.assertEquals(3, segmentPaths.size()); appenderator.close(); segmentPaths = ((BatchAppenderator) appenderator).getPersistedidentifierPaths(); + Assert.assertNotNull(segmentPaths); Assert.assertEquals(0, segmentPaths.size()); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); - Assert.assertEquals(0, ((BatchAppenderator) appenderator).getTotalRowCount()); + Assert.assertEquals(0, appenderator.getTotalRowCount()); } @@ -875,8 +866,7 @@ public void testVerifyRowIngestionMetrics() throws Exception Assert.assertEquals(0, rowIngestionMeters.getThrownAway()); } } - - + private static SegmentIdWithShardSpec createSegmentId(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java index e1fe1fff0fc1..e42f10b52ae4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java @@ -51,6 +51,7 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.net.URI; @@ -119,7 +120,7 @@ public BatchAppenderatorTester( public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, - final File basePersistDirectory, + @Nullable final File basePersistDirectory, final boolean enablePushFailure, final RowIngestionMeters rowIngestionMeters, final boolean skipBytesInMemoryOverheadCheck, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/TestUsedSegmentChecker.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/TestUsedSegmentChecker.java deleted file mode 100644 index 5a89a81cfff6..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/TestUsedSegmentChecker.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.appenderator; - -import com.google.common.collect.Ordering; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.TimelineObjectHolder; -import org.apache.druid.timeline.VersionedIntervalTimeline; -import org.apache.druid.timeline.partition.PartitionChunk; - -import java.util.HashSet; -import java.util.Set; - -public class TestUsedSegmentChecker implements UsedSegmentChecker -{ - private final BatchAppenderatorTester appenderatorTester; - - public TestUsedSegmentChecker(BatchAppenderatorTester appenderatorTester) - { - this.appenderatorTester = appenderatorTester; - } - - @Override - public Set findUsedSegments(Set identifiers) - { - final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); - VersionedIntervalTimeline.addSegments(timeline, appenderatorTester.getPushedSegments().iterator()); - - final Set retVal = new HashSet<>(); - for (SegmentIdWithShardSpec identifier : identifiers) { - for (TimelineObjectHolder holder : timeline.lookup(identifier.getInterval())) { - for (PartitionChunk chunk : holder.getObject()) { - if (identifiers.contains(SegmentIdWithShardSpec.fromDataSegment(chunk.getObject()))) { - retVal.add(chunk.getObject()); - } - } - } - } - - return retVal; - } -} diff --git a/server/pom.xml b/server/pom.xml index d4d4f4ba2c20..a9569a38aef3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -470,7 +470,6 @@ org/apache/druid/metadata/BasicDataSourceExt.class - org/apache/druid/segment/realtime/appenderator/BatchAppenderator.class diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java index a0793020c8ba..2d538e77a473 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java @@ -214,18 +214,6 @@ ListenableFuture push( */ void closeNow(); - /** - * Flag to tell internals whether appenderator is working on behalf of a real time task. - * This is to manage certain aspects as needed. For example, for batch, non-real time tasks, - * physical segments (i.e. hydrants) do not need to memory map their persisted - * files. In this case, the code will avoid memory mapping them thus ameliorating the occurance - * of OOMs. - */ - default boolean isRealTime() - { - return false; - } - /** * Result of {@link Appenderator#add} containing following information * - {@link SegmentIdWithShardSpec} - identifier of segment to which rows are being added @@ -238,7 +226,7 @@ class AppenderatorAddResult private final int numRowsInSegment; private final boolean isPersistRequired; - public AppenderatorAddResult( + AppenderatorAddResult( SegmentIdWithShardSpec identifier, int numRowsInSegment, boolean isPersistRequired diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 510d67c063d1..56c9b301b477 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -109,7 +109,7 @@ public static Appenderator createOffline( ) { if (batchMemoryMappedIndex) { - // fallback to code know to be working, this is just a fallback option in case new + // 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( @@ -136,7 +136,6 @@ public static Appenderator createOffline( metrics, dataSegmentPusher, objectMapper, - new NoopDataSegmentAnnouncer(), null, indexIO, indexMerger, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java index c0d646820d66..bdd572cc1af9 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java @@ -260,6 +260,7 @@ Stream getAllSegmentsOfInterval() ); } + @VisibleForTesting public Map getSegments() { return segments; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 999df6688267..c7d3aa22aacf 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -21,7 +21,6 @@ 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; @@ -31,8 +30,6 @@ import com.google.common.primitives.Ints; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import org.apache.commons.lang.mutable.MutableInt; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputRow; @@ -43,7 +40,6 @@ 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; @@ -63,7 +59,6 @@ import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.segment.realtime.plumber.Sink; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -79,10 +74,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -108,9 +101,9 @@ public class BatchAppenderator implements Appenderator private final IndexIO indexIO; private final IndexMerger indexMerger; /** - * This map needs to be concurrent because it's accessed and mutated from multiple threads: both the thread from where + * This map needs to be concurrent because it's accessed and mutated from multiple threads from where * this Appenderator is used (and methods like {@link #add(SegmentIdWithShardSpec, InputRow, Supplier, boolean)} are - * called) and from {@link #persistExecutor}. It could also be accessed (but not mutated) potentially in the context + * called). It could also be accessed (but not mutated) potentially in the context * of any thread from {@link #drop}. */ private final ConcurrentMap sinks = new ConcurrentHashMap<>(); @@ -184,17 +177,9 @@ public int getNumHydrants() private final AtomicBoolean closed = new AtomicBoolean(false); - private volatile ListeningExecutorService persistExecutor = null; - private volatile ListeningExecutorService pushExecutor = null; - // use intermediate executor so that deadlock conditions can be prevented - // where persist and push Executor try to put tasks in each other queues - // thus creating circular dependency - private volatile ListeningExecutorService intermediateTempExecutor = null; private volatile FileLock basePersistDirLock = null; private volatile FileChannel basePersistDirLockChannel = null; - private volatile Throwable persistError; - /** * This constructor allows the caller to provide its own SinkQuerySegmentWalker. *

@@ -211,7 +196,6 @@ public int getNumHydrants() FireDepartmentMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, - DataSegmentAnnouncer segmentAnnouncer, @Nullable SinkQuerySegmentWalker sinkQuerySegmentWalker, IndexIO indexIO, IndexMerger indexMerger, @@ -256,17 +240,9 @@ public Object startJob() { tuningConfig.getBasePersistDirectory().mkdirs(); lockBasePersistDirectory(); - initializeExecutors(); return null; } - private void throwPersistErrorIfExists() - { - if (persistError != null) { - throw new RE(persistError, "Error while persisting"); - } - } - @Override public AppenderatorAddResult add( final SegmentIdWithShardSpec identifier, @@ -276,8 +252,6 @@ public AppenderatorAddResult add( ) throws IndexSizeExceededException, SegmentNotWritableException { - throwPersistErrorIfExists(); - Preconditions.checkArgument( committerSupplier == null, "Batch appenderator does not need a committer!" @@ -331,9 +305,8 @@ public AppenderatorAddResult add( rowsCurrentlyInMemory.addAndGet(numAddedRows); bytesCurrentlyInMemory.addAndGet(bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd); totalRows.addAndGet(numAddedRows); - sinksMetadata.computeIfAbsent(identifier, Void -> new SinkMetadata()).addRows(numAddedRows); + sinksMetadata.computeIfAbsent(identifier, unused -> new SinkMetadata()).addRows(numAddedRows); - boolean isPersistRequired = false; boolean persist = false; List persistReasons = new ArrayList<>(); @@ -368,7 +341,7 @@ public AppenderatorAddResult add( bytesToBePersisted += sinkEntry.getBytesInMemory(); if (sinkEntry.swappable()) { // Code for batch no longer memory maps hydrants but they still take memory... - int memoryStillInUse = calculateMemoryUsedByHydrants(sink.getCurrHydrant()); + int memoryStillInUse = calculateMemoryUsedByHydrant(); bytesCurrentlyInMemory.addAndGet(memoryStillInUse); } } @@ -408,7 +381,7 @@ public AppenderatorAddResult add( persistAllAndClear(); } - return new AppenderatorAddResult(identifier, sinksMetadata.get(identifier).numRowsInSegment, isPersistRequired); + return new AppenderatorAddResult(identifier, sinksMetadata.get(identifier).numRowsInSegment, false); } @Override @@ -468,7 +441,7 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) maxBytesTuningConfig, null ); - bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed(retVal)); + bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed()); sinks.put(identifier, retVal); metrics.setSinkCount(sinks.size()); @@ -495,22 +468,13 @@ public void clear() throws InterruptedException clear(true); } - private void clear(boolean removeOnDiskData) throws InterruptedException + private void clear(boolean removeOnDiskData) { // Drop commit metadata, then abandon all segments. log.info("Clearing all sinks & hydrants, removing data on disk: [%s]", removeOnDiskData); - try { - throwPersistErrorIfExists(); - // Drop everything. - final List> futures = new ArrayList<>(); - for (Map.Entry entry : sinks.entrySet()) { - futures.add(removeSink(entry.getKey(), entry.getValue(), removeOnDiskData)); - } - // Await dropping. - Futures.allAsList(futures).get(); - } - catch (ExecutionException e) { - throw new RuntimeException(e); + // Drop everything. + for (Map.Entry entry : sinks.entrySet()) { + removeSink(entry.getKey(), entry.getValue(), removeOnDiskData); } } @@ -529,42 +493,31 @@ public ListenableFuture drop(final SegmentIdWithShardSpec identifier) totalRows.set(Math.max(totalRowsAfter, 0)); } if (sink != null) { - return removeSink(identifier, sink, true); - } else { - return Futures.immediateFuture(null); + removeSink(identifier, sink, true); } + return Futures.immediateFuture(null); } - private SegmentsAndCommitMetadata persistAllAndClear() + private void persistAllAndClear() { - final ListenableFuture toPersist = Futures.transform( - persistAll(null), - (Function) future -> future - ); - // make sure sinks are cleared before push is called - final SegmentsAndCommitMetadata commitMetadata; try { - commitMetadata = (SegmentsAndCommitMetadata) toPersist.get(); + persistAll(null).get(); clear(false); - return commitMetadata; } catch (Throwable t) { - persistError = t; + throw new RE(t, "Error while persisting"); } - return null; } @Override public ListenableFuture persistAll(@Nullable final Committer committer) { - throwPersistErrorIfExists(); final List> indexesToPersist = new ArrayList<>(); int numPersistedRows = 0; long bytesPersisted = 0L; MutableInt totalHydrantsCount = new MutableInt(); MutableInt totalHydrantsPersistedAcrossSinks = new MutableInt(); - SegmentIdWithShardSpec startIdentifier = null; final long totalSinks = sinks.size(); for (Map.Entry entry : sinks.entrySet()) { final SegmentIdWithShardSpec identifier = entry.getKey(); @@ -604,48 +557,39 @@ public ListenableFuture persistAll(@Nullable final Committer committer) final Stopwatch runExecStopwatch = Stopwatch.createStarted(); final Stopwatch persistStopwatch = Stopwatch.createStarted(); AtomicLong totalPersistedRows = new AtomicLong(numPersistedRows); - final ListenableFuture future = persistExecutor.submit( - new Callable() - { - @Override - public Object call() - { - try { - for (Pair pair : indexesToPersist) { - metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs)); - } - - log.info( - "Persisted in-memory data for segments: %s", - indexesToPersist.stream() - .map(itp -> itp.rhs.asSegmentId().toString()) - .distinct() - .collect(Collectors.joining(", ")) - ); - log.info( - "Persisted stats: processed rows: [%d], persisted rows[%d], sinks: [%d], total fireHydrants (across sinks): [%d], persisted fireHydrants (across sinks): [%d]", - rowIngestionMeters.getProcessed(), - totalPersistedRows.get(), - totalSinks, - totalHydrantsCount.longValue(), - totalHydrantsPersistedAcrossSinks.longValue() - ); - - // return null if committer is null - return null; - } - catch (Exception e) { - metrics.incrementFailedPersists(); - throw e; - } - finally { - metrics.incrementNumPersists(); - metrics.incrementPersistTimeMillis(persistStopwatch.elapsed(TimeUnit.MILLISECONDS)); - persistStopwatch.stop(); - } - } - } - ); + + try { + for (Pair pair : indexesToPersist) { + metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs)); + } + + log.info( + "Persisted in-memory data for segments: %s", + indexesToPersist.stream() + .filter(itp -> itp.rhs != null) + .map(itp -> itp.rhs.asSegmentId().toString()) + .distinct() + .collect(Collectors.joining(", ")) + ); + log.info( + "Persisted stats: processed rows: [%d], persisted rows[%d], sinks: [%d], total fireHydrants (across sinks): [%d], persisted fireHydrants (across sinks): [%d]", + rowIngestionMeters.getProcessed(), + totalPersistedRows.get(), + totalSinks, + totalHydrantsCount.longValue(), + totalHydrantsPersistedAcrossSinks.longValue() + ); + + } + catch (Exception e) { + metrics.incrementFailedPersists(); + throw e; + } + finally { + metrics.incrementNumPersists(); + metrics.incrementPersistTimeMillis(persistStopwatch.elapsed(TimeUnit.MILLISECONDS)); + persistStopwatch.stop(); + } final long startDelay = runExecStopwatch.elapsed(TimeUnit.MILLISECONDS); metrics.incrementPersistBackPressureMillis(startDelay); @@ -660,7 +604,7 @@ public Object call() log.info("Persisted rows[%,d] and bytes[%,d]", numPersistedRows, bytesPersisted); - return future; + return Futures.immediateFuture(null); } @Override @@ -676,57 +620,40 @@ public ListenableFuture push( } // Any sinks not persisted so far will be persisted before push: - final SegmentsAndCommitMetadata commitMetadata = persistAllAndClear(); - - final ListenableFuture pushFuture = pushExecutor.submit( - new Callable() - { - @Override - public SegmentsAndCommitMetadata call() - { - log.info("Preparing to push..."); - - final List dataSegments = new ArrayList<>(); - List persistedIdentifiers = getPersistedidentifierPaths(); - for (File identifier : persistedIdentifiers) { - Pair identifiersAndSinks = getIdentifierAndSinkForPersistedFile(identifier); - final DataSegment dataSegment = mergeAndPush( - identifiersAndSinks.lhs, - identifiersAndSinks.rhs, - useUniquePath - ); - if (dataSegment != null) { - dataSegments.add(dataSegment); - } else { - log.warn("mergeAndPush[%s] returned null, skipping.", identifiersAndSinks.lhs); - } - } - - log.info("Push complete..."); - - return new SegmentsAndCommitMetadata(dataSegments, commitMetadata); - } - }); + persistAllAndClear(); - return pushFuture; - } + log.info("Preparing to push..."); + final List dataSegments = new ArrayList<>(); + List persistedIdentifiers = getPersistedidentifierPaths(); + if (persistedIdentifiers == null) { + throw new ISE("Identifiers were persisted but could not be retrieved"); + } + for (File identifier : persistedIdentifiers) { + Pair identifiersAndSinks; + try { + identifiersAndSinks = getIdentifierAndSinkForPersistedFile(identifier); + } + catch (IOException e) { + throw new ISE(e, "Failed to retrieve sinks for identifier", identifier); + } + final DataSegment dataSegment = mergeAndPush( + identifiersAndSinks.lhs, + identifiersAndSinks.rhs, + useUniquePath + ); + if (dataSegment != null) { + dataSegments.add(dataSegment); + } else { + log.warn("mergeAndPush[%s] returned null, skipping.", identifiersAndSinks.lhs); + } + } + log.info("Push complete..."); - /** - * Insert a barrier into the merge-and-push queue. When this future resolves, all pending pushes will have finished. - * This is useful if we're going to do something that would otherwise potentially break currently in-progress - * pushes. - */ - private ListenableFuture pushBarrier() - { - return intermediateTempExecutor.submit( - (Runnable) () -> pushExecutor.submit(() -> { - }) - ); + return Futures.immediateFuture(new SegmentsAndCommitMetadata(dataSegments, null)); } /** - * Merge segment, push to deep storage. Should only be used on segments that have been fully persisted. Must only - * be run in the single-threaded pushExecutor. + * Merge segment, push to deep storage. Should only be used on segments that have been fully persisted. * * @param identifier sink identifier * @param sink sink to push @@ -764,7 +691,7 @@ private DataSegment mergeAndPush( SinkMetadata sm = sinksMetadata.get(identifier); if (sm == null) { log.warn("Sink metadata not found just before merge for identifier [%s]", identifier); - } else if (numHydrants != sinksMetadata.get(identifier).getNumHydrants()) { + } else if (numHydrants != sm.getNumHydrants()) { throw new ISE("Number of restored hydrants[%d] for identifier[%s] does not match expected value[%d]", numHydrants, identifier, sinksMetadata.get(identifier).getNumHydrants()); } @@ -892,53 +819,18 @@ public void close() log.debug("Shutting down..."); - final List> futures = new ArrayList<>(); for (Map.Entry entry : sinks.entrySet()) { - futures.add(removeSink(entry.getKey(), entry.getValue(), false)); - } - - try { - Futures.allAsList(futures).get(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - log.warn(e, "Interrupted during close()"); - } - catch (ExecutionException e) { - log.warn(e, "Unable to abandon existing segments during close()"); + removeSink(entry.getKey(), entry.getValue(), false); } - try { - shutdownExecutors(); - Preconditions.checkState( - persistExecutor == null || persistExecutor.awaitTermination(365, TimeUnit.DAYS), - "persistExecutor not terminated" - ); - Preconditions.checkState( - pushExecutor == null || pushExecutor.awaitTermination(365, TimeUnit.DAYS), - "pushExecutor not terminated" - ); - Preconditions.checkState( - intermediateTempExecutor == null || intermediateTempExecutor.awaitTermination(365, TimeUnit.DAYS), - "intermediateTempExecutor not terminated" - ); - persistExecutor = null; - pushExecutor = null; - intermediateTempExecutor = null; - - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new ISE("Failed to shutdown executors during close()"); - } - - // Only unlock if executors actually shut down. unlockBasePersistDirectory(); // cleanup: List persistedIdentifiers = getPersistedidentifierPaths(); - for (File identifier : persistedIdentifiers) { - removeDirectory(identifier); + if (persistedIdentifiers != null) { + for (File identifier : persistedIdentifiers) { + removeDirectory(identifier); + } } totalRows.set(0); @@ -946,12 +838,7 @@ public void close() } /** - * Unannounce the segments and wait for outstanding persists to finish. - * Do not unlock base persist dir as we are not waiting for push executor to shut down - * relying on current JVM to shutdown to not cause any locking problem if the task is restored. - * In case when task is restored and current task is still active because of push executor (which it shouldn't be - * since push executor starts daemon threads) then the locking should fail and new task should fail to start. - * This also means that this method should only be called when task is shutting down. + Nothing to do since there are no executors */ @Override public void closeNow() @@ -962,24 +849,6 @@ public void closeNow() } log.debug("Shutting down immediately..."); - try { - shutdownExecutors(); - // We don't wait for pushExecutor to be terminated. See Javadoc for more details. - Preconditions.checkState( - persistExecutor == null || persistExecutor.awaitTermination(365, TimeUnit.DAYS), - "persistExecutor not terminated" - ); - Preconditions.checkState( - intermediateTempExecutor == null || intermediateTempExecutor.awaitTermination(365, TimeUnit.DAYS), - "intermediateTempExecutor not terminated" - ); - persistExecutor = null; - intermediateTempExecutor = null; - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new ISE("Failed to shutdown executors during close()"); - } } private void lockBasePersistDirectory() @@ -1017,51 +886,8 @@ private void unlockBasePersistDirectory() } } - private void initializeExecutors() - { - final int maxPendingPersists = tuningConfig.getMaxPendingPersists(); - - if (persistExecutor == null) { - // use a blocking single threaded executor to throttle the firehose when write to disk is slow - persistExecutor = MoreExecutors.listeningDecorator( - Execs.newBlockingSingleThreaded( - "[" + StringUtils.encodeForFormat(myId) + "]-appenderator-persist", - maxPendingPersists - ) - ); - } - - if (pushExecutor == null) { - // use a blocking single threaded executor to throttle the firehose when write to disk is slow - pushExecutor = MoreExecutors.listeningDecorator( - Execs.newBlockingSingleThreaded("[" + StringUtils.encodeForFormat(myId) + "]-appenderator-merge", 1) - ); - } - - if (intermediateTempExecutor == null) { - // use single threaded executor with SynchronousQueue so that all abandon operations occur sequentially - intermediateTempExecutor = MoreExecutors.listeningDecorator( - Execs.newBlockingSingleThreaded("[" + StringUtils.encodeForFormat(myId) + "]-appenderator-abandon", 0) - ); - } - } - - private void shutdownExecutors() - { - if (persistExecutor != null) { - persistExecutor.shutdownNow(); - } - - if (pushExecutor != null) { - pushExecutor.shutdownNow(); - } - - if (intermediateTempExecutor != null) { - intermediateTempExecutor.shutdownNow(); - } - } - @VisibleForTesting + @Nullable public List getPersistedidentifierPaths() { @@ -1089,65 +915,61 @@ public List getPersistedidentifierPaths() return retVal; } - Pair getIdentifierAndSinkForPersistedFile(File identifierPath) + private Pair getIdentifierAndSinkForPersistedFile(File identifierPath) + throws IOException { - try { - final SegmentIdWithShardSpec identifier = objectMapper.readValue( - new File(identifierPath, "identifier.json"), - SegmentIdWithShardSpec.class - ); - - // To avoid reading and listing of "merged" dir and other special files - final File[] sinkFiles = identifierPath.listFiles( - (dir, fileName) -> !(Ints.tryParse(fileName) == null) - ); + final SegmentIdWithShardSpec identifier = objectMapper.readValue( + new File(identifierPath, IDENTIFIER_FILE_NAME), + SegmentIdWithShardSpec.class + ); - Arrays.sort( - sinkFiles, - (o1, o2) -> Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName())) - ); + // To avoid reading and listing of "merged" dir and other special files + final File[] sinkFiles = identifierPath.listFiles( + (dir, fileName) -> !(Ints.tryParse(fileName) == null) + ); + if (sinkFiles == null) { + throw new ISE("Problem reading persisted sinks in path", identifierPath); + } - List hydrants = new ArrayList<>(); - for (File hydrantDir : sinkFiles) { - final int hydrantNumber = Integer.parseInt(hydrantDir.getName()); + Arrays.sort( + sinkFiles, + (o1, o2) -> Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName())) + ); - log.debug("Loading previously persisted partial segment at [%s]", hydrantDir); - if (hydrantNumber != hydrants.size()) { - throw new ISE("Missing hydrant [%,d] in identifier [%s].", hydrants.size(), identifier); - } + List hydrants = new ArrayList<>(); + for (File hydrantDir : sinkFiles) { + final int hydrantNumber = Integer.parseInt(hydrantDir.getName()); - hydrants.add( - new FireHydrant( - new QueryableIndexSegment(indexIO.loadIndex(hydrantDir), identifier.asSegmentId()), - hydrantNumber - ) - ); + log.debug("Loading previously persisted partial segment at [%s]", hydrantDir); + if (hydrantNumber != hydrants.size()) { + throw new ISE("Missing hydrant [%,d] in identifier [%s].", hydrants.size(), identifier); } - Sink currSink = new Sink( - identifier.getInterval(), - schema, - identifier.getShardSpec(), - identifier.getVersion(), - tuningConfig.getAppendableIndexSpec(), - tuningConfig.getMaxRowsInMemory(), - maxBytesTuningConfig, - null, - hydrants + hydrants.add( + new FireHydrant( + new QueryableIndexSegment(indexIO.loadIndex(hydrantDir), identifier.asSegmentId()), + hydrantNumber + ) ); - currSink.finishWriting(); // this sink is not writable - return new Pair<>(identifier, currSink); } - catch (IOException e) { - log.makeAlert(e, "Problem loading sink[%s] from disk.", schema.getDataSource()) - .addData("identifier path", identifierPath) - .emit(); - } - return null; + + Sink currSink = new Sink( + identifier.getInterval(), + schema, + identifier.getShardSpec(), + identifier.getVersion(), + tuningConfig.getAppendableIndexSpec(), + tuningConfig.getMaxRowsInMemory(), + maxBytesTuningConfig, + null, + hydrants + ); + currSink.finishWriting(); // this sink is not writable + return new Pair<>(identifier, currSink); } - private ListenableFuture removeSink( + private void removeSink( final SegmentIdWithShardSpec identifier, final Sink sink, final boolean removeOnDiskData @@ -1159,46 +981,30 @@ private ListenableFuture removeSink( // i.e. those that haven't been persisted for *InMemory counters, or pushed to deep storage for the total counter. rowsCurrentlyInMemory.addAndGet(-sink.getNumRowsInMemory()); bytesCurrentlyInMemory.addAndGet(-sink.getBytesInMemory()); - bytesCurrentlyInMemory.addAndGet(-calculateSinkMemoryInUsed(sink)); + bytesCurrentlyInMemory.addAndGet(-calculateSinkMemoryInUsed()); for (FireHydrant hydrant : sink) { // Decrement memory used by all Memory Mapped Hydrant if (!hydrant.equals(sink.getCurrHydrant())) { - bytesCurrentlyInMemory.addAndGet(-calculateMemoryUsedByHydrants(hydrant)); + bytesCurrentlyInMemory.addAndGet(-calculateMemoryUsedByHydrant()); } } // 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. } - // Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread. - return Futures.transform( - pushBarrier(), - new Function() - { - @Nullable - @Override - public Void apply(@Nullable Object input) - { - if (!sinks.remove(identifier, sink)) { - log.error("Sink for segment[%s] no longer valid, not abandoning.", identifier); - return null; - } - - metrics.setSinkCount(sinks.size()); - - if (removeOnDiskData) { - removeDirectory(computePersistDir(identifier)); - } - - log.info("Removed sink for segment[%s].", identifier); - - return null; - } - }, - // use persistExecutor to make sure that all the pending persists completes before - // starting to abandon segments - persistExecutor - ); + + if (!sinks.remove(identifier, sink)) { + log.error("Sink for segment[%s] no longer valid, not abandoning.", identifier); + } + + metrics.setSinkCount(sinks.size()); + + if (removeOnDiskData) { + removeDirectory(computePersistDir(identifier)); + } + + log.info("Removed sink for segment[%s].", identifier); + } private File computeLockFile() @@ -1232,8 +1038,7 @@ private File createPersistDirIfNeeded(SegmentIdWithShardSpec identifier) throws } /** - * Persists the given hydrant and returns the number of rows persisted. Must only be called in the single-threaded - * persistExecutor. + * Persists the given hydrant and returns the number of rows persisted. * * @param indexToPersist hydrant to persist * @param identifier the segment this hydrant is going to be part of @@ -1261,6 +1066,9 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id // 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) { + throw new ISE("Sink must not be null for identifier when persisting hydrant", identifier); + } final File persistDir = createPersistDirIfNeeded(identifier); indexMerger.persist( indexToPersist.getIndex(), @@ -1311,7 +1119,7 @@ private void removeDirectory(final File target) } } - private int calculateMemoryUsedByHydrants(FireHydrant hydrant) + private int calculateMemoryUsedByHydrant() { if (skipBytesInMemoryOverheadCheck) { return 0; @@ -1324,7 +1132,7 @@ private int calculateMemoryUsedByHydrants(FireHydrant hydrant) return total; } - private int calculateSinkMemoryInUsed(Sink sink) + private int calculateSinkMemoryInUsed() { if (skipBytesInMemoryOverheadCheck) { return 0; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index ce5ea255649b..3a2b716d49d6 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -166,7 +166,16 @@ public class StreamAppenderator implements Appenderator private volatile Throwable persistError; + + /** + * Flag to tell internals whether appenderator is working on behalf of a real time task. + * This is to manage certain aspects as needed. For example, for batch, non-real time tasks, + * physical segments (i.e. hydrants) do not need to memory map their persisted + * files. In this case, the code will avoid memory mapping them thus ameliorating the occurance + * of OOMs. + */ private final boolean isRealTime; + /** * Use next Map to store metadata (File, SegmentId) for a hydrant for batch appenderator * in order to facilitate the mapping of the QueryableIndex associated with a given hydrant @@ -1077,13 +1086,11 @@ public void closeNow() } } - @Override - public boolean isRealTime() + private boolean isRealTime() { return isRealTime; } - private void lockBasePersistDirectory() { if (basePersistDirLock == null) { diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java index fa1f7a9472f9..2ea3486d4a35 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java @@ -115,7 +115,7 @@ public void setUp() appenderatorTester.getAppenderator(), allocator, segmentHandoffNotifierFactory, - new TestUsedSegmentChecker(appenderatorTester), + new TestUsedSegmentChecker(appenderatorTester.getPushedSegments()), dataSegmentKiller, OBJECT_MAPPER, new FireDepartmentMetrics() diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java index 9db997be895c..2c7fd562a53f 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java @@ -26,22 +26,23 @@ import org.apache.druid.timeline.partition.PartitionChunk; import java.util.HashSet; +import java.util.List; import java.util.Set; public class TestUsedSegmentChecker implements UsedSegmentChecker { - private final AppenderatorTester appenderatorTester; + private final List pushedSegments; - public TestUsedSegmentChecker(AppenderatorTester appenderatorTester) + public TestUsedSegmentChecker(List pushedSegments) { - this.appenderatorTester = appenderatorTester; + this.pushedSegments = pushedSegments; } @Override public Set findUsedSegments(Set identifiers) { final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); - VersionedIntervalTimeline.addSegments(timeline, appenderatorTester.getPushedSegments().iterator()); + VersionedIntervalTimeline.addSegments(timeline, pushedSegments.iterator()); final Set retVal = new HashSet<>(); for (SegmentIdWithShardSpec identifier : identifiers) { From afb61412360a5873cf5061a56ccc2590afec0ab8 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 25 Jun 2021 01:00:44 -0700 Subject: [PATCH 32/47] Update batchMemoryMappedIndex name --- docs/configuration/index.md | 2 +- .../druid/indexing/common/config/TaskConfig.java | 10 +++++----- .../druid/indexing/common/task/BatchAppenderators.java | 2 +- .../indexing/appenderator/BatchAppenderatorTester.java | 4 ++-- .../indexing/common/task/TestAppenderatorsManager.java | 4 ++-- .../segment/realtime/appenderator/Appenderators.java | 4 ++-- .../realtime/appenderator/AppenderatorsManager.java | 2 +- .../DummyForInjectionAppenderatorsManager.java | 2 +- .../appenderator/PeonAppenderatorsManager.java | 4 ++-- .../UnifiedIndexerAppenderatorsManager.java | 4 ++-- 10 files changed, 19 insertions(+), 19 deletions(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 2c09b89303e6..a189fb7b2621 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1334,7 +1334,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.batchMemoryMappedIndex`|If false, native batch ingestion will not map indexes thus saving heap space. 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 batch ingestion code that avoids memory mapping indices. 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.batchFallback`|If false, native batch ingestion will use memory optimized code. 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.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.8.5| |`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|75000| |`druid.indexer.task.directoryLockTimeout`|Wait this long for zombie peons to exit before giving up on their replacements.|PT10M| diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java index 0285b33cd85a..e0b3ac8dcaa3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java @@ -77,7 +77,7 @@ public class TaskConfig private final boolean ignoreTimestampSpecForDruidInputSource; @JsonProperty - private final boolean batchMemoryMappedIndex; + private final boolean batchFallback; @JsonCreator public TaskConfig( @@ -91,7 +91,7 @@ public TaskConfig( @JsonProperty("directoryLockTimeout") Period directoryLockTimeout, @JsonProperty("shuffleDataLocations") List shuffleDataLocations, @JsonProperty("ignoreTimestampSpecForDruidInputSource") boolean ignoreTimestampSpecForDruidInputSource, - @JsonProperty("batchMemoryMappedIndex") boolean batchMemoryMapIndex // only set to true to fall back to older behavior + @JsonProperty("batchFallback") boolean batchFallback // only set to true to fall back to older behavior ) { this.baseDir = baseDir == null ? System.getProperty("java.io.tmpdir") : baseDir; @@ -117,7 +117,7 @@ public TaskConfig( this.shuffleDataLocations = shuffleDataLocations; } this.ignoreTimestampSpecForDruidInputSource = ignoreTimestampSpecForDruidInputSource; - this.batchMemoryMappedIndex = batchMemoryMapIndex; + this.batchFallback = batchFallback; } @JsonProperty @@ -201,9 +201,9 @@ public boolean isIgnoreTimestampSpecForDruidInputSource() } @JsonProperty - public boolean getBatchMemoryMappedIndex() + public boolean getbatchFallback() { - return batchMemoryMappedIndex; + return batchFallback; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java index bff138fb7172..3d45bf0926c3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java @@ -81,7 +81,7 @@ public static Appenderator newAppenderator( toolbox.getIndexMergerV9(), rowIngestionMeters, parseExceptionHandler, - toolbox.getConfig().getBatchMemoryMappedIndex() + toolbox.getConfig().getbatchFallback() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java index e42f10b52ae4..381dcc5e17f0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java @@ -124,7 +124,7 @@ public BatchAppenderatorTester( final boolean enablePushFailure, final RowIngestionMeters rowIngestionMeters, final boolean skipBytesInMemoryOverheadCheck, - final boolean batchMemoryMappedIndex + final boolean batchFallback ) { objectMapper = new DefaultObjectMapper(); @@ -249,7 +249,7 @@ public Map makeLoadSpec(URI uri) indexMerger, rowIngestionMeters, new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - batchMemoryMappedIndex + batchFallback ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java index 6e963e6f9136..b9786eb8f4d3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java @@ -106,7 +106,7 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean batchMemoryMappedIndex + boolean batchFallback ) { return Appenderators.createOffline( @@ -120,7 +120,7 @@ public Appenderator createOfflineAppenderatorForTask( indexMerger, rowIngestionMeters, parseExceptionHandler, - batchMemoryMappedIndex + batchFallback ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 56c9b301b477..4700028311d2 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -105,10 +105,10 @@ public static Appenderator createOffline( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean batchMemoryMappedIndex + boolean batchFallback ) { - if (batchMemoryMappedIndex) { + if (batchFallback) { // 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 diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java index c5b22cfb1b57..737631fccf65 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java @@ -99,7 +99,7 @@ Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean batchMemoryMappedIndex + boolean batchFallback ); /** diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java index f1e2f3c91347..12e676481444 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java @@ -91,7 +91,7 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean batchMemoryMappedIndex + boolean batchFallback ) { throw new UOE(ERROR_MSG); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java index 88a4f5720c18..cc8ca3e6a45b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java @@ -123,7 +123,7 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean batchMemoryMappedIndex + boolean batchFallback ) { // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators @@ -141,7 +141,7 @@ public Appenderator createOfflineAppenderatorForTask( indexMerger, rowIngestionMeters, parseExceptionHandler, - batchMemoryMappedIndex + batchFallback ); return batchAppenderator; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index d91d6d387b62..2d59ee1b7f99 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -210,7 +210,7 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean batchMemoryMappedIndex + boolean batchFallback ) { synchronized (this) { @@ -230,7 +230,7 @@ public Appenderator createOfflineAppenderatorForTask( wrapIndexMerger(indexMerger), rowIngestionMeters, parseExceptionHandler, - batchMemoryMappedIndex + batchFallback ); datasourceBundle.addAppenderator(taskId, appenderator); return appenderator; From 8077c014df450c4f2c3054da1f38a08b71820b09 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Wed, 30 Jun 2021 17:58:21 -0700 Subject: [PATCH 33/47] Code review comments --- .../appenderator/BatchAppenderatorTest.java | 8 ++--- .../appenderator/BatchAppenderatorTester.java | 20 +++++------ .../realtime/appenderator/Appenderators.java | 1 - .../appenderator/BatchAppenderator.java | 34 +++---------------- 4 files changed, 18 insertions(+), 45 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java index 2795f57bcf41..d7ac67285dc5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java @@ -154,7 +154,7 @@ public void testSimpleIngestionWithFallbackCodePath() throws Exception final Appenderator appenderator = tester.getAppenderator(); // startJob - Assert.assertEquals(null, appenderator.startJob()); + Assert.assertNull(appenderator.startJob()); // getDataSource Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource()); @@ -309,7 +309,7 @@ public void testMaxBytesInMemory() throws Exception //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; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = 1 * BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + int sinkSizeOverhead = BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet Assert.assertEquals( currentInMemoryIndexSize, @@ -324,7 +324,7 @@ public void testMaxBytesInMemory() throws Exception for (int i = 0; i < 53; i++) { appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null); } - sinkSizeOverhead = 1 * BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + 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. @@ -430,7 +430,7 @@ public void testTaskCleanupInMemoryCounterAfterCloseWithRowInMemory() throws Exc // Still under maxSizeInBytes after the add. Hence, we do not persist yet int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = 1 * BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + int sinkSizeOverhead = BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; Assert.assertEquals( currentInMemoryIndexSize + sinkSizeOverhead, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java index 381dcc5e17f0..66cc625f2494 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java @@ -46,6 +46,7 @@ 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.Appenderators; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.DataSegment; @@ -64,13 +65,11 @@ public class BatchAppenderatorTester implements AutoCloseable public static final String DATASOURCE = "foo"; private final DataSchema schema; - private final IndexTask.IndexTuningConfig tuningConfig; + private final AppenderatorConfig tuningConfig; 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 List pushedSegments = new CopyOnWriteArrayList<>(); @@ -182,7 +181,7 @@ public BatchAppenderatorTester( metrics = new FireDepartmentMetrics(); - indexIO = new IndexIO( + IndexIO indexIO = new IndexIO( objectMapper, new ColumnConfig() { @@ -193,7 +192,11 @@ public int columnCacheSizeBytes() } } ); - indexMerger = new IndexMergerV9(objectMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); + IndexMerger indexMerger = new IndexMergerV9( + objectMapper, + indexIO, + OffHeapMemorySegmentWriteOutMediumFactory.instance() + ); emitter = new ServiceEmitter( "test", @@ -263,7 +266,7 @@ public DataSchema getSchema() return schema; } - public IndexTask.IndexTuningConfig getTuningConfig() + public AppenderatorConfig getTuningConfig() { return tuningConfig; } @@ -273,11 +276,6 @@ public FireDepartmentMetrics getMetrics() return metrics; } - public DataSegmentPusher getDataSegmentPusher() - { - return dataSegmentPusher; - } - public ObjectMapper getObjectMapper() { return objectMapper; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 4700028311d2..7d6df35c1003 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -136,7 +136,6 @@ public static Appenderator createOffline( metrics, dataSegmentPusher, objectMapper, - null, indexIO, indexMerger, rowIngestionMeters, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index c7d3aa22aacf..79a86a1bb195 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -89,7 +89,6 @@ public class BatchAppenderator implements Appenderator public static final int ROUGH_OVERHEAD_PER_HYDRANT = 1000; private static final EmittingLogger log = new EmittingLogger(BatchAppenderator.class); - private static final int WARN_DELAY = 1000; private static final String IDENTIFIER_FILE_NAME = "identifier.json"; private final String myId; @@ -180,15 +179,6 @@ public int getNumHydrants() private volatile FileLock basePersistDirLock = null; private volatile FileChannel basePersistDirLockChannel = null; - /** - * This constructor allows the caller to provide its own SinkQuerySegmentWalker. - *

- * The sinkTimeline is set to the sink timeline of the provided SinkQuerySegmentWalker. - * If the SinkQuerySegmentWalker is null, a new sink timeline is initialized. - *

- * It is used by UnifiedIndexerAppenderatorsManager which allows queries on data associated with multiple - * Appenderators. - */ BatchAppenderator( String id, DataSchema schema, @@ -196,18 +186,12 @@ public int getNumHydrants() FireDepartmentMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, - @Nullable SinkQuerySegmentWalker sinkQuerySegmentWalker, IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler ) { - Preconditions.checkArgument( - sinkQuerySegmentWalker == null, - "Batch appenderator does not use a versioned timeline" - ); - this.myId = id; this.schema = Preconditions.checkNotNull(schema, "schema"); this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); @@ -463,7 +447,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final } @Override - public void clear() throws InterruptedException + public void clear() { clear(true); } @@ -554,7 +538,6 @@ public ListenableFuture persistAll(@Nullable final Committer committer) if (indexesToPersist.isEmpty()) { log.info("No indexes will be peristed"); } - final Stopwatch runExecStopwatch = Stopwatch.createStarted(); final Stopwatch persistStopwatch = Stopwatch.createStarted(); AtomicLong totalPersistedRows = new AtomicLong(numPersistedRows); @@ -591,13 +574,6 @@ public ListenableFuture persistAll(@Nullable final Committer committer) persistStopwatch.stop(); } - final long startDelay = runExecStopwatch.elapsed(TimeUnit.MILLISECONDS); - metrics.incrementPersistBackPressureMillis(startDelay); - if (startDelay > WARN_DELAY) { - log.warn("Ingestion was throttled for [%,d] millis because persists were pending.", startDelay); - } - runExecStopwatch.stop(); - // NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes. rowsCurrentlyInMemory.addAndGet(-numPersistedRows); bytesCurrentlyInMemory.addAndGet(-bytesPersisted); @@ -634,7 +610,7 @@ public ListenableFuture push( identifiersAndSinks = getIdentifierAndSinkForPersistedFile(identifier); } catch (IOException e) { - throw new ISE(e, "Failed to retrieve sinks for identifier", identifier); + throw new ISE(e, "Failed to retrieve sinks for identifier[%s]", identifier); } final DataSegment dataSegment = mergeAndPush( identifiersAndSinks.lhs, @@ -693,7 +669,7 @@ private DataSegment mergeAndPush( log.warn("Sink metadata not found just before merge for identifier [%s]", identifier); } else if (numHydrants != sm.getNumHydrants()) { throw new ISE("Number of restored hydrants[%d] for identifier[%s] does not match expected value[%d]", - numHydrants, identifier, sinksMetadata.get(identifier).getNumHydrants()); + numHydrants, identifier, sm.getNumHydrants()); } try { @@ -929,7 +905,7 @@ private Pair getIdentifierAndSinkForPersistedFile( (dir, fileName) -> !(Ints.tryParse(fileName) == null) ); if (sinkFiles == null) { - throw new ISE("Problem reading persisted sinks in path", identifierPath); + throw new ISE("Problem reading persisted sinks in path[%s]", identifierPath); } Arrays.sort( @@ -1067,7 +1043,7 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id // to pull it from there.... SinkMetadata sm = sinksMetadata.get(identifier); if (sm == null) { - throw new ISE("Sink must not be null for identifier when persisting hydrant", identifier); + throw new ISE("Sink must not be null for identifier when persisting hydrant[%s]", identifier); } final File persistDir = createPersistDirIfNeeded(identifier); indexMerger.persist( From 2c71c7d4a80af94de091b646f88d11e6c9179642 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Thu, 1 Jul 2021 12:29:56 -0700 Subject: [PATCH 34/47] Exclude class from coverage, will include again when packaging gets fixed --- server/pom.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/pom.xml b/server/pom.xml index a9569a38aef3..53a902c85f33 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -470,6 +470,9 @@ org/apache/druid/metadata/BasicDataSourceExt.class + + + org/apache/druid/segment/realtime/appenderator/BatchAppenderator.class From ff5e2ed59ed8a36ff217e480190e0d44a45786df Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Thu, 1 Jul 2021 15:56:15 -0700 Subject: [PATCH 35/47] Moved test classes to server module --- .../appenderator/BatchAppenderatorTester.java | 306 ------- server/pom.xml | 3 - .../BatchAppenderatorDriverTest.java | 9 +- .../appenderator/BatchAppenderatorTest.java | 7 +- .../appenderator/BatchAppenderatorTester.java | 847 ++++++++++++++++++ 5 files changed, 849 insertions(+), 323 deletions(-) delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java rename {indexing-service/src/test/java/org/apache/druid/indexing => server/src/test/java/org/apache/druid/segment/realtime}/appenderator/BatchAppenderatorDriverTest.java (93%) rename {indexing-service/src/test/java/org/apache/druid/indexing => server/src/test/java/org/apache/druid/segment/realtime}/appenderator/BatchAppenderatorTest.java (98%) create mode 100644 server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java deleted file mode 100644 index 66cc625f2494..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTester.java +++ /dev/null @@ -1,306 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.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.indexing.common.task.IndexTask; -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.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.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.Appenderators; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.LinearShardSpec; - -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.concurrent.CopyOnWriteArrayList; - -public class BatchAppenderatorTester implements AutoCloseable -{ - public static final String DATASOURCE = "foo"; - - private final DataSchema schema; - private final AppenderatorConfig tuningConfig; - private final FireDepartmentMetrics metrics; - private final DataSegmentPusher dataSegmentPusher; - private final ObjectMapper objectMapper; - private final Appenderator appenderator; - private final ServiceEmitter emitter; - - private final List pushedSegments = new CopyOnWriteArrayList<>(); - - public BatchAppenderatorTester( - final int maxRowsInMemory - ) - { - this(maxRowsInMemory, -1, null, false); - } - - public BatchAppenderatorTester( - final int maxRowsInMemory, - final boolean enablePushFailure - ) - { - this(maxRowsInMemory, -1, null, enablePushFailure); - } - - public BatchAppenderatorTester( - final int maxRowsInMemory, - final long maxSizeInBytes, - final boolean enablePushFailure - ) - { - this(maxRowsInMemory, maxSizeInBytes, null, enablePushFailure); - } - - public BatchAppenderatorTester( - final int maxRowsInMemory, - final long maxSizeInBytes, - final File basePersistDirectory, - final boolean enablePushFailure - ) - { - this( - maxRowsInMemory, - maxSizeInBytes, - basePersistDirectory, - enablePushFailure, - new SimpleRowIngestionMeters(), - false, - false - ); - } - - public BatchAppenderatorTester( - final int maxRowsInMemory, - final long maxSizeInBytes, - @Nullable final File basePersistDirectory, - final boolean enablePushFailure, - final RowIngestionMeters rowIngestionMeters, - final boolean skipBytesInMemoryOverheadCheck, - final boolean batchFallback - ) - { - objectMapper = new DefaultObjectMapper(); - objectMapper.registerSubtypes(LinearShardSpec.class); - - final Map parserMap = objectMapper.convertValue( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec("ts", "auto", null), - new DimensionsSpec(null, null, null), - null, - null, - null - ) - ), - Map.class - ); - schema = new DataSchema( - DATASOURCE, - parserMap, - new AggregatorFactory[]{ - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("met", "met") - }, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null, - objectMapper - ); - tuningConfig = new IndexTask.IndexTuningConfig( - null, - 2, - null, - maxRowsInMemory, - maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, - skipBytesInMemoryOverheadCheck, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - true, - null, - null, - null, - null - ).withBasePersistDirectory(basePersistDirectory != null ? basePersistDirectory : createNewBasePersistDirectory()); - - metrics = new FireDepartmentMetrics(); - - IndexIO indexIO = new IndexIO( - objectMapper, - new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 0; - } - } - ); - IndexMerger indexMerger = new IndexMergerV9( - objectMapper, - indexIO, - OffHeapMemorySegmentWriteOutMediumFactory.instance() - ); - - emitter = new ServiceEmitter( - "test", - "test", - new NoopEmitter() - ); - emitter.start(); - EmittingLogger.registerEmitter(emitter); - dataSegmentPusher = new DataSegmentPusher() - { - private boolean mustFail = true; - - @Deprecated - @Override - public String getPathForHadoop(String dataSource) - { - return getPathForHadoop(); - } - - @Override - public String getPathForHadoop() - { - throw new UnsupportedOperationException(); - } - - @Override - public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException - { - if (enablePushFailure && mustFail) { - mustFail = false; - throw new IOException("Push failure test"); - } else if (enablePushFailure) { - mustFail = true; - } - pushedSegments.add(segment); - return segment; - } - - @Override - public Map makeLoadSpec(URI uri) - { - throw new UnsupportedOperationException(); - } - }; - appenderator = Appenderators.createOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - batchFallback - ); - } - - private long getDefaultMaxBytesInMemory() - { - return (Runtime.getRuntime().totalMemory()) / 3; - } - - public DataSchema getSchema() - { - return schema; - } - - public AppenderatorConfig getTuningConfig() - { - return tuningConfig; - } - - public FireDepartmentMetrics getMetrics() - { - return metrics; - } - - public ObjectMapper getObjectMapper() - { - return objectMapper; - } - - public Appenderator getAppenderator() - { - return appenderator; - } - - public List getPushedSegments() - { - return pushedSegments; - } - - @Override - public void close() throws Exception - { - appenderator.close(); - emitter.close(); - FileUtils.deleteDirectory(tuningConfig.getBasePersistDirectory()); - } - - private static File createNewBasePersistDirectory() - { - return FileUtils.createTempDir("druid-batch-persist"); - } -} diff --git a/server/pom.xml b/server/pom.xml index 53a902c85f33..a9569a38aef3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -470,9 +470,6 @@ org/apache/druid/metadata/BasicDataSourceExt.class - - - org/apache/druid/segment/realtime/appenderator/BatchAppenderator.class diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java similarity index 93% rename from indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorDriverTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java index 31c97b33420d..2cf78cc937f0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.appenderator; +package org.apache.druid.segment.realtime.appenderator; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -32,14 +32,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.loading.DataSegmentKiller; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver.SegmentsForSequence; -import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; -import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.segment.realtime.appenderator.SegmentWithState; import org.apache.druid.segment.realtime.appenderator.SegmentWithState.SegmentState; -import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; -import org.apache.druid.segment.realtime.appenderator.TestUsedSegmentChecker; -import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java similarity index 98% rename from indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java index d7ac67285dc5..9aad2c4ecaf8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/BatchAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.appenderator; +package org.apache.druid.segment.realtime.appenderator; import com.google.common.base.Function; import com.google.common.collect.ImmutableList; @@ -30,11 +30,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; -import org.apache.druid.segment.realtime.appenderator.Appenderator; -import org.apache.druid.segment.realtime.appenderator.AppenderatorTester; -import org.apache.druid.segment.realtime.appenderator.BatchAppenderator; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java new file mode 100644 index 000000000000..bc9c0cdaf946 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java @@ -0,0 +1,847 @@ +/* + * 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.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +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.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +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.IAE; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.core.NoopEmitter; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.incremental.AppendableIndexSpec; +import org.apache.druid.segment.incremental.ParseExceptionHandler; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +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 javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CopyOnWriteArrayList; + +public class BatchAppenderatorTester implements AutoCloseable +{ + public static final String DATASOURCE = "foo"; + + private final DataSchema schema; + private final AppenderatorConfig tuningConfig; + private final FireDepartmentMetrics metrics; + private final DataSegmentPusher dataSegmentPusher; + private final ObjectMapper objectMapper; + private final Appenderator appenderator; + private final ServiceEmitter emitter; + + private final List pushedSegments = new CopyOnWriteArrayList<>(); + + public BatchAppenderatorTester( + final int maxRowsInMemory + ) + { + this(maxRowsInMemory, -1, null, false); + } + + public BatchAppenderatorTester( + final int maxRowsInMemory, + final boolean enablePushFailure + ) + { + this(maxRowsInMemory, -1, null, enablePushFailure); + } + + public BatchAppenderatorTester( + final int maxRowsInMemory, + final long maxSizeInBytes, + final boolean enablePushFailure + ) + { + this(maxRowsInMemory, maxSizeInBytes, null, enablePushFailure); + } + + public BatchAppenderatorTester( + final int maxRowsInMemory, + final long maxSizeInBytes, + final File basePersistDirectory, + final boolean enablePushFailure + ) + { + this( + maxRowsInMemory, + maxSizeInBytes, + basePersistDirectory, + enablePushFailure, + new SimpleRowIngestionMeters(), + false, + false + ); + } + + public BatchAppenderatorTester( + final int maxRowsInMemory, + final long maxSizeInBytes, + @Nullable final File basePersistDirectory, + final boolean enablePushFailure, + final RowIngestionMeters rowIngestionMeters, + final boolean skipBytesInMemoryOverheadCheck, + final boolean batchFallback + ) + { + objectMapper = new DefaultObjectMapper(); + objectMapper.registerSubtypes(LinearShardSpec.class); + + final Map parserMap = objectMapper.convertValue( + new MapInputRowParser( + new JSONParseSpec( + new TimestampSpec("ts", "auto", null), + new DimensionsSpec(null, null, null), + null, + null, + null + ) + ), + Map.class + ); + schema = new DataSchema( + DATASOURCE, + parserMap, + new AggregatorFactory[]{ + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("met", "met") + }, + new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), + null, + objectMapper + ); + tuningConfig = new IndexTuningConfig( + null, + 2, + null, + maxRowsInMemory, + maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, + skipBytesInMemoryOverheadCheck, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + true, + null, + null, + null, + null + ).withBasePersistDirectory(basePersistDirectory != null ? basePersistDirectory : createNewBasePersistDirectory()); + + + metrics = new FireDepartmentMetrics(); + + IndexIO indexIO = new IndexIO( + objectMapper, + new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 0; + } + } + ); + IndexMerger indexMerger = new IndexMergerV9( + objectMapper, + indexIO, + OffHeapMemorySegmentWriteOutMediumFactory.instance() + ); + + emitter = new ServiceEmitter( + "test", + "test", + new NoopEmitter() + ); + emitter.start(); + EmittingLogger.registerEmitter(emitter); + dataSegmentPusher = new DataSegmentPusher() + { + private boolean mustFail = true; + + @Deprecated + @Override + public String getPathForHadoop(String dataSource) + { + return getPathForHadoop(); + } + + @Override + public String getPathForHadoop() + { + throw new UnsupportedOperationException(); + } + + @Override + public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException + { + if (enablePushFailure && mustFail) { + mustFail = false; + throw new IOException("Push failure test"); + } else if (enablePushFailure) { + mustFail = true; + } + pushedSegments.add(segment); + return segment; + } + + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException(); + } + }; + appenderator = Appenderators.createOffline( + schema.getDataSource(), + schema, + tuningConfig, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), + batchFallback + ); + } + + private long getDefaultMaxBytesInMemory() + { + return (Runtime.getRuntime().totalMemory()) / 3; + } + + public DataSchema getSchema() + { + return schema; + } + + public AppenderatorConfig getTuningConfig() + { + return tuningConfig; + } + + public FireDepartmentMetrics getMetrics() + { + return metrics; + } + + public ObjectMapper getObjectMapper() + { + return objectMapper; + } + + public Appenderator getAppenderator() + { + return appenderator; + } + + public List getPushedSegments() + { + return pushedSegments; + } + + @Override + public void close() throws Exception + { + appenderator.close(); + emitter.close(); + FileUtils.deleteDirectory(tuningConfig.getBasePersistDirectory()); + } + + private static File createNewBasePersistDirectory() + { + return FileUtils.createTempDir("druid-batch-persist"); + } + + + // copied from druid-indexing as is for testing since it is not accessible from server module, + // we could simplify since not all its functionality is being used + // but leaving as is, it could be useful later + private static class IndexTuningConfig implements AppenderatorConfig + { + private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); + private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; + private static final boolean DEFAULT_GUARANTEE_ROLLUP = false; + private static final boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = false; + private static final long DEFAULT_PUSH_TIMEOUT = 0; + + private final AppendableIndexSpec appendableIndexSpec; + private final int maxRowsInMemory; + private final long maxBytesInMemory; + private final boolean skipBytesInMemoryOverheadCheck; + private final int maxColumnsToMerge; + + // null if all partitionsSpec related params are null. see getDefaultPartitionsSpec() for details. + @Nullable + private final PartitionsSpec partitionsSpec; + private final IndexSpec indexSpec; + private final IndexSpec indexSpecForIntermediatePersists; + private final File basePersistDirectory; + private final int maxPendingPersists; + + private final boolean forceGuaranteedRollup; + private final boolean reportParseExceptions; + private final long pushTimeout; + private final boolean logParseExceptions; + private final int maxParseExceptions; + private final int maxSavedParseExceptions; + private final long awaitSegmentAvailabilityTimeoutMillis; + + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + + @Nullable + private static PartitionsSpec getPartitionsSpec( + boolean forceGuaranteedRollup, + @Nullable PartitionsSpec partitionsSpec, + @Nullable Integer maxRowsPerSegment, + @Nullable Long maxTotalRows, + @Nullable Integer numShards, + @Nullable List partitionDimensions + ) + { + if (partitionsSpec == null) { + if (forceGuaranteedRollup) { + if (maxRowsPerSegment != null + || numShards != null + || (partitionDimensions != null && !partitionDimensions.isEmpty())) { + return new HashedPartitionsSpec(maxRowsPerSegment, numShards, partitionDimensions); + } else { + return null; + } + } else { + if (maxRowsPerSegment != null || maxTotalRows != null) { + return new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); + } else { + return null; + } + } + } else { + if (forceGuaranteedRollup) { + if (!partitionsSpec.isForceGuaranteedRollupCompatibleType()) { + throw new IAE(partitionsSpec.getClass().getSimpleName() + " cannot be used for perfect rollup"); + } + } else { + if (!(partitionsSpec instanceof DynamicPartitionsSpec)) { + throw new IAE("DynamicPartitionsSpec must be used for best-effort rollup"); + } + } + return partitionsSpec; + } + } + + @JsonCreator + public IndexTuningConfig( + @JsonProperty("targetPartitionSize") @Deprecated @Nullable Integer targetPartitionSize, + @JsonProperty("maxRowsPerSegment") @Deprecated @Nullable Integer maxRowsPerSegment, + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, + @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, + @JsonProperty("skipBytesInMemoryOverheadCheck") @Nullable Boolean skipBytesInMemoryOverheadCheck, + @JsonProperty("maxTotalRows") @Deprecated @Nullable Long maxTotalRows, + @JsonProperty("rowFlushBoundary") @Deprecated @Nullable Integer rowFlushBoundary_forBackCompatibility, + @JsonProperty("numShards") @Deprecated @Nullable Integer numShards, + @JsonProperty("partitionDimensions") @Deprecated @Nullable List partitionDimensions, + @JsonProperty("partitionsSpec") @Nullable PartitionsSpec partitionsSpec, + @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, + @JsonProperty("indexSpecForIntermediatePersists") @Nullable IndexSpec indexSpecForIntermediatePersists, + @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, + @JsonProperty("forceGuaranteedRollup") @Nullable Boolean forceGuaranteedRollup, + @JsonProperty("reportParseExceptions") @Deprecated @Nullable Boolean reportParseExceptions, + @JsonProperty("publishTimeout") @Deprecated @Nullable Long publishTimeout, + @JsonProperty("pushTimeout") @Nullable Long pushTimeout, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable + SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, + @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge, + @JsonProperty("awaitSegmentAvailabilityTimeoutMillis") @Nullable Long awaitSegmentAvailabilityTimeoutMillis + ) + { + this( + appendableIndexSpec, + maxRowsInMemory != null ? maxRowsInMemory : rowFlushBoundary_forBackCompatibility, + maxBytesInMemory != null ? maxBytesInMemory : 0, + skipBytesInMemoryOverheadCheck != null + ? skipBytesInMemoryOverheadCheck + : DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK, + getPartitionsSpec( + forceGuaranteedRollup == null ? DEFAULT_GUARANTEE_ROLLUP : forceGuaranteedRollup, + partitionsSpec, + maxRowsPerSegment == null ? targetPartitionSize : maxRowsPerSegment, + maxTotalRows, + numShards, + partitionDimensions + ), + indexSpec, + indexSpecForIntermediatePersists, + maxPendingPersists, + forceGuaranteedRollup, + reportParseExceptions, + pushTimeout != null ? pushTimeout : publishTimeout, + null, + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions, + maxColumnsToMerge, + awaitSegmentAvailabilityTimeoutMillis + ); + + Preconditions.checkArgument( + targetPartitionSize == null || maxRowsPerSegment == null, + "Can't use targetPartitionSize and maxRowsPerSegment together" + ); + } + + private IndexTuningConfig() + { + this(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); + } + + private IndexTuningConfig( + @Nullable AppendableIndexSpec appendableIndexSpec, + @Nullable Integer maxRowsInMemory, + @Nullable Long maxBytesInMemory, + @Nullable Boolean skipBytesInMemoryOverheadCheck, + @Nullable PartitionsSpec partitionsSpec, + @Nullable IndexSpec indexSpec, + @Nullable IndexSpec indexSpecForIntermediatePersists, + @Nullable Integer maxPendingPersists, + @Nullable Boolean forceGuaranteedRollup, + @Nullable Boolean reportParseExceptions, + @Nullable Long pushTimeout, + @Nullable File basePersistDirectory, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @Nullable Boolean logParseExceptions, + @Nullable Integer maxParseExceptions, + @Nullable Integer maxSavedParseExceptions, + @Nullable Integer maxColumnsToMerge, + @Nullable Long awaitSegmentAvailabilityTimeoutMillis + ) + { + this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; + this.maxRowsInMemory = maxRowsInMemory == null ? TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; + // initializing this to 0, it will be lazily initialized to a value + // @see #getMaxBytesInMemoryOrDefault() + this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; + this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck == null + ? + DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK + : skipBytesInMemoryOverheadCheck; + this.maxColumnsToMerge = maxColumnsToMerge == null + ? IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE + : maxColumnsToMerge; + this.partitionsSpec = partitionsSpec; + this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; + this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists == null ? + this.indexSpec : indexSpecForIntermediatePersists; + this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists; + this.forceGuaranteedRollup = forceGuaranteedRollup == null ? DEFAULT_GUARANTEE_ROLLUP : forceGuaranteedRollup; + this.reportParseExceptions = reportParseExceptions == null + ? DEFAULT_REPORT_PARSE_EXCEPTIONS + : reportParseExceptions; + this.pushTimeout = pushTimeout == null ? DEFAULT_PUSH_TIMEOUT : pushTimeout; + this.basePersistDirectory = basePersistDirectory; + + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + + if (this.reportParseExceptions) { + this.maxParseExceptions = 0; + this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions); + } else { + this.maxParseExceptions = maxParseExceptions == null + ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS + : maxParseExceptions; + this.maxSavedParseExceptions = maxSavedParseExceptions == null + ? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS + : maxSavedParseExceptions; + } + this.logParseExceptions = logParseExceptions == null + ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS + : logParseExceptions; + if (awaitSegmentAvailabilityTimeoutMillis == null || awaitSegmentAvailabilityTimeoutMillis < 0) { + this.awaitSegmentAvailabilityTimeoutMillis = DEFAULT_AWAIT_SEGMENT_AVAILABILITY_TIMEOUT_MILLIS; + } else { + this.awaitSegmentAvailabilityTimeoutMillis = awaitSegmentAvailabilityTimeoutMillis; + } + } + + @Override + public IndexTuningConfig withBasePersistDirectory(File dir) + { + return new IndexTuningConfig( + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + partitionsSpec, + indexSpec, + indexSpecForIntermediatePersists, + maxPendingPersists, + forceGuaranteedRollup, + reportParseExceptions, + pushTimeout, + dir, + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions, + maxColumnsToMerge, + awaitSegmentAvailabilityTimeoutMillis + ); + } + + public IndexTuningConfig withPartitionsSpec(PartitionsSpec partitionsSpec) + { + return new IndexTuningConfig( + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + partitionsSpec, + indexSpec, + indexSpecForIntermediatePersists, + maxPendingPersists, + forceGuaranteedRollup, + reportParseExceptions, + pushTimeout, + basePersistDirectory, + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions, + maxColumnsToMerge, + awaitSegmentAvailabilityTimeoutMillis + ); + } + + @JsonProperty + @Override + public AppendableIndexSpec getAppendableIndexSpec() + { + return appendableIndexSpec; + } + + @JsonProperty + @Override + public int getMaxRowsInMemory() + { + return maxRowsInMemory; + } + + @JsonProperty + @Override + public long getMaxBytesInMemory() + { + return maxBytesInMemory; + } + + @JsonProperty + @Override + public boolean isSkipBytesInMemoryOverheadCheck() + { + return skipBytesInMemoryOverheadCheck; + } + + @JsonProperty + @Nullable + @Override + public PartitionsSpec getPartitionsSpec() + { + return partitionsSpec; + } + + public PartitionsSpec getGivenOrDefaultPartitionsSpec() + { + if (partitionsSpec != null) { + return partitionsSpec; + } + return forceGuaranteedRollup + ? new HashedPartitionsSpec(null, null, null) + : new DynamicPartitionsSpec(null, null); + } + + @JsonProperty + @Override + public IndexSpec getIndexSpec() + { + return indexSpec; + } + + @JsonProperty + @Override + public IndexSpec getIndexSpecForIntermediatePersists() + { + return indexSpecForIntermediatePersists; + } + + @JsonProperty + @Override + public int getMaxPendingPersists() + { + return maxPendingPersists; + } + + + @JsonProperty + public boolean isForceGuaranteedRollup() + { + return forceGuaranteedRollup; + } + + @JsonProperty + @Override + public boolean isReportParseExceptions() + { + return reportParseExceptions; + } + + @JsonProperty + public long getPushTimeout() + { + return pushTimeout; + } + + @Nullable + @Override + @JsonProperty + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + + @Override + @JsonProperty + public int getMaxColumnsToMerge() + { + return maxColumnsToMerge; + } + + @JsonProperty + public boolean isLogParseExceptions() + { + return logParseExceptions; + } + + @JsonProperty + public int getMaxParseExceptions() + { + return maxParseExceptions; + } + + @JsonProperty + public int getMaxSavedParseExceptions() + { + return maxSavedParseExceptions; + } + + /** + * Return the max number of rows per segment. This returns null if it's not specified in tuningConfig. + * Deprecated in favor of {@link #getGivenOrDefaultPartitionsSpec()}. + */ + @Nullable + @Override + @Deprecated + @JsonProperty + public Integer getMaxRowsPerSegment() + { + return partitionsSpec == null ? null : partitionsSpec.getMaxRowsPerSegment(); + } + + /** + * Return the max number of total rows in appenderator. This returns null if it's not specified in tuningConfig. + * Deprecated in favor of {@link #getGivenOrDefaultPartitionsSpec()}. + */ + @Override + @Nullable + @Deprecated + @JsonProperty + public Long getMaxTotalRows() + { + return partitionsSpec instanceof DynamicPartitionsSpec + ? ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() + : null; + } + + @Deprecated + @Nullable + @JsonProperty + public Integer getNumShards() + { + return partitionsSpec instanceof HashedPartitionsSpec + ? ((HashedPartitionsSpec) partitionsSpec).getNumShards() + : null; + } + + @Deprecated + @JsonProperty + public List getPartitionDimensions() + { + return partitionsSpec instanceof HashedPartitionsSpec + ? ((HashedPartitionsSpec) partitionsSpec).getPartitionDimensions() + : Collections.emptyList(); + } + + @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 + } + + @JsonProperty + public long getAwaitSegmentAvailabilityTimeoutMillis() + { + return awaitSegmentAvailabilityTimeoutMillis; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IndexTuningConfig that = (IndexTuningConfig) o; + return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) && + maxRowsInMemory == that.maxRowsInMemory && + maxBytesInMemory == that.maxBytesInMemory && + skipBytesInMemoryOverheadCheck == that.skipBytesInMemoryOverheadCheck && + maxColumnsToMerge == that.maxColumnsToMerge && + maxPendingPersists == that.maxPendingPersists && + forceGuaranteedRollup == that.forceGuaranteedRollup && + reportParseExceptions == that.reportParseExceptions && + pushTimeout == that.pushTimeout && + logParseExceptions == that.logParseExceptions && + maxParseExceptions == that.maxParseExceptions && + maxSavedParseExceptions == that.maxSavedParseExceptions && + 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) && + Objects.equals(awaitSegmentAvailabilityTimeoutMillis, that.awaitSegmentAvailabilityTimeoutMillis); + } + + @Override + public int hashCode() + { + return Objects.hash( + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + maxColumnsToMerge, + partitionsSpec, + indexSpec, + indexSpecForIntermediatePersists, + basePersistDirectory, + maxPendingPersists, + forceGuaranteedRollup, + reportParseExceptions, + pushTimeout, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions, + segmentWriteOutMediumFactory, + awaitSegmentAvailabilityTimeoutMillis + ); + } + + @Override + public String toString() + { + return "IndexTuningConfig{" + + "maxRowsInMemory=" + maxRowsInMemory + + ", maxBytesInMemory=" + maxBytesInMemory + + ", skipBytesInMemoryOverheadCheck=" + skipBytesInMemoryOverheadCheck + + ", maxColumnsToMerge=" + maxColumnsToMerge + + ", partitionsSpec=" + partitionsSpec + + ", indexSpec=" + indexSpec + + ", indexSpecForIntermediatePersists=" + indexSpecForIntermediatePersists + + ", basePersistDirectory=" + basePersistDirectory + + ", maxPendingPersists=" + maxPendingPersists + + ", forceGuaranteedRollup=" + forceGuaranteedRollup + + ", reportParseExceptions=" + reportParseExceptions + + ", pushTimeout=" + pushTimeout + + ", logParseExceptions=" + logParseExceptions + + ", maxParseExceptions=" + maxParseExceptions + + ", maxSavedParseExceptions=" + maxSavedParseExceptions + + ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + + ", awaitSegmentAvailabilityTimeoutMillis=" + awaitSegmentAvailabilityTimeoutMillis + + '}'; + } + } + +} From 34e1342a8582f60b0da867b1daba82f33b87b0c7 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Thu, 1 Jul 2021 17:52:34 -0700 Subject: [PATCH 36/47] More BatchAppenderator cleanup --- .../appenderator/BatchAppenderator.java | 74 +++++++++---------- .../appenderator/BatchAppenderatorTester.java | 1 - 2 files changed, 36 insertions(+), 39 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 79a86a1bb195..32b72c7542ef 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -37,7 +37,6 @@ 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.io.Closer; @@ -362,7 +361,7 @@ public AppenderatorAddResult add( throw new RuntimeException(errorMessage); } - persistAllAndClear(); + persistAllAndRemoveSinks(); } return new AppenderatorAddResult(identifier, sinksMetadata.get(identifier).numRowsInSegment, false); @@ -482,21 +481,23 @@ public ListenableFuture drop(final SegmentIdWithShardSpec identifier) return Futures.immediateFuture(null); } - private void persistAllAndClear() + @Override + public ListenableFuture persistAll(@Nullable final Committer committer) { - // make sure sinks are cleared before push is called - try { - persistAll(null).get(); - clear(false); - } - catch (Throwable t) { - throw new RE(t, "Error while persisting"); + if (committer != null) { + throw new ISE("committer must be null for BatchAppenderator"); } + persistAllAndRemoveSinks(); + return Futures.immediateFuture(null); } - @Override - public ListenableFuture persistAll(@Nullable final Committer committer) + /** + * Persist all sinks & their hydrants, keep their metadata, and then remove them completely from + * memory (to be resurrected right before merge & push) + */ + private void persistAllAndRemoveSinks() { + final List> indexesToPersist = new ArrayList<>(); int numPersistedRows = 0; long bytesPersisted = 0L; @@ -511,36 +512,31 @@ public ListenableFuture persistAll(@Nullable final Committer committer) } final List hydrants = Lists.newArrayList(sink); - totalHydrantsCount.add(hydrants.size()); + // 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 onw hydrant for identifier[%s] but there are[%s]", + identifier, totalHydrantsForSink + ); + } + totalHydrantsCount.add(totalHydrantsCount); numPersistedRows += sink.getNumRowsInMemory(); bytesPersisted += sink.getBytesInMemory(); - final int limit = sink.isWritable() ? hydrants.size() - 1 : hydrants.size(); - - // gather hydrants that have not been persisted: - for (FireHydrant hydrant : hydrants.subList(0, limit)) { - if (!hydrant.hasSwapped()) { - log.debug("Hydrant[%s] hasn't persisted yet, persisting. Segment[%s]", hydrant, identifier); - indexesToPersist.add(Pair.of(hydrant, identifier)); - totalHydrantsPersistedAcrossSinks.add(1); - } - } - - if (sink.swappable()) { - // It is swappable. Get the old one to persist it and create a new one: - indexesToPersist.add(Pair.of(sink.swap(), identifier)); - totalHydrantsPersistedAcrossSinks.add(1); + if (!sink.swappable()) { + throw new ISE("Sink is not swappable![%s]", identifier); } + indexesToPersist.add(Pair.of(sink.swap(), identifier)); + totalHydrantsPersistedAcrossSinks.add(1); } - log.debug("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); if (indexesToPersist.isEmpty()) { log.info("No indexes will be peristed"); } final Stopwatch persistStopwatch = Stopwatch.createStarted(); AtomicLong totalPersistedRows = new AtomicLong(numPersistedRows); - try { for (Pair pair : indexesToPersist) { metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs)); @@ -578,9 +574,12 @@ public ListenableFuture persistAll(@Nullable final Committer committer) rowsCurrentlyInMemory.addAndGet(-numPersistedRows); bytesCurrentlyInMemory.addAndGet(-bytesPersisted); - log.info("Persisted rows[%,d] and bytes[%,d]", numPersistedRows, 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); - return Futures.immediateFuture(null); } @Override @@ -595,8 +594,8 @@ public ListenableFuture push( throw new ISE("There should be no committer for batch ingestion"); } - // Any sinks not persisted so far will be persisted before push: - persistAllAndClear(); + // Any sinks not persisted so far need to be persisted before push: + persistAllAndRemoveSinks(); log.info("Preparing to push..."); final List dataSegments = new ArrayList<>(); @@ -1023,13 +1022,12 @@ private File createPersistDirIfNeeded(SegmentIdWithShardSpec identifier) throws private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec identifier) { synchronized (indexToPersist) { + if (indexToPersist.hasSwapped()) { - log.info( - "Segment[%s] hydrant[%s] already swapped. Ignoring request to persist.", + throw new ISE("Segment[%s] hydrant[%s] already swapped. This cannot happen.", identifier, indexToPersist ); - return 0; } log.debug("Segment[%s], persisting Hydrant[%s]", identifier, indexToPersist); @@ -1064,7 +1062,7 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id indexToPersist.swapSegment(null); // remember hydrant count: - sinksMetadata.get(identifier).addHydrants(1); + sm.addHydrants(1); return numRows; } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java index bc9c0cdaf946..5a21214e3300 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java @@ -189,7 +189,6 @@ public BatchAppenderatorTester( null ).withBasePersistDirectory(basePersistDirectory != null ? basePersistDirectory : createNewBasePersistDirectory()); - metrics = new FireDepartmentMetrics(); IndexIO indexIO = new IndexIO( From dfcdf8f2c8f0cd0b75d245741901fc0672bbb892 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 2 Jul 2021 10:12:30 -0700 Subject: [PATCH 37/47] Fix bug in wrong counting of totalHydrants plus minor cleanup in add --- .../realtime/appenderator/BatchAppenderator.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 32b72c7542ef..d50dc3e944a7 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -501,8 +501,7 @@ private void persistAllAndRemoveSinks() final List> indexesToPersist = new ArrayList<>(); int numPersistedRows = 0; long bytesPersisted = 0L; - MutableInt totalHydrantsCount = new MutableInt(); - MutableInt totalHydrantsPersistedAcrossSinks = new MutableInt(); + int totalHydrantsCount = 0; final long totalSinks = sinks.size(); for (Map.Entry entry : sinks.entrySet()) { final SegmentIdWithShardSpec identifier = entry.getKey(); @@ -520,7 +519,7 @@ private void persistAllAndRemoveSinks() identifier, totalHydrantsForSink ); } - totalHydrantsCount.add(totalHydrantsCount); + totalHydrantsCount += 1; numPersistedRows += sink.getNumRowsInMemory(); bytesPersisted += sink.getBytesInMemory(); @@ -528,7 +527,6 @@ private void persistAllAndRemoveSinks() throw new ISE("Sink is not swappable![%s]", identifier); } indexesToPersist.add(Pair.of(sink.swap(), identifier)); - totalHydrantsPersistedAcrossSinks.add(1); } @@ -536,7 +534,6 @@ private void persistAllAndRemoveSinks() log.info("No indexes will be peristed"); } final Stopwatch persistStopwatch = Stopwatch.createStarted(); - AtomicLong totalPersistedRows = new AtomicLong(numPersistedRows); try { for (Pair pair : indexesToPersist) { metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs)); @@ -551,12 +548,11 @@ private void persistAllAndRemoveSinks() .collect(Collectors.joining(", ")) ); log.info( - "Persisted stats: processed rows: [%d], persisted rows[%d], sinks: [%d], total fireHydrants (across sinks): [%d], persisted fireHydrants (across sinks): [%d]", + "Persisted stats: processed rows: [%d], persisted rows[%d], persisted sinks: [%d], persisted fireHydrants (across sinks): [%d]", rowIngestionMeters.getProcessed(), - totalPersistedRows.get(), + numPersistedRows, totalSinks, - totalHydrantsCount.longValue(), - totalHydrantsPersistedAcrossSinks.longValue() + totalHydrantsCount ); } From 3e7fcff887e1130968ba15216062ba18cfe763d7 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 2 Jul 2021 10:16:24 -0700 Subject: [PATCH 38/47] Removed left over comments --- .../druid/segment/realtime/appenderator/BatchAppenderator.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index d50dc3e944a7..9958e8626f63 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -747,8 +747,6 @@ private DataSegment mergeAndPush( // Drop the queryable indexes behind the hydrants... they are not needed anymore and their // mapped file references // can generate OOMs during merge if enough of them are held back... - // agfixme: Since we cannot keep sinks due to memory growth then we have to add the sink metadata table and keep it up to date - //sinks.put(identifier,sink); for (FireHydrant fireHydrant : sink) { fireHydrant.swapSegment(null); } From d082d01a141dd230db88b87523ee795877495ce3 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 2 Jul 2021 14:37:42 -0700 Subject: [PATCH 39/47] Have BatchAppenderator follow the Appenderator contract for push & getSegments --- .../appenderator/BatchAppenderator.java | 73 +++++++++++++++---- .../appenderator/AppenderatorPlumberTest.java | 16 ++-- .../appenderator/BatchAppenderatorTest.java | 70 ++++++++++++++++-- .../appenderator/BatchAppenderatorTester.java | 13 ++++ .../StreamAppenderatorDriverTest.java | 8 +- .../appenderator/StreamAppenderatorTest.java | 54 +++++++------- ...ter.java => StreamAppenderatorTester.java} | 14 ++-- 7 files changed, 179 insertions(+), 69 deletions(-) rename server/src/test/java/org/apache/druid/segment/realtime/appenderator/{AppenderatorTester.java => StreamAppenderatorTester.java} (97%) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 9958e8626f63..cbae10702360 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -30,7 +30,6 @@ import com.google.common.primitives.Ints; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.commons.lang.mutable.MutableInt; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.FileUtils; @@ -132,6 +131,8 @@ private static class SinkMetadata * push time and also to remember the fire hydrant "count" when persisting it. */ private int numHydrants; + /* Reference to directory that holds the persisted data */ + File persistedFileDir; public SinkMetadata() { @@ -164,6 +165,16 @@ public int getNumHydrants() return numHydrants; } + public void setPersistedFileDir(File persistedFileDir) + { + this.persistedFileDir = persistedFileDir; + } + + public File getPersistedFileDir() + { + return persistedFileDir; + } + } // This variable updated in add(), persist(), and drop() @@ -368,7 +379,16 @@ public AppenderatorAddResult add( } @Override + /** + * Returns all active segments regardless whether they are in memory or persisted + */ public List getSegments() + { + return ImmutableList.copyOf(sinksMetadata.keySet()); + } + + @VisibleForTesting + public List getInMemorySegments() { return ImmutableList.copyOf(sinks.keySet()); } @@ -590,16 +610,38 @@ public ListenableFuture push( throw new ISE("There should be no committer for batch ingestion"); } + if (useUniquePath) { + 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..."); - final List dataSegments = new ArrayList<>(); - List persistedIdentifiers = getPersistedidentifierPaths(); - if (persistedIdentifiers == null) { + + // get the dirs for the identfiers: + List identifiersDirs = 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[%s]", identifier); + } + identifiersDirs.add(persistedDir); + } + if (identifiersDirs == null) { throw new ISE("Identifiers were persisted but could not be retrieved"); } - for (File identifier : persistedIdentifiers) { + + // push all sinks for identifiers: + final List dataSegments = new ArrayList<>(); + for (File identifier : identifiersDirs) { + + // retrieve sink from disk: Pair identifiersAndSinks; try { identifiersAndSinks = getIdentifierAndSinkForPersistedFile(identifier); @@ -607,16 +649,21 @@ public ListenableFuture push( catch (IOException e) { throw new ISE(e, "Failed to retrieve sinks for identifier[%s]", identifier); } + + // push it: final DataSegment dataSegment = mergeAndPush( identifiersAndSinks.lhs, identifiersAndSinks.rhs, - useUniquePath + false ); + + // record it: if (dataSegment != null) { dataSegments.add(dataSegment); } else { log.warn("mergeAndPush[%s] returned null, skipping.", identifiersAndSinks.lhs); } + } log.info("Push complete..."); @@ -639,6 +686,7 @@ private DataSegment mergeAndPush( ) { + // Use a descriptor file to indicate that pushing has completed. final File persistDir = computePersistDir(identifier); final File mergedTarget = new File(persistDir, "merged"); @@ -670,14 +718,8 @@ private DataSegment mergeAndPush( try { if (descriptorFile.exists()) { // Already pushed. - if (useUniquePath) { - // Don't reuse the descriptor, because the caller asked for a unique path. Leave the old one as-is, since - // it might serve some unknown purpose. - log.debug( - "Segment[%s] already pushed, but we want a unique path, so will push again with a new path.", - identifier - ); + throw new ISE("Merge and push for batch appenderator does not use unique path"); } else { log.info("Segment[%s] already pushed, skipping.", identifier); return objectMapper.readValue(descriptorFile, DataSegment.class); @@ -728,9 +770,7 @@ private DataSegment mergeAndPush( // Retry pushing segments because uploading to deep storage might fail especially for cloud storage types final DataSegment segment = RetryUtils.retry( - // The appenderator is currently being used for the local indexing task and the Kafka indexing task. For the - // Kafka indexing task, pushers must use unique file paths in deep storage in order to maintain exactly-once - // semantics. + // This appenderator is used only for the local indexing task so unique paths are not required () -> dataSegmentPusher.push( mergedFile, sink.getSegment() @@ -1045,6 +1085,7 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id tuningConfig.getIndexSpecForIntermediatePersists(), tuningConfig.getSegmentWriteOutMediumFactory() ); + sm.setPersistedFileDir(persistDir); log.info( "Persisted in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).", diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java index d5d138bbff8c..f2d793e05153 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java @@ -34,11 +34,11 @@ public class AppenderatorPlumberTest { private final AppenderatorPlumber plumber; - private final AppenderatorTester appenderatorTester; + private final StreamAppenderatorTester streamAppenderatorTester; public AppenderatorPlumberTest() throws Exception { - this.appenderatorTester = new AppenderatorTester(10); + this.streamAppenderatorTester = new StreamAppenderatorTester(10); DataSegmentAnnouncer segmentAnnouncer = EasyMock .createMock(DataSegmentAnnouncer.class); segmentAnnouncer.announceSegment(EasyMock.anyObject()); @@ -84,23 +84,23 @@ public AppenderatorPlumberTest() throws Exception null ); - this.plumber = new AppenderatorPlumber(appenderatorTester.getSchema(), - tuningConfig, appenderatorTester.getMetrics(), - segmentAnnouncer, segmentPublisher, handoffNotifier, - appenderatorTester.getAppenderator()); + this.plumber = new AppenderatorPlumber(streamAppenderatorTester.getSchema(), + tuningConfig, streamAppenderatorTester.getMetrics(), + segmentAnnouncer, segmentPublisher, handoffNotifier, + streamAppenderatorTester.getAppenderator()); } @Test public void testSimpleIngestion() throws Exception { - Appenderator appenderator = appenderatorTester.getAppenderator(); + Appenderator appenderator = streamAppenderatorTester.getAppenderator(); // startJob Assert.assertEquals(null, plumber.startJob()); // getDataSource - Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(StreamAppenderatorTester.DATASOURCE, appenderator.getDataSource()); InputRow[] rows = new InputRow[] { StreamAppenderatorTest.ir("2000", "foo", 1), diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java index 9aad2c4ecaf8..3b325d10997b 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java @@ -59,7 +59,7 @@ public void testSimpleIngestion() throws Exception Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // add #1 Assert.assertEquals( @@ -93,7 +93,7 @@ public void testSimpleIngestion() throws Exception // above should be cleared now Assert.assertEquals( Collections.emptyList(), - appenderator.getSegments().stream().sorted().collect(Collectors.toList()) + ((BatchAppenderator) appenderator).getInMemorySegments().stream().sorted().collect(Collectors.toList()) ); // add #4, this will add one more temporary segment: @@ -152,7 +152,7 @@ public void testSimpleIngestionWithFallbackCodePath() throws Exception Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(BatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // add #1 Assert.assertEquals( @@ -802,7 +802,7 @@ public void testCleanupFromDiskAfterClose() throws Exception @Test(timeout = 60_000L) public void testTotalRowCount() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(3, true)) { + try (final BatchAppenderatorTester tester = new BatchAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, appenderator.getTotalRowCount()); @@ -843,8 +843,8 @@ public void testTotalRowCount() throws Exception public void testVerifyRowIngestionMetrics() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final AppenderatorTester tester = - new AppenderatorTester(5, + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(5, 10000L, null, false, rowIngestionMeters )) { @@ -861,11 +861,67 @@ public void testVerifyRowIngestionMetrics() throws Exception Assert.assertEquals(0, rowIngestionMeters.getThrownAway()); } } + + @Test + public void testPushContract() throws Exception + { + final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); + try (final BatchAppenderatorTester tester = + new BatchAppenderatorTester(1, + 50000L, + null, false, rowIngestionMeters + )) { + final Appenderator appenderator = tester.getAppenderator(); + appenderator.startJob(); + + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar", 1), null); + appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar2", 1), null); + appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar3", 1), null); + + // push only a single segment + final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push( + Collections.singletonList(IDENTIFIERS.get(0)), + null, + false + ).get(); + + // only one segment must have been pushed: + Assert.assertEquals( + Collections.singletonList(IDENTIFIERS.get(0)), + Lists.transform( + segmentsAndCommitMetadata.getSegments(), + new Function() + { + @Override + public SegmentIdWithShardSpec apply(DataSegment input) + { + return SegmentIdWithShardSpec.fromDataSegment(input); + } + } + ).stream().sorted().collect(Collectors.toList()) + ); + + Assert.assertEquals( + tester.getPushedSegments().stream().sorted().collect(Collectors.toList()), + segmentsAndCommitMetadata.getSegments().stream().sorted().collect(Collectors.toList()) + ); + // the responsability for dropping is in the BatchAppenderatorDriver, drop manually: + appenderator.drop(IDENTIFIERS.get(0)); + + // and the segment that was not pushed should still be active + Assert.assertEquals( + Collections.singletonList(IDENTIFIERS.get(1)), + appenderator.getSegments() + ); + + + } + } private static SegmentIdWithShardSpec createSegmentId(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( - AppenderatorTester.DATASOURCE, + BatchAppenderatorTester.DATASOURCE, Intervals.of(interval), version, new LinearShardSpec(partitionNum) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java index 5a21214e3300..8fe3ccab68a0 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java @@ -126,6 +126,19 @@ public BatchAppenderatorTester( ); } + public BatchAppenderatorTester( + final int maxRowsInMemory, + final long maxSizeInBytes, + @Nullable final File basePersistDirectory, + final boolean enablePushFailure, + final RowIngestionMeters rowIngestionMeters + ) + { + this(maxRowsInMemory, maxSizeInBytes, basePersistDirectory, enablePushFailure, rowIngestionMeters, + false, false + ); + } + public BatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java index 2ea3486d4a35..b6154a6afe51 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java @@ -95,7 +95,7 @@ public class StreamAppenderatorDriverTest extends EasyMockSupport ); private SegmentAllocator allocator; - private AppenderatorTester appenderatorTester; + private StreamAppenderatorTester streamAppenderatorTester; private TestSegmentHandoffNotifierFactory segmentHandoffNotifierFactory; private StreamAppenderatorDriver driver; private DataSegmentKiller dataSegmentKiller; @@ -107,15 +107,15 @@ public class StreamAppenderatorDriverTest extends EasyMockSupport @Before public void setUp() { - appenderatorTester = new AppenderatorTester(MAX_ROWS_IN_MEMORY); + streamAppenderatorTester = new StreamAppenderatorTester(MAX_ROWS_IN_MEMORY); allocator = new TestSegmentAllocator(DATA_SOURCE, Granularities.HOUR); segmentHandoffNotifierFactory = new TestSegmentHandoffNotifierFactory(); dataSegmentKiller = createStrictMock(DataSegmentKiller.class); driver = new StreamAppenderatorDriver( - appenderatorTester.getAppenderator(), + streamAppenderatorTester.getAppenderator(), allocator, segmentHandoffNotifierFactory, - new TestUsedSegmentChecker(appenderatorTester.getPushedSegments()), + new TestUsedSegmentChecker(streamAppenderatorTester.getPushedSegments()), dataSegmentKiller, OBJECT_MAPPER, new FireDepartmentMetrics() diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java index 3dd0a69912a9..bc123b189684 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java @@ -72,7 +72,7 @@ public class StreamAppenderatorTest extends InitializedNullHandlingTest @Test public void testSimpleIngestion() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(2, true)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(2, true)) { final Appenderator appenderator = tester.getAppenderator(); boolean thrown; @@ -83,7 +83,7 @@ public void testSimpleIngestion() throws Exception Assert.assertEquals(null, appenderator.startJob()); // getDataSource - Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(StreamAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // add commitMetadata.put("x", "1"); @@ -157,7 +157,7 @@ public SegmentIdWithShardSpec apply(DataSegment input) public void testMaxBytesInMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final AppenderatorTester tester = new AppenderatorTester( + final StreamAppenderatorTester tester = new StreamAppenderatorTester( 100, 1024, null, @@ -209,7 +209,7 @@ public void run() public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final AppenderatorTester tester = new AppenderatorTester( + final StreamAppenderatorTester tester = new StreamAppenderatorTester( 100, 1024, null, @@ -257,7 +257,7 @@ public void run() @Test public void testMaxBytesInMemory() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(100, 15000, true)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(100, 15000, true)) { final Appenderator appenderator = tester.getAppenderator(); final AtomicInteger eventCount = new AtomicInteger(0); final Supplier committerSupplier = () -> { @@ -363,7 +363,7 @@ public void run() @Test(expected = RuntimeException.class) public void testTaskFailAsPersistCannotFreeAnyMoreMemory() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(100, 5180, true)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(100, 5180, true)) { final Appenderator appenderator = tester.getAppenderator(); final AtomicInteger eventCount = new AtomicInteger(0); final Supplier committerSupplier = () -> { @@ -394,7 +394,7 @@ public void run() public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final AppenderatorTester tester = new AppenderatorTester( + final StreamAppenderatorTester tester = new StreamAppenderatorTester( 100, 10, null, @@ -443,7 +443,7 @@ public void run() @Test public void testTaskCleanupInMemoryCounterAfterCloseWithRowInMemory() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(100, 10000, true)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(100, 10000, true)) { final Appenderator appenderator = tester.getAppenderator(); final AtomicInteger eventCount = new AtomicInteger(0); final Supplier committerSupplier = () -> { @@ -488,7 +488,7 @@ public void run() @Test public void testMaxBytesInMemoryInMultipleSinks() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(100, 31100, true)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(100, 31100, true)) { final Appenderator appenderator = tester.getAppenderator(); final AtomicInteger eventCount = new AtomicInteger(0); final Supplier committerSupplier = () -> { @@ -628,7 +628,7 @@ public void run() @Test public void testIgnoreMaxBytesInMemory() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(100, -1, true)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(100, -1, true)) { final Appenderator appenderator = tester.getAppenderator(); final AtomicInteger eventCount = new AtomicInteger(0); final Supplier committerSupplier = () -> { @@ -676,7 +676,7 @@ public void run() @Test public void testMaxRowsInMemory() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(3, true)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); final AtomicInteger eventCount = new AtomicInteger(0); final Supplier committerSupplier = new Supplier() @@ -727,7 +727,7 @@ public void run() @Test public void testMaxRowsInMemoryDisallowIncrementalPersists() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(3, false)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); final AtomicInteger eventCount = new AtomicInteger(0); final Supplier committerSupplier = () -> { @@ -774,7 +774,7 @@ public void run() public void testRestoreFromDisk() throws Exception { final RealtimeTuningConfig tuningConfig; - try (final AppenderatorTester tester = new AppenderatorTester(2, true)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(2, true)) { final Appenderator appenderator = tester.getAppenderator(); tuningConfig = tester.getTuningConfig(); @@ -816,7 +816,7 @@ public void run() appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 5), committerSupplier); appenderator.close(); - try (final AppenderatorTester tester2 = new AppenderatorTester( + try (final StreamAppenderatorTester tester2 = new StreamAppenderatorTester( 2, -1, tuningConfig.getBasePersistDirectory(), @@ -833,7 +833,7 @@ public void run() @Test(timeout = 60_000L) public void testTotalRowCount() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(3, true)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); final ConcurrentMap commitMetadata = new ConcurrentHashMap<>(); final Supplier committerSupplier = committerSupplierFromConcurrentMap(commitMetadata); @@ -876,7 +876,7 @@ public void testTotalRowCount() throws Exception public void testVerifyRowIngestionMetrics() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final AppenderatorTester tester = new AppenderatorTester(5, 10000L, null, false, rowIngestionMeters)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(5, 10000L, null, false, rowIngestionMeters)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", "invalid_met"), Committers.nilSupplier()); @@ -892,7 +892,7 @@ public void testVerifyRowIngestionMetrics() throws Exception @Test public void testQueryByIntervals() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(2, true)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(2, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -906,7 +906,7 @@ public void testQueryByIntervals() throws Exception // Query1: 2000/2001 final TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() - .dataSource(AppenderatorTester.DATASOURCE) + .dataSource(StreamAppenderatorTester.DATASOURCE) .intervals(ImmutableList.of(Intervals.of("2000/2001"))) .aggregators( Arrays.asList( @@ -932,7 +932,7 @@ public void testQueryByIntervals() throws Exception // Query2: 2000/2002 final TimeseriesQuery query2 = Druids.newTimeseriesQueryBuilder() - .dataSource(AppenderatorTester.DATASOURCE) + .dataSource(StreamAppenderatorTester.DATASOURCE) .intervals(ImmutableList.of(Intervals.of("2000/2002"))) .aggregators( Arrays.asList( @@ -962,7 +962,7 @@ public void testQueryByIntervals() throws Exception // Query3: 2000/2001T01 final TimeseriesQuery query3 = Druids.newTimeseriesQueryBuilder() - .dataSource(AppenderatorTester.DATASOURCE) + .dataSource(StreamAppenderatorTester.DATASOURCE) .intervals(ImmutableList.of(Intervals.of("2000/2001T01"))) .aggregators( Arrays.asList( @@ -991,7 +991,7 @@ public void testQueryByIntervals() throws Exception // Query4: 2000/2001T01, 2001T03/2001T04 final TimeseriesQuery query4 = Druids.newTimeseriesQueryBuilder() - .dataSource(AppenderatorTester.DATASOURCE) + .dataSource(StreamAppenderatorTester.DATASOURCE) .intervals( ImmutableList.of( Intervals.of("2000/2001T01"), @@ -1028,7 +1028,7 @@ public void testQueryByIntervals() throws Exception @Test public void testQueryBySegments() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(2, true)) { + try (final StreamAppenderatorTester tester = new StreamAppenderatorTester(2, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -1042,7 +1042,7 @@ public void testQueryBySegments() throws Exception // Query1: segment #2 final TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() - .dataSource(AppenderatorTester.DATASOURCE) + .dataSource(StreamAppenderatorTester.DATASOURCE) .aggregators( Arrays.asList( new LongSumAggregatorFactory("count", "count"), @@ -1078,7 +1078,7 @@ public void testQueryBySegments() throws Exception // Query2: segment #2, partial final TimeseriesQuery query2 = Druids.newTimeseriesQueryBuilder() - .dataSource(AppenderatorTester.DATASOURCE) + .dataSource(StreamAppenderatorTester.DATASOURCE) .aggregators( Arrays.asList( new LongSumAggregatorFactory("count", "count"), @@ -1114,7 +1114,7 @@ public void testQueryBySegments() throws Exception // Query3: segment #2, two disjoint intervals final TimeseriesQuery query3 = Druids.newTimeseriesQueryBuilder() - .dataSource(AppenderatorTester.DATASOURCE) + .dataSource(StreamAppenderatorTester.DATASOURCE) .aggregators( Arrays.asList( new LongSumAggregatorFactory("count", "count"), @@ -1154,7 +1154,7 @@ public void testQueryBySegments() throws Exception ); final ScanQuery query4 = Druids.newScanQueryBuilder() - .dataSource(AppenderatorTester.DATASOURCE) + .dataSource(StreamAppenderatorTester.DATASOURCE) .intervals( new MultipleSpecificSegmentSpec( ImmutableList.of( @@ -1194,7 +1194,7 @@ public void testQueryBySegments() throws Exception private static SegmentIdWithShardSpec si(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( - AppenderatorTester.DATASOURCE, + StreamAppenderatorTester.DATASOURCE, Intervals.of(interval), version, new LinearShardSpec(partitionNum) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java similarity index 97% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java index 600662e1f98c..feae06877d05 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java @@ -76,7 +76,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; -public class AppenderatorTester implements AutoCloseable +public class StreamAppenderatorTester implements AutoCloseable { public static final String DATASOURCE = "foo"; @@ -93,14 +93,14 @@ public class AppenderatorTester implements AutoCloseable private final List pushedSegments = new CopyOnWriteArrayList<>(); - public AppenderatorTester( + public StreamAppenderatorTester( final int maxRowsInMemory ) { this(maxRowsInMemory, -1, null, false); } - public AppenderatorTester( + public StreamAppenderatorTester( final int maxRowsInMemory, final boolean enablePushFailure ) @@ -108,7 +108,7 @@ public AppenderatorTester( this(maxRowsInMemory, -1, null, enablePushFailure); } - public AppenderatorTester( + public StreamAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final boolean enablePushFailure @@ -117,7 +117,7 @@ public AppenderatorTester( this(maxRowsInMemory, maxSizeInBytes, null, enablePushFailure); } - public AppenderatorTester( + public StreamAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final File basePersistDirectory, @@ -127,7 +127,7 @@ public AppenderatorTester( this(maxRowsInMemory, maxSizeInBytes, basePersistDirectory, enablePushFailure, new SimpleRowIngestionMeters(), false); } - public AppenderatorTester( + public StreamAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final File basePersistDirectory, @@ -138,7 +138,7 @@ public AppenderatorTester( this(maxRowsInMemory, maxSizeInBytes, basePersistDirectory, enablePushFailure, rowIngestionMeters, false); } - public AppenderatorTester( + public StreamAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final File basePersistDirectory, From 6704a250352160999d659f3b28412391b98e1f6d Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Fri, 2 Jul 2021 17:00:33 -0700 Subject: [PATCH 40/47] Fix LGTM violations --- .../segment/realtime/appenderator/BatchAppenderator.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index cbae10702360..c1bc312975ad 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -629,13 +629,10 @@ public ListenableFuture push( } File persistedDir = sm.getPersistedFileDir(); if (persistedDir == null) { - throw new ISE("Sink for identifier[%s] not found in local file system[%s]", identifier); + throw new ISE("Sink for identifier[%s] not found in local file system", identifier); } identifiersDirs.add(persistedDir); } - if (identifiersDirs == null) { - throw new ISE("Identifiers were persisted but could not be retrieved"); - } // push all sinks for identifiers: final List dataSegments = new ArrayList<>(); From 3f274c1e92ee57fa91a62658ee4d50282feffeae Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Tue, 6 Jul 2021 11:54:35 -0700 Subject: [PATCH 41/47] Review comments --- .../appenderator/BatchAppenderator.java | 123 ++++++++---------- 1 file changed, 56 insertions(+), 67 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index c1bc312975ad..3d5fb1479272 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -535,7 +535,7 @@ private void persistAllAndRemoveSinks() // 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 onw hydrant for identifier[%s] but there are[%s]", + throw new ISE("There should be only one hydrant for identifier[%s] but there are[%s]", identifier, totalHydrantsForSink ); } @@ -551,7 +551,7 @@ private void persistAllAndRemoveSinks() } if (indexesToPersist.isEmpty()) { - log.info("No indexes will be peristed"); + log.info("No indexes will be persisted"); } final Stopwatch persistStopwatch = Stopwatch.createStarted(); try { @@ -650,8 +650,7 @@ public ListenableFuture push( // push it: final DataSegment dataSegment = mergeAndPush( identifiersAndSinks.lhs, - identifiersAndSinks.rhs, - false + identifiersAndSinks.rhs ); // record it: @@ -672,14 +671,12 @@ public ListenableFuture push( * * @param identifier sink identifier * @param sink sink to push - * @param useUniquePath true if the segment should be written to a path with a unique identifier * @return segment descriptor, or null if the sink is no longer valid */ @Nullable private DataSegment mergeAndPush( final SegmentIdWithShardSpec identifier, - final Sink sink, - final boolean useUniquePath + final Sink sink ) { @@ -696,10 +693,8 @@ private DataSegment mergeAndPush( int numHydrants = 0; for (FireHydrant hydrant : sink) { - synchronized (hydrant) { - if (!hydrant.hasSwapped()) { - throw new ISE("Expected sink to be fully persisted before mergeAndPush for segment[%s].", identifier); - } + if (!hydrant.hasSwapped()) { + throw new ISE("Expected sink to be fully persisted before mergeAndPush for segment[%s].", identifier); } numHydrants++; } @@ -715,12 +710,8 @@ private DataSegment mergeAndPush( try { if (descriptorFile.exists()) { // Already pushed. - if (useUniquePath) { - throw new ISE("Merge and push for batch appenderator does not use unique path"); - } else { - log.info("Segment[%s] already pushed, skipping.", identifier); - return objectMapper.readValue(descriptorFile, DataSegment.class); - } + log.info("Segment[%s] already pushed, skipping.", identifier); + return objectMapper.readValue(descriptorFile, DataSegment.class); } removeDirectory(mergedTarget); @@ -775,7 +766,7 @@ private DataSegment mergeAndPush( indexes, schema.getDimensionsSpec() )), - useUniquePath + false ), exception -> exception instanceof Exception, 5 @@ -1052,61 +1043,59 @@ private File createPersistDirIfNeeded(SegmentIdWithShardSpec identifier) throws */ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec identifier) { - synchronized (indexToPersist) { + if (indexToPersist.hasSwapped()) { + throw new ISE( + "Segment[%s] hydrant[%s] already swapped. This cannot happen.", + identifier, + indexToPersist + ); + } - if (indexToPersist.hasSwapped()) { - throw new ISE("Segment[%s] hydrant[%s] already swapped. This cannot happen.", - identifier, - indexToPersist - ); - } + log.debug("Segment[%s], persisting Hydrant[%s]", identifier, indexToPersist); - log.debug("Segment[%s], persisting Hydrant[%s]", identifier, indexToPersist); + try { + final long startTime = System.nanoTime(); + int numRows = indexToPersist.getIndex().size(); - try { - final long startTime = System.nanoTime(); - 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 - // to pull it from there.... - SinkMetadata sm = sinksMetadata.get(identifier); - if (sm == null) { - throw new ISE("Sink must not be null for identifier when persisting hydrant[%s]", identifier); - } - final File persistDir = createPersistDirIfNeeded(identifier); - indexMerger.persist( - indexToPersist.getIndex(), - identifier.getInterval(), - new File(persistDir, String.valueOf(sm.getNumHydrants())), - tuningConfig.getIndexSpecForIntermediatePersists(), - tuningConfig.getSegmentWriteOutMediumFactory() - ); - sm.setPersistedFileDir(persistDir); - - log.info( - "Persisted in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).", - indexToPersist.getSegmentId(), - indexToPersist.getCount(), - (System.nanoTime() - startTime) / 1000000, - numRows - ); + // 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 + // to pull it from there.... + SinkMetadata sm = sinksMetadata.get(identifier); + if (sm == null) { + throw new ISE("Sink must not be null for identifier when persisting hydrant[%s]", identifier); + } + final File persistDir = createPersistDirIfNeeded(identifier); + indexMerger.persist( + indexToPersist.getIndex(), + identifier.getInterval(), + new File(persistDir, String.valueOf(sm.getNumHydrants())), + tuningConfig.getIndexSpecForIntermediatePersists(), + tuningConfig.getSegmentWriteOutMediumFactory() + ); + sm.setPersistedFileDir(persistDir); - indexToPersist.swapSegment(null); - // remember hydrant count: - sm.addHydrants(1); + log.info( + "Persisted in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).", + indexToPersist.getSegmentId(), + indexToPersist.getCount(), + (System.nanoTime() - startTime) / 1000000, + numRows + ); - return numRows; - } - catch (IOException e) { - log.makeAlert("Incremental persist failed") - .addData("segment", identifier.toString()) - .addData("dataSource", schema.getDataSource()) - .addData("count", indexToPersist.getCount()) - .emit(); + indexToPersist.swapSegment(null); + // remember hydrant count: + sm.addHydrants(1); - throw new RuntimeException(e); - } + return numRows; + } + catch (IOException e) { + log.makeAlert("Incremental persist failed") + .addData("segment", identifier.toString()) + .addData("dataSource", schema.getDataSource()) + .addData("count", indexToPersist.getCount()) + .emit(); + + throw new RuntimeException(e); } } From cea716a5ccefd6b024675056e4fb106852467a46 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Tue, 6 Jul 2021 13:16:55 -0700 Subject: [PATCH 42/47] Add stats after push is done --- .../segment/realtime/appenderator/BatchAppenderator.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 3d5fb1479272..a17d0d0ed3e9 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -614,7 +614,6 @@ public ListenableFuture push( throw new ISE("Batch ingestion does not require uniquePath"); } - // Any sinks not persisted so far need to be persisted before push: persistAllAndRemoveSinks(); @@ -622,6 +621,7 @@ public ListenableFuture push( // get the dirs for the identfiers: List identifiersDirs = new ArrayList<>(); + int totalHydrantsMerged = 0; for (SegmentIdWithShardSpec identifier : identifiers) { SinkMetadata sm = sinksMetadata.get(identifier); if (sm == null) { @@ -632,6 +632,7 @@ public ListenableFuture push( throw new ISE("Sink for identifier[%s] not found in local file system", identifier); } identifiersDirs.add(persistedDir); + totalHydrantsMerged += sm.getNumHydrants(); } // push all sinks for identifiers: @@ -661,7 +662,9 @@ public ListenableFuture push( } } - log.info("Push complete..."); + + log.info("Push complete: total sinks merged[%d], total hydrants merged[%d]", + identifiers.size(), totalHydrantsMerged); return Futures.immediateFuture(new SegmentsAndCommitMetadata(dataSegments, null)); } From 1f267a8c1eeb8df2a74bb2aaed4c1d91cc7ca5ba Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Wed, 7 Jul 2021 17:25:12 -0700 Subject: [PATCH 43/47] Code review comments (cleanup, remove rest of synchronization constructs in batch appenderator, reneame feature flag, remove real time flag stuff from stream appenderator, etc.) --- docs/configuration/index.md | 2 +- .../indexing/common/config/TaskConfig.java | 10 +- .../common/task/BatchAppenderators.java | 2 +- .../common/task/TestAppenderatorsManager.java | 4 +- .../realtime/appenderator/Appenderators.java | 10 +- .../appenderator/AppenderatorsManager.java | 2 +- .../appenderator/BatchAppenderator.java | 223 +++++++++--------- ...DummyForInjectionAppenderatorsManager.java | 2 +- .../PeonAppenderatorsManager.java | 4 +- .../appenderator/StreamAppenderator.java | 102 +------- .../UnifiedIndexerAppenderatorsManager.java | 7 +- .../appenderator/BatchAppenderatorTest.java | 38 +-- .../appenderator/BatchAppenderatorTester.java | 223 ++++++------------ 13 files changed, 212 insertions(+), 417 deletions(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index e2e6d15c6ee4..f01079254dcb 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1334,7 +1334,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.batchFallback`|If false, native batch ingestion will use memory optimized code. 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.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.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.8.5| |`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|75000| |`druid.indexer.task.directoryLockTimeout`|Wait this long for zombie peons to exit before giving up on their replacements.|PT10M| diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java index e0b3ac8dcaa3..27e2c552a3b3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java @@ -77,7 +77,7 @@ public class TaskConfig private final boolean ignoreTimestampSpecForDruidInputSource; @JsonProperty - private final boolean batchFallback; + private final boolean useLegacyBatchProcessing; @JsonCreator public TaskConfig( @@ -91,7 +91,7 @@ public TaskConfig( @JsonProperty("directoryLockTimeout") Period directoryLockTimeout, @JsonProperty("shuffleDataLocations") List shuffleDataLocations, @JsonProperty("ignoreTimestampSpecForDruidInputSource") boolean ignoreTimestampSpecForDruidInputSource, - @JsonProperty("batchFallback") boolean batchFallback // only set to true to fall back to older behavior + @JsonProperty("useLegacyBatchProcessing") boolean useLegacyBatchProcessing // only set to true to fall back to older behavior ) { this.baseDir = baseDir == null ? System.getProperty("java.io.tmpdir") : baseDir; @@ -117,7 +117,7 @@ public TaskConfig( this.shuffleDataLocations = shuffleDataLocations; } this.ignoreTimestampSpecForDruidInputSource = ignoreTimestampSpecForDruidInputSource; - this.batchFallback = batchFallback; + this.useLegacyBatchProcessing = useLegacyBatchProcessing; } @JsonProperty @@ -201,9 +201,9 @@ public boolean isIgnoreTimestampSpecForDruidInputSource() } @JsonProperty - public boolean getbatchFallback() + public boolean getuseLegacyBatchProcessing() { - return batchFallback; + return useLegacyBatchProcessing; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java index 3d45bf0926c3..1a437063060c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java @@ -81,7 +81,7 @@ public static Appenderator newAppenderator( toolbox.getIndexMergerV9(), rowIngestionMeters, parseExceptionHandler, - toolbox.getConfig().getbatchFallback() + toolbox.getConfig().getuseLegacyBatchProcessing() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java index 5470888d3b44..225769e23beb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java @@ -105,7 +105,7 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean batchFallback + boolean useLegacyBatchProcessing ) { return Appenderators.createOffline( @@ -119,7 +119,7 @@ public Appenderator createOfflineAppenderatorForTask( indexMerger, rowIngestionMeters, parseExceptionHandler, - batchFallback + useLegacyBatchProcessing ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 14b9ab79d18a..743df6bb4ebe 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -88,8 +88,7 @@ public static Appenderator createRealtime( indexMerger, cache, rowIngestionMeters, - parseExceptionHandler, - true + parseExceptionHandler ); } @@ -104,10 +103,10 @@ public static Appenderator createOffline( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean batchFallback + boolean useLegacyBatchProcessing ) { - if (batchFallback) { + 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 @@ -124,8 +123,7 @@ public static Appenderator createOffline( indexMerger, null, rowIngestionMeters, - parseExceptionHandler, - false + parseExceptionHandler ); } return new BatchAppenderator( diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java index 3910093680f7..3e25bc5b097f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java @@ -98,7 +98,7 @@ Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean batchFallback + boolean useLegacyBatchProcessing ); /** diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index a17d0d0ed3e9..953438436105 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -70,14 +70,12 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +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.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; public class BatchAppenderator implements Appenderator @@ -103,7 +101,7 @@ public class BatchAppenderator implements Appenderator * called). It could also be accessed (but not mutated) potentially in the context * of any thread from {@link #drop}. */ - private final ConcurrentMap sinks = new ConcurrentHashMap<>(); + private final Map sinks = new HashMap<>(); private final long maxBytesTuningConfig; private final boolean skipBytesInMemoryOverheadCheck; @@ -111,76 +109,12 @@ public class BatchAppenderator implements Appenderator * 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. */ - private final ConcurrentHashMap sinksMetadata = new ConcurrentHashMap<>(); - - /** - * 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 - * fields is annotated as comments in the class - */ - private static class SinkMetadata - { - /** This is used to maintain the rows in the sink accross persists of the sink - * used for functionality (i.e. to detect whether an incremental push - * is needed {@link AppenderatorDriverAddResult#isPushRequired(Integer, Long)} - **/ - private int numRowsInSegment; - /** For sanity check as well as functionality: to make sure that all hydrants for a sink are restored from disk at - * push time and also to remember the fire hydrant "count" when persisting it. - */ - private int numHydrants; - /* Reference to directory that holds the persisted data */ - File persistedFileDir; - - public SinkMetadata() - { - this(0, 0); - } - - public SinkMetadata(int numRowsInSegment, int numHydrants) - { - this.numRowsInSegment = numRowsInSegment; - this.numHydrants = numHydrants; - } - - public void addRows(int num) - { - numRowsInSegment += num; - } - - public void addHydrants(int num) - { - numHydrants += num; - } - - public int getNumRowsInSegment() - { - return numRowsInSegment; - } - - public int getNumHydrants() - { - return numHydrants; - } - - public void setPersistedFileDir(File persistedFileDir) - { - this.persistedFileDir = persistedFileDir; - } - - public File getPersistedFileDir() - { - return persistedFileDir; - } - - } + private final Map sinksMetadata = new HashMap<>(); // This variable updated in add(), persist(), and drop() - private final AtomicInteger rowsCurrentlyInMemory = new AtomicInteger(); - private final AtomicInteger totalRows = new AtomicInteger(); - private final AtomicLong bytesCurrentlyInMemory = new AtomicLong(); + private int rowsCurrentlyInMemory = 0; + private int totalRows = 0; + private long bytesCurrentlyInMemory = 0; private final RowIngestionMeters rowIngestionMeters; private final ParseExceptionHandler parseExceptionHandler; @@ -296,9 +230,9 @@ public AppenderatorAddResult add( } final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd; - rowsCurrentlyInMemory.addAndGet(numAddedRows); - bytesCurrentlyInMemory.addAndGet(bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd); - totalRows.addAndGet(numAddedRows); + rowsCurrentlyInMemory += numAddedRows; + bytesCurrentlyInMemory += bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd; + totalRows += numAddedRows; sinksMetadata.computeIfAbsent(identifier, unused -> new SinkMetadata()).addRows(numAddedRows); boolean persist = false; @@ -308,19 +242,19 @@ public AppenderatorAddResult add( persist = true; persistReasons.add("No more rows can be appended to sink"); } - if (rowsCurrentlyInMemory.get() >= tuningConfig.getMaxRowsInMemory()) { + if (rowsCurrentlyInMemory >= tuningConfig.getMaxRowsInMemory()) { persist = true; persistReasons.add(StringUtils.format( "rowsCurrentlyInMemory[%d] is greater than maxRowsInMemory[%d]", - rowsCurrentlyInMemory.get(), + rowsCurrentlyInMemory, tuningConfig.getMaxRowsInMemory() )); } - if (bytesCurrentlyInMemory.get() >= maxBytesTuningConfig) { + if (bytesCurrentlyInMemory >= maxBytesTuningConfig) { persist = true; persistReasons.add(StringUtils.format( "bytesCurrentlyInMemory[%d] is greater than maxBytesInMemory[%d]", - bytesCurrentlyInMemory.get(), + bytesCurrentlyInMemory, maxBytesTuningConfig )); } @@ -336,13 +270,13 @@ public AppenderatorAddResult add( if (sinkEntry.swappable()) { // Code for batch no longer memory maps hydrants but they still take memory... int memoryStillInUse = calculateMemoryUsedByHydrant(); - bytesCurrentlyInMemory.addAndGet(memoryStillInUse); + bytesCurrentlyInMemory += memoryStillInUse; } } } if (!skipBytesInMemoryOverheadCheck - && bytesCurrentlyInMemory.get() - bytesToBePersisted > maxBytesTuningConfig) { + && bytesCurrentlyInMemory - bytesToBePersisted > maxBytesTuningConfig) { // We are still over maxBytesTuningConfig even after persisting. // This means that we ran out of all available memory to ingest (due to overheads created as part of ingestion) final String alertMessage = StringUtils.format( @@ -352,7 +286,7 @@ public AppenderatorAddResult add( sinks.size(), sinks.values().stream().mapToInt(Iterables::size).sum(), getTotalRowCount(), - bytesCurrentlyInMemory.get(), + bytesCurrentlyInMemory, bytesToBePersisted, maxBytesTuningConfig ); @@ -402,19 +336,19 @@ public int getRowCount(final SegmentIdWithShardSpec identifier) @Override public int getTotalRowCount() { - return totalRows.get(); + return totalRows; } @VisibleForTesting public int getRowsInMemory() { - return rowsCurrentlyInMemory.get(); + return rowsCurrentlyInMemory; } @VisibleForTesting public long getBytesCurrentlyInMemory() { - return bytesCurrentlyInMemory.get(); + return bytesCurrentlyInMemory; } @VisibleForTesting @@ -444,7 +378,7 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) maxBytesTuningConfig, null ); - bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed()); + bytesCurrentlyInMemory += calculateSinkMemoryInUsed(); sinks.put(identifier, retVal); metrics.setSinkCount(sinks.size()); @@ -474,11 +408,14 @@ public void clear() private void clear(boolean removeOnDiskData) { // Drop commit metadata, then abandon all segments. - log.info("Clearing all sinks & hydrants, removing data on disk: [%s]", removeOnDiskData); + log.info("Clearing all[%d] sinks & their hydrants, removing data on disk: [%s]", sinks.size(), removeOnDiskData); // Drop everything. - for (Map.Entry entry : sinks.entrySet()) { - removeSink(entry.getKey(), entry.getValue(), removeOnDiskData); - } + Iterator> sinksIterator = sinks.entrySet().iterator(); + sinksIterator.forEachRemaining(entry -> { + clearSinkMetadata(entry.getKey(), entry.getValue(), removeOnDiskData); + sinksIterator.remove(); + }); + metrics.setSinkCount(sinks.size()); } @Override @@ -493,10 +430,13 @@ public ListenableFuture drop(final SegmentIdWithShardSpec identifier) if (totalRowsAfter < 0) { log.warn("Total rows[%d] after dropping segment[%s] rows [%d]", totalRowsAfter, identifier, rowsToDrop); } - totalRows.set(Math.max(totalRowsAfter, 0)); + totalRows = Math.max(totalRowsAfter, 0); } if (sink != null) { - removeSink(identifier, sink, true); + clearSinkMetadata(identifier, sink, true); + if (sinks.remove(identifier) == null) { + log.warn("Sink for identifier[%s] not found, skipping", identifier); + } } return Futures.immediateFuture(null); } @@ -587,8 +527,8 @@ private void persistAllAndRemoveSinks() } // NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes. - rowsCurrentlyInMemory.addAndGet(-numPersistedRows); - bytesCurrentlyInMemory.addAndGet(-bytesPersisted); + rowsCurrentlyInMemory -= numPersistedRows; + bytesCurrentlyInMemory -= bytesPersisted; // remove all sinks after persisting: clear(false); @@ -819,9 +759,7 @@ public void close() log.debug("Shutting down..."); - for (Map.Entry entry : sinks.entrySet()) { - removeSink(entry.getKey(), entry.getValue(), false); - } + clear(false); unlockBasePersistDirectory(); @@ -833,7 +771,7 @@ public void close() } } - totalRows.set(0); + totalRows = 0; sinksMetadata.clear(); } @@ -969,7 +907,10 @@ private Pair getIdentifierAndSinkForPersistedFile( return new Pair<>(identifier, currSink); } - private void removeSink( + // 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( final SegmentIdWithShardSpec identifier, final Sink sink, final boolean removeOnDiskData @@ -979,26 +920,19 @@ private void removeSink( if (sink.finishWriting()) { // Decrement this sink's rows from the counters. we only count active sinks so that we don't double decrement, // i.e. those that haven't been persisted for *InMemory counters, or pushed to deep storage for the total counter. - rowsCurrentlyInMemory.addAndGet(-sink.getNumRowsInMemory()); - bytesCurrentlyInMemory.addAndGet(-sink.getBytesInMemory()); - bytesCurrentlyInMemory.addAndGet(-calculateSinkMemoryInUsed()); + rowsCurrentlyInMemory -= sink.getNumRowsInMemory(); + bytesCurrentlyInMemory -= sink.getBytesInMemory(); + bytesCurrentlyInMemory -= calculateSinkMemoryInUsed(); for (FireHydrant hydrant : sink) { // Decrement memory used by all Memory Mapped Hydrant if (!hydrant.equals(sink.getCurrHydrant())) { - bytesCurrentlyInMemory.addAndGet(-calculateMemoryUsedByHydrant()); + bytesCurrentlyInMemory -= calculateMemoryUsedByHydrant(); } } // 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. } - - if (!sinks.remove(identifier, sink)) { - log.error("Sink for segment[%s] no longer valid, not abandoning.", identifier); - } - - metrics.setSinkCount(sinks.size()); - if (removeOnDiskData) { removeDirectory(computePersistDir(identifier)); } @@ -1123,8 +1057,6 @@ private int calculateMemoryUsedByHydrant() return 0; } // These calculations are approximated from actual heap dumps. - // Memory footprint includes count integer in FireHydrant, shorts in ReferenceCountingSegment, - // Objects in SimpleQueryableIndex (such as SmooshedFileMapper, each ColumnHolder in column map, etc.) int total; total = Integer.BYTES + (4 * Short.BYTES) + ROUGH_OVERHEAD_PER_HYDRANT; return total; @@ -1138,4 +1070,69 @@ private int calculateSinkMemoryInUsed() // Rough estimate of memory footprint of empty Sink based on actual heap dumps return ROUGH_OVERHEAD_PER_SINK; } + + /** + * 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 + * fields is annotated as comments in the class + */ + private static class SinkMetadata + { + /** This is used to maintain the rows in the sink accross persists of the sink + * used for functionality (i.e. to detect whether an incremental push + * is needed {@link AppenderatorDriverAddResult#isPushRequired(Integer, Long)} + **/ + private int numRowsInSegment; + /** For sanity check as well as functionality: to make sure that all hydrants for a sink are restored from disk at + * push time and also to remember the fire hydrant "count" when persisting it. + */ + private int numHydrants; + /* Reference to directory that holds the persisted data */ + File persistedFileDir; + + public SinkMetadata() + { + this(0, 0); + } + + public SinkMetadata(int numRowsInSegment, int numHydrants) + { + this.numRowsInSegment = numRowsInSegment; + this.numHydrants = numHydrants; + } + + public void addRows(int num) + { + numRowsInSegment += num; + } + + public void addHydrants(int num) + { + numHydrants += num; + } + + public int getNumRowsInSegment() + { + return numRowsInSegment; + } + + public int getNumHydrants() + { + return numHydrants; + } + + public void setPersistedFileDir(File persistedFileDir) + { + this.persistedFileDir = persistedFileDir; + } + + public File getPersistedFileDir() + { + return persistedFileDir; + } + + } + } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java index 40ca12a49727..0dcd7d61a76a 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java @@ -90,7 +90,7 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean batchFallback + boolean useLegacyBatchProcessing ) { throw new UOE(ERROR_MSG); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java index 8bce7a98a732..1c44b301cc35 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java @@ -122,7 +122,7 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean batchFallback + boolean useLegacyBatchProcessing ) { // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators @@ -140,7 +140,7 @@ public Appenderator createOfflineAppenderatorForTask( indexMerger, rowIngestionMeters, parseExceptionHandler, - batchFallback + useLegacyBatchProcessing ); return batchAppenderator; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index 3a2b716d49d6..1ee36f5505d2 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -60,7 +60,6 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.Segment; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.ParseExceptionHandler; @@ -72,7 +71,6 @@ import org.apache.druid.segment.realtime.plumber.Sink; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.joda.time.Interval; @@ -86,9 +84,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; -import java.util.IdentityHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -166,28 +162,6 @@ public class StreamAppenderator implements Appenderator private volatile Throwable persistError; - - /** - * Flag to tell internals whether appenderator is working on behalf of a real time task. - * This is to manage certain aspects as needed. For example, for batch, non-real time tasks, - * physical segments (i.e. hydrants) do not need to memory map their persisted - * files. In this case, the code will avoid memory mapping them thus ameliorating the occurance - * of OOMs. - */ - private final boolean isRealTime; - - /** - * Use next Map to store metadata (File, SegmentId) for a hydrant for batch appenderator - * in order to facilitate the mapping of the QueryableIndex associated with a given hydrant - * at merge time. This is necessary since batch appenderator will not map the QueryableIndex - * at persist time in order to minimize its memory footprint. This has to be synchronized since the - * map may be accessed from multiple threads. - * Use {@link IdentityHashMap} to better reflect the fact that the key needs to be interpreted - * with reference semantics. - */ - private final Map> persistedHydrantMetadata = - Collections.synchronizedMap(new IdentityHashMap<>()); - /** * This constructor allows the caller to provide its own SinkQuerySegmentWalker. * @@ -210,8 +184,7 @@ public class StreamAppenderator implements Appenderator IndexMerger indexMerger, Cache cache, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler, - boolean isRealTime + ParseExceptionHandler parseExceptionHandler ) { this.myId = id; @@ -227,7 +200,6 @@ public class StreamAppenderator implements Appenderator this.texasRanger = sinkQuerySegmentWalker; this.rowIngestionMeters = Preconditions.checkNotNull(rowIngestionMeters, "rowIngestionMeters"); this.parseExceptionHandler = Preconditions.checkNotNull(parseExceptionHandler, "parseExceptionHandler"); - this.isRealTime = isRealTime; if (sinkQuerySegmentWalker == null) { this.sinkTimeline = new VersionedIntervalTimeline<>( @@ -564,9 +536,6 @@ public void clear() throws InterruptedException futures.add(abandonSegment(entry.getKey(), entry.getValue(), true)); } - // Re-initialize hydrant map: - persistedHydrantMetadata.clear(); - // Await dropping. Futures.allAsList(futures).get(); } @@ -876,34 +845,6 @@ private DataSegment mergeAndPush( Closer closer = Closer.create(); try { for (FireHydrant fireHydrant : sink) { - - // if batch, swap/persist did not memory map the incremental index, we need it mapped now: - if (!isRealTime()) { - - // sanity - Pair persistedMetadata = persistedHydrantMetadata.get(fireHydrant); - if (persistedMetadata == null) { - throw new ISE("Persisted metadata for batch hydrant [%s] is null!", fireHydrant); - } - - File persistedFile = persistedMetadata.lhs; - SegmentId persistedSegmentId = persistedMetadata.rhs; - - // sanity: - if (persistedFile == null) { - throw new ISE("Persisted file for batch hydrant [%s] is null!", fireHydrant); - } else if (persistedSegmentId == null) { - throw new ISE( - "Persisted segmentId for batch hydrant in file [%s] is null!", - persistedFile.getPath() - ); - } - fireHydrant.swapSegment(new QueryableIndexSegment( - indexIO.loadIndex(persistedFile), - persistedSegmentId - )); - } - Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex(); log.debug("Segment[%s] adding hydrant[%s]", identifier, fireHydrant); @@ -951,15 +892,6 @@ private DataSegment mergeAndPush( 5 ); - if (!isRealTime()) { - // Drop the queryable indexes behind the hydrants... they are not needed anymore and their - // mapped file references - // can generate OOMs during merge if enough of them are held back... - for (FireHydrant fireHydrant : sink) { - fireHydrant.swapSegment(null); - } - } - final long pushFinishTime = System.nanoTime(); objectMapper.writeValue(descriptorFile, segment); @@ -1086,11 +1018,6 @@ public void closeNow() } } - private boolean isRealTime() - { - return isRealTime; - } - private void lockBasePersistDirectory() { if (basePersistDirLock == null) { @@ -1408,8 +1335,6 @@ public Void apply(@Nullable Object input) cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant)); } hydrant.swapSegment(null); - // remove hydrant from persisted metadata: - persistedHydrantMetadata.remove(hydrant); } if (removeOnDiskData) { @@ -1524,15 +1449,10 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id numRows ); - // Map only when this appenderator is being driven by a real time task: - Segment segmentToSwap = null; - if (isRealTime()) { - segmentToSwap = new QueryableIndexSegment(indexIO.loadIndex(persistedFile), indexToPersist.getSegmentId()); - } else { - // remember file path & segment id to rebuild the queryable index for merge: - persistedHydrantMetadata.put(indexToPersist, new Pair<>(persistedFile, indexToPersist.getSegmentId())); - } - indexToPersist.swapSegment(segmentToSwap); + indexToPersist.swapSegment(new QueryableIndexSegment( + indexIO.loadIndex(persistedFile), + indexToPersist.getSegmentId() + )); return numRows; } @@ -1570,14 +1490,10 @@ private int calculateMMappedHydrantMemoryInUsed(FireHydrant hydrant) // These calculations are approximated from actual heap dumps. // Memory footprint includes count integer in FireHydrant, shorts in ReferenceCountingSegment, // Objects in SimpleQueryableIndex (such as SmooshedFileMapper, each ColumnHolder in column map, etc.) - int total; - total = Integer.BYTES + (4 * Short.BYTES) + ROUGH_OVERHEAD_PER_HYDRANT; - if (isRealTime()) { - // for real time add references to byte memory mapped references.. - total += (hydrant.getSegmentNumDimensionColumns() * ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER) + - (hydrant.getSegmentNumMetricColumns() * ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + - ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER; - } + int total = Integer.BYTES + (4 * Short.BYTES) + ROUGH_OVERHEAD_PER_HYDRANT + + (hydrant.getSegmentNumDimensionColumns() * ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER) + + (hydrant.getSegmentNumMetricColumns() * ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + + ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER; return total; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index 79f587c7ec3e..f390f767ae7d 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -188,8 +188,7 @@ public Appenderator createRealtimeAppenderatorForTask( wrapIndexMerger(indexMerger), cache, rowIngestionMeters, - parseExceptionHandler, - true + parseExceptionHandler ); datasourceBundle.addAppenderator(taskId, appenderator); @@ -209,7 +208,7 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean batchFallback + boolean useLegacyBatchProcessing ) { synchronized (this) { @@ -229,7 +228,7 @@ public Appenderator createOfflineAppenderatorForTask( wrapIndexMerger(indexMerger), rowIngestionMeters, parseExceptionHandler, - batchFallback + useLegacyBatchProcessing ); datasourceBundle.addAppenderator(taskId, appenderator); return appenderator; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java index 3b325d10997b..05b26b4770df 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTest.java @@ -19,7 +19,6 @@ 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; @@ -31,7 +30,6 @@ import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; 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; @@ -114,14 +112,7 @@ public void testSimpleIngestion() throws Exception IDENTIFIERS.subList(0, 3), Lists.transform( segmentsAndCommitMetadata.getSegments(), - new Function() - { - @Override - public SegmentIdWithShardSpec apply(DataSegment input) - { - return SegmentIdWithShardSpec.fromDataSegment(input); - } - } + SegmentIdWithShardSpec::fromDataSegment ).stream().sorted().collect(Collectors.toList()) ); Assert.assertEquals( @@ -207,14 +198,7 @@ public void testSimpleIngestionWithFallbackCodePath() throws Exception IDENTIFIERS.subList(0, 3), Lists.transform( segmentsAndCommitMetadata.getSegments(), - new Function() - { - @Override - public SegmentIdWithShardSpec apply(DataSegment input) - { - return SegmentIdWithShardSpec.fromDataSegment(input); - } - } + SegmentIdWithShardSpec::fromDataSegment ).stream().sorted().collect(Collectors.toList()) ); Assert.assertEquals( @@ -648,14 +632,7 @@ public void testAllHydrantsAreRecovered() throws Exception IDENTIFIERS.subList(0, 1), Lists.transform( segmentsAndCommitMetadata.getSegments(), - new Function() - { - @Override - public SegmentIdWithShardSpec apply(DataSegment input) - { - return SegmentIdWithShardSpec.fromDataSegment(input); - } - } + SegmentIdWithShardSpec::fromDataSegment ).stream().sorted().collect(Collectors.toList()) ); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -890,14 +867,7 @@ public void testPushContract() throws Exception Collections.singletonList(IDENTIFIERS.get(0)), Lists.transform( segmentsAndCommitMetadata.getSegments(), - new Function() - { - @Override - public SegmentIdWithShardSpec apply(DataSegment input) - { - return SegmentIdWithShardSpec.fromDataSegment(input); - } - } + SegmentIdWithShardSpec::fromDataSegment ).stream().sorted().collect(Collectors.toList()) ); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java index 8fe3ccab68a0..ecb788cd746a 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java @@ -19,8 +19,6 @@ package org.apache.druid.segment.realtime.appenderator; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -44,7 +42,6 @@ import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -77,7 +74,6 @@ public class BatchAppenderatorTester implements AutoCloseable private final DataSchema schema; private final AppenderatorConfig tuningConfig; private final FireDepartmentMetrics metrics; - private final DataSegmentPusher dataSegmentPusher; private final ObjectMapper objectMapper; private final Appenderator appenderator; private final ServiceEmitter emitter; @@ -146,7 +142,7 @@ public BatchAppenderatorTester( final boolean enablePushFailure, final RowIngestionMeters rowIngestionMeters, final boolean skipBytesInMemoryOverheadCheck, - final boolean batchFallback + final boolean useLegacyBatchProcessing ) { objectMapper = new DefaultObjectMapper(); @@ -175,7 +171,7 @@ public BatchAppenderatorTester( null, objectMapper ); - tuningConfig = new IndexTuningConfig( + tuningConfig = new TestIndexTuningConfig( null, 2, null, @@ -199,21 +195,14 @@ public BatchAppenderatorTester( null, null, null, - null - ).withBasePersistDirectory(basePersistDirectory != null ? basePersistDirectory : createNewBasePersistDirectory()); - + null, + basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory + ); metrics = new FireDepartmentMetrics(); IndexIO indexIO = new IndexIO( objectMapper, - new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 0; - } - } + () -> 0 ); IndexMerger indexMerger = new IndexMergerV9( objectMapper, @@ -228,7 +217,7 @@ public int columnCacheSizeBytes() ); emitter.start(); EmittingLogger.registerEmitter(emitter); - dataSegmentPusher = new DataSegmentPusher() + DataSegmentPusher dataSegmentPusher = new DataSegmentPusher() { private boolean mustFail = true; @@ -275,7 +264,7 @@ public Map makeLoadSpec(URI uri) indexMerger, rowIngestionMeters, new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - batchFallback + useLegacyBatchProcessing ); } @@ -328,10 +317,11 @@ private static File createNewBasePersistDirectory() } - // copied from druid-indexing as is for testing since it is not accessible from server module, - // we could simplify since not all its functionality is being used - // but leaving as is, it could be useful later - private static class IndexTuningConfig implements AppenderatorConfig + // copied from druid-indexing testing since it is not accessible from server module, + // Cleaned up a little but Leaving mostly as-is since most of the functionality is + // setting defaults when passing null and those defaults are + // required for the appenderator to work + private static class TestIndexTuningConfig implements AppenderatorConfig { private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; @@ -404,33 +394,32 @@ private static PartitionsSpec getPartitionsSpec( } } - @JsonCreator - public IndexTuningConfig( - @JsonProperty("targetPartitionSize") @Deprecated @Nullable Integer targetPartitionSize, - @JsonProperty("maxRowsPerSegment") @Deprecated @Nullable Integer maxRowsPerSegment, - @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, - @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, - @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, - @JsonProperty("skipBytesInMemoryOverheadCheck") @Nullable Boolean skipBytesInMemoryOverheadCheck, - @JsonProperty("maxTotalRows") @Deprecated @Nullable Long maxTotalRows, - @JsonProperty("rowFlushBoundary") @Deprecated @Nullable Integer rowFlushBoundary_forBackCompatibility, - @JsonProperty("numShards") @Deprecated @Nullable Integer numShards, - @JsonProperty("partitionDimensions") @Deprecated @Nullable List partitionDimensions, - @JsonProperty("partitionsSpec") @Nullable PartitionsSpec partitionsSpec, - @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, - @JsonProperty("indexSpecForIntermediatePersists") @Nullable IndexSpec indexSpecForIntermediatePersists, - @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, - @JsonProperty("forceGuaranteedRollup") @Nullable Boolean forceGuaranteedRollup, - @JsonProperty("reportParseExceptions") @Deprecated @Nullable Boolean reportParseExceptions, - @JsonProperty("publishTimeout") @Deprecated @Nullable Long publishTimeout, - @JsonProperty("pushTimeout") @Nullable Long pushTimeout, - @JsonProperty("segmentWriteOutMediumFactory") @Nullable - SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, - @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, - @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, - @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge, - @JsonProperty("awaitSegmentAvailabilityTimeoutMillis") @Nullable Long awaitSegmentAvailabilityTimeoutMillis + public TestIndexTuningConfig( + Integer targetPartitionSize, + Integer maxRowsPerSegment, + AppendableIndexSpec appendableIndexSpec, + Integer maxRowsInMemory, + Long maxBytesInMemory, + Boolean skipBytesInMemoryOverheadCheck, + Long maxTotalRows, + Integer rowFlushBoundary_forBackCompatibility, + Integer numShards, + List partitionDimensions, + PartitionsSpec partitionsSpec, + IndexSpec indexSpec, + IndexSpec indexSpecForIntermediatePersists, + Integer maxPendingPersists, + Boolean forceGuaranteedRollup, + Boolean reportParseExceptions, + Long publishTimeout, + Long pushTimeout, + SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + Boolean logParseExceptions, + Integer maxParseExceptions, + Integer maxSavedParseExceptions, + Integer maxColumnsToMerge, + Long awaitSegmentAvailabilityTimeoutMillis, + File basePersistDir ) { this( @@ -454,7 +443,7 @@ public IndexTuningConfig( forceGuaranteedRollup, reportParseExceptions, pushTimeout != null ? pushTimeout : publishTimeout, - null, + basePersistDir, segmentWriteOutMediumFactory, logParseExceptions, maxParseExceptions, @@ -469,30 +458,25 @@ public IndexTuningConfig( ); } - private IndexTuningConfig() - { - this(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); - } - - private IndexTuningConfig( - @Nullable AppendableIndexSpec appendableIndexSpec, - @Nullable Integer maxRowsInMemory, - @Nullable Long maxBytesInMemory, - @Nullable Boolean skipBytesInMemoryOverheadCheck, - @Nullable PartitionsSpec partitionsSpec, - @Nullable IndexSpec indexSpec, - @Nullable IndexSpec indexSpecForIntermediatePersists, - @Nullable Integer maxPendingPersists, - @Nullable Boolean forceGuaranteedRollup, - @Nullable Boolean reportParseExceptions, - @Nullable Long pushTimeout, - @Nullable File basePersistDirectory, - @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - @Nullable Boolean logParseExceptions, - @Nullable Integer maxParseExceptions, - @Nullable Integer maxSavedParseExceptions, - @Nullable Integer maxColumnsToMerge, - @Nullable Long awaitSegmentAvailabilityTimeoutMillis + private TestIndexTuningConfig( + AppendableIndexSpec appendableIndexSpec, + Integer maxRowsInMemory, + Long maxBytesInMemory, + Boolean skipBytesInMemoryOverheadCheck, + @Nullable PartitionsSpec partitionsSpec, + IndexSpec indexSpec, + IndexSpec indexSpecForIntermediatePersists, + Integer maxPendingPersists, + Boolean forceGuaranteedRollup, + Boolean reportParseExceptions, + Long pushTimeout, + File basePersistDirectory, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + Boolean logParseExceptions, + Integer maxParseExceptions, + Integer maxSavedParseExceptions, + Integer maxColumnsToMerge, + Long awaitSegmentAvailabilityTimeoutMillis ) { this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; @@ -543,83 +527,35 @@ private IndexTuningConfig( } @Override - public IndexTuningConfig withBasePersistDirectory(File dir) + public TestIndexTuningConfig withBasePersistDirectory(File dir) { - return new IndexTuningConfig( - appendableIndexSpec, - maxRowsInMemory, - maxBytesInMemory, - skipBytesInMemoryOverheadCheck, - partitionsSpec, - indexSpec, - indexSpecForIntermediatePersists, - maxPendingPersists, - forceGuaranteedRollup, - reportParseExceptions, - pushTimeout, - dir, - segmentWriteOutMediumFactory, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions, - maxColumnsToMerge, - awaitSegmentAvailabilityTimeoutMillis - ); + throw new UnsupportedOperationException(); } - public IndexTuningConfig withPartitionsSpec(PartitionsSpec partitionsSpec) - { - return new IndexTuningConfig( - appendableIndexSpec, - maxRowsInMemory, - maxBytesInMemory, - skipBytesInMemoryOverheadCheck, - partitionsSpec, - indexSpec, - indexSpecForIntermediatePersists, - maxPendingPersists, - forceGuaranteedRollup, - reportParseExceptions, - pushTimeout, - basePersistDirectory, - segmentWriteOutMediumFactory, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions, - maxColumnsToMerge, - awaitSegmentAvailabilityTimeoutMillis - ); - } - - @JsonProperty @Override public AppendableIndexSpec getAppendableIndexSpec() { return appendableIndexSpec; } - - @JsonProperty + @Override public int getMaxRowsInMemory() { return maxRowsInMemory; } - - @JsonProperty + @Override public long getMaxBytesInMemory() { return maxBytesInMemory; } - - @JsonProperty + @Override public boolean isSkipBytesInMemoryOverheadCheck() { return skipBytesInMemoryOverheadCheck; } - - @JsonProperty + @Nullable @Override public PartitionsSpec getPartitionsSpec() @@ -637,75 +573,58 @@ public PartitionsSpec getGivenOrDefaultPartitionsSpec() : new DynamicPartitionsSpec(null, null); } - @JsonProperty @Override public IndexSpec getIndexSpec() { return indexSpec; } - - @JsonProperty + @Override public IndexSpec getIndexSpecForIntermediatePersists() { return indexSpecForIntermediatePersists; } - - @JsonProperty + @Override public int getMaxPendingPersists() { return maxPendingPersists; } - - @JsonProperty public boolean isForceGuaranteedRollup() { return forceGuaranteedRollup; } - @JsonProperty @Override public boolean isReportParseExceptions() { return reportParseExceptions; } - @JsonProperty - public long getPushTimeout() - { - return pushTimeout; - } - @Nullable @Override - @JsonProperty public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() { return segmentWriteOutMediumFactory; } @Override - @JsonProperty public int getMaxColumnsToMerge() { return maxColumnsToMerge; } - @JsonProperty public boolean isLogParseExceptions() { return logParseExceptions; } - @JsonProperty public int getMaxParseExceptions() { return maxParseExceptions; } - @JsonProperty public int getMaxSavedParseExceptions() { return maxSavedParseExceptions; @@ -718,7 +637,6 @@ public int getMaxSavedParseExceptions() @Nullable @Override @Deprecated - @JsonProperty public Integer getMaxRowsPerSegment() { return partitionsSpec == null ? null : partitionsSpec.getMaxRowsPerSegment(); @@ -731,7 +649,6 @@ public Integer getMaxRowsPerSegment() @Override @Nullable @Deprecated - @JsonProperty public Long getMaxTotalRows() { return partitionsSpec instanceof DynamicPartitionsSpec @@ -741,7 +658,6 @@ public Long getMaxTotalRows() @Deprecated @Nullable - @JsonProperty public Integer getNumShards() { return partitionsSpec instanceof HashedPartitionsSpec @@ -750,7 +666,6 @@ public Integer getNumShards() } @Deprecated - @JsonProperty public List getPartitionDimensions() { return partitionsSpec instanceof HashedPartitionsSpec @@ -770,7 +685,7 @@ public Period getIntermediatePersistPeriod() return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs } - @JsonProperty + public long getAwaitSegmentAvailabilityTimeoutMillis() { return awaitSegmentAvailabilityTimeoutMillis; @@ -785,7 +700,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - IndexTuningConfig that = (IndexTuningConfig) o; + TestIndexTuningConfig that = (TestIndexTuningConfig) o; return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) && maxRowsInMemory == that.maxRowsInMemory && maxBytesInMemory == that.maxBytesInMemory && From 291039e5388a9b6f835204e097dd4a6aeaefad98 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Thu, 8 Jul 2021 12:00:58 -0700 Subject: [PATCH 44/47] Update javadocs --- .../realtime/appenderator/Appenderator.java | 6 ++++++ .../appenderator/BatchAppenderator.java | 18 ++++++++++++------ 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java index 2d538e77a473..26d6f5804fa8 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java @@ -43,6 +43,12 @@ * Concurrency: all methods defined in this class directly, including {@link #close()} and {@link #closeNow()}, i. e. * all methods of the data appending and indexing lifecycle except {@link #drop} must be called from a single thread. * Methods inherited from {@link QuerySegmentWalker} can be called concurrently from multiple threads. + *

+ * 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 + * makes sense to have as an {@code Appenderator}. In the future we may want to refactor away the {@code Appenderator} + * interface from {@code BatchAppenderator}. */ public interface Appenderator extends QuerySegmentWalker { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 953438436105..03f674e387ef 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -78,6 +78,18 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; +/** + * This is a new class produced when the old {@code AppenderatorImpl} was split. For historical + * reasons, the code for creating segments was all handled by the same code path in that class. The code + * was correct but inefficient for batch ingestion from a memory perspective. If the input file being processed + * by batch ingestion had enough sinks & hydrants produced then it may run out of memory either in the + * 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}. + *

+ * It is important to realize that this class is completely synchronous despite the {@link Appenderator} + * interface suggesting otherwise. + */ public class BatchAppenderator implements Appenderator { public static final int ROUGH_OVERHEAD_PER_SINK = 5000; @@ -95,12 +107,6 @@ public class BatchAppenderator implements Appenderator private final ObjectMapper objectMapper; private final IndexIO indexIO; private final IndexMerger indexMerger; - /** - * This map needs to be concurrent because it's accessed and mutated from multiple threads from where - * this Appenderator is used (and methods like {@link #add(SegmentIdWithShardSpec, InputRow, Supplier, boolean)} are - * called). It could also be accessed (but not mutated) potentially in the context - * of any thread from {@link #drop}. - */ private final Map sinks = new HashMap<>(); private final long maxBytesTuningConfig; private final boolean skipBytesInMemoryOverheadCheck; From daf233bc5fc6626a1751172ec87ee3b52997b66f Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Thu, 8 Jul 2021 12:31:28 -0700 Subject: [PATCH 45/47] Add thread safety notice to BatchAppenderator --- .../segment/realtime/appenderator/BatchAppenderator.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 03f674e387ef..be5cadd0d5b4 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -61,6 +61,7 @@ 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; @@ -87,9 +88,11 @@ * {@code BatchAppenderator}, this class, was created to specialize in batch ingestion and the old class * for stream ingestion was renamed to {@link StreamAppenderator}. *

+ * This class is not thread safe!. * It is important to realize that this class is completely synchronous despite the {@link Appenderator} - * interface suggesting otherwise. + * 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; From 0dedc8720bdb3155a1baae21d9bfbf7cedaa1722 Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Thu, 8 Jul 2021 18:14:45 -0700 Subject: [PATCH 46/47] Further cleanup config --- .../appenderator/BatchAppenderatorTester.java | 271 +++--------------- 1 file changed, 36 insertions(+), 235 deletions(-) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java index ecb788cd746a..0ed323eabdf9 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.realtime.appenderator; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.MapInputRowParser; @@ -30,7 +29,6 @@ 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.IAE; 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; @@ -61,7 +59,6 @@ import java.io.File; import java.io.IOException; import java.net.URI; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -160,42 +157,37 @@ public BatchAppenderatorTester( ), Map.class ); + schema = new DataSchema( DATASOURCE, - parserMap, + null, + null, new AggregatorFactory[]{ new CountAggregatorFactory("count"), new LongSumAggregatorFactory("met", "met") }, new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), null, + parserMap, objectMapper ); + tuningConfig = new TestIndexTuningConfig( - null, - 2, - null, + TuningConfig.DEFAULT_APPENDABLE_INDEX, maxRowsInMemory, maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, skipBytesInMemoryOverheadCheck, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, + new IndexSpec(), + 0, + false, + false, + 0L, OffHeapMemorySegmentWriteOutMediumFactory.instance(), - true, - null, - null, - null, - null, + false, + 0, + 1, + IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, + 0L, basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory ); metrics = new FireDepartmentMetrics(); @@ -317,32 +309,17 @@ private static File createNewBasePersistDirectory() } - // copied from druid-indexing testing since it is not accessible from server module, - // Cleaned up a little but Leaving mostly as-is since most of the functionality is - // setting defaults when passing null and those defaults are - // required for the appenderator to work private static class TestIndexTuningConfig implements AppenderatorConfig { - private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); - private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; - private static final boolean DEFAULT_GUARANTEE_ROLLUP = false; - private static final boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = false; - private static final long DEFAULT_PUSH_TIMEOUT = 0; - private final AppendableIndexSpec appendableIndexSpec; private final int maxRowsInMemory; private final long maxBytesInMemory; private final boolean skipBytesInMemoryOverheadCheck; private final int maxColumnsToMerge; - - // null if all partitionsSpec related params are null. see getDefaultPartitionsSpec() for details. - @Nullable private final PartitionsSpec partitionsSpec; private final IndexSpec indexSpec; - private final IndexSpec indexSpecForIntermediatePersists; private final File basePersistDirectory; private final int maxPendingPersists; - private final boolean forceGuaranteedRollup; private final boolean reportParseExceptions; private final long pushTimeout; @@ -350,180 +327,48 @@ private static class TestIndexTuningConfig implements AppenderatorConfig private final int maxParseExceptions; private final int maxSavedParseExceptions; private final long awaitSegmentAvailabilityTimeoutMillis; - + private final IndexSpec indexSpecForIntermediatePersists; @Nullable private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - @Nullable - private static PartitionsSpec getPartitionsSpec( - boolean forceGuaranteedRollup, - @Nullable PartitionsSpec partitionsSpec, - @Nullable Integer maxRowsPerSegment, - @Nullable Long maxTotalRows, - @Nullable Integer numShards, - @Nullable List partitionDimensions - ) - { - if (partitionsSpec == null) { - if (forceGuaranteedRollup) { - if (maxRowsPerSegment != null - || numShards != null - || (partitionDimensions != null && !partitionDimensions.isEmpty())) { - return new HashedPartitionsSpec(maxRowsPerSegment, numShards, partitionDimensions); - } else { - return null; - } - } else { - if (maxRowsPerSegment != null || maxTotalRows != null) { - return new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); - } else { - return null; - } - } - } else { - if (forceGuaranteedRollup) { - if (!partitionsSpec.isForceGuaranteedRollupCompatibleType()) { - throw new IAE(partitionsSpec.getClass().getSimpleName() + " cannot be used for perfect rollup"); - } - } else { - if (!(partitionsSpec instanceof DynamicPartitionsSpec)) { - throw new IAE("DynamicPartitionsSpec must be used for best-effort rollup"); - } - } - return partitionsSpec; - } - } - public TestIndexTuningConfig( - Integer targetPartitionSize, - Integer maxRowsPerSegment, - AppendableIndexSpec appendableIndexSpec, - Integer maxRowsInMemory, - Long maxBytesInMemory, - Boolean skipBytesInMemoryOverheadCheck, - Long maxTotalRows, - Integer rowFlushBoundary_forBackCompatibility, - Integer numShards, - List partitionDimensions, - PartitionsSpec partitionsSpec, - IndexSpec indexSpec, - IndexSpec indexSpecForIntermediatePersists, - Integer maxPendingPersists, - Boolean forceGuaranteedRollup, - Boolean reportParseExceptions, - Long publishTimeout, - Long pushTimeout, - SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - Boolean logParseExceptions, - Integer maxParseExceptions, - Integer maxSavedParseExceptions, - Integer maxColumnsToMerge, - Long awaitSegmentAvailabilityTimeoutMillis, - File basePersistDir - ) - { - this( - appendableIndexSpec, - maxRowsInMemory != null ? maxRowsInMemory : rowFlushBoundary_forBackCompatibility, - maxBytesInMemory != null ? maxBytesInMemory : 0, - skipBytesInMemoryOverheadCheck != null - ? skipBytesInMemoryOverheadCheck - : DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK, - getPartitionsSpec( - forceGuaranteedRollup == null ? DEFAULT_GUARANTEE_ROLLUP : forceGuaranteedRollup, - partitionsSpec, - maxRowsPerSegment == null ? targetPartitionSize : maxRowsPerSegment, - maxTotalRows, - numShards, - partitionDimensions - ), - indexSpec, - indexSpecForIntermediatePersists, - maxPendingPersists, - forceGuaranteedRollup, - reportParseExceptions, - pushTimeout != null ? pushTimeout : publishTimeout, - basePersistDir, - segmentWriteOutMediumFactory, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions, - maxColumnsToMerge, - awaitSegmentAvailabilityTimeoutMillis - ); - - Preconditions.checkArgument( - targetPartitionSize == null || maxRowsPerSegment == null, - "Can't use targetPartitionSize and maxRowsPerSegment together" - ); - } - - private TestIndexTuningConfig( AppendableIndexSpec appendableIndexSpec, Integer maxRowsInMemory, Long maxBytesInMemory, Boolean skipBytesInMemoryOverheadCheck, - @Nullable PartitionsSpec partitionsSpec, IndexSpec indexSpec, - IndexSpec indexSpecForIntermediatePersists, Integer maxPendingPersists, Boolean forceGuaranteedRollup, Boolean reportParseExceptions, Long pushTimeout, - File basePersistDirectory, @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, Boolean logParseExceptions, Integer maxParseExceptions, Integer maxSavedParseExceptions, Integer maxColumnsToMerge, - Long awaitSegmentAvailabilityTimeoutMillis + Long awaitSegmentAvailabilityTimeoutMillis, + File basePersistDirectory ) { - this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; - this.maxRowsInMemory = maxRowsInMemory == null ? TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - // initializing this to 0, it will be lazily initialized to a value - // @see #getMaxBytesInMemoryOrDefault() - this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; - this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck == null - ? - DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK - : skipBytesInMemoryOverheadCheck; - this.maxColumnsToMerge = maxColumnsToMerge == null - ? IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE - : maxColumnsToMerge; - this.partitionsSpec = partitionsSpec; - this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; - this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists == null ? - this.indexSpec : indexSpecForIntermediatePersists; - this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists; - this.forceGuaranteedRollup = forceGuaranteedRollup == null ? DEFAULT_GUARANTEE_ROLLUP : forceGuaranteedRollup; - this.reportParseExceptions = reportParseExceptions == null - ? DEFAULT_REPORT_PARSE_EXCEPTIONS - : reportParseExceptions; - this.pushTimeout = pushTimeout == null ? DEFAULT_PUSH_TIMEOUT : pushTimeout; - this.basePersistDirectory = basePersistDirectory; - + this.appendableIndexSpec = appendableIndexSpec; + this.maxRowsInMemory = maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory; + this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck; + this.indexSpec = indexSpec; + this.maxPendingPersists = maxPendingPersists; + this.forceGuaranteedRollup = forceGuaranteedRollup; + this.reportParseExceptions = reportParseExceptions; + this.pushTimeout = pushTimeout; this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + this.logParseExceptions = logParseExceptions; + this.maxParseExceptions = maxParseExceptions; + this.maxSavedParseExceptions = Math.min(1, maxSavedParseExceptions); + this.maxColumnsToMerge = maxColumnsToMerge; + this.awaitSegmentAvailabilityTimeoutMillis = awaitSegmentAvailabilityTimeoutMillis; + this.basePersistDirectory = basePersistDirectory; - if (this.reportParseExceptions) { - this.maxParseExceptions = 0; - this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions); - } else { - this.maxParseExceptions = maxParseExceptions == null - ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS - : maxParseExceptions; - this.maxSavedParseExceptions = maxSavedParseExceptions == null - ? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS - : maxSavedParseExceptions; - } - this.logParseExceptions = logParseExceptions == null - ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS - : logParseExceptions; - if (awaitSegmentAvailabilityTimeoutMillis == null || awaitSegmentAvailabilityTimeoutMillis < 0) { - this.awaitSegmentAvailabilityTimeoutMillis = DEFAULT_AWAIT_SEGMENT_AVAILABILITY_TIMEOUT_MILLIS; - } else { - this.awaitSegmentAvailabilityTimeoutMillis = awaitSegmentAvailabilityTimeoutMillis; - } + this.partitionsSpec = null; + this.indexSpecForIntermediatePersists = this.indexSpec; } @Override @@ -630,49 +475,6 @@ public int getMaxSavedParseExceptions() return maxSavedParseExceptions; } - /** - * Return the max number of rows per segment. This returns null if it's not specified in tuningConfig. - * Deprecated in favor of {@link #getGivenOrDefaultPartitionsSpec()}. - */ - @Nullable - @Override - @Deprecated - public Integer getMaxRowsPerSegment() - { - return partitionsSpec == null ? null : partitionsSpec.getMaxRowsPerSegment(); - } - - /** - * Return the max number of total rows in appenderator. This returns null if it's not specified in tuningConfig. - * Deprecated in favor of {@link #getGivenOrDefaultPartitionsSpec()}. - */ - @Override - @Nullable - @Deprecated - public Long getMaxTotalRows() - { - return partitionsSpec instanceof DynamicPartitionsSpec - ? ((DynamicPartitionsSpec) partitionsSpec).getMaxTotalRows() - : null; - } - - @Deprecated - @Nullable - public Integer getNumShards() - { - return partitionsSpec instanceof HashedPartitionsSpec - ? ((HashedPartitionsSpec) partitionsSpec).getNumShards() - : null; - } - - @Deprecated - public List getPartitionDimensions() - { - return partitionsSpec instanceof HashedPartitionsSpec - ? ((HashedPartitionsSpec) partitionsSpec).getPartitionDimensions() - : Collections.emptyList(); - } - @Override public File getBasePersistDirectory() { @@ -684,7 +486,6 @@ public Period getIntermediatePersistPeriod() { return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs } - public long getAwaitSegmentAvailabilityTimeoutMillis() { From effe090b1425bed880e7468462dd8e5b1e026e3c Mon Sep 17 00:00:00 2001 From: Agustin Gonzalez Date: Thu, 8 Jul 2021 19:15:05 -0700 Subject: [PATCH 47/47] More config cleanup --- .../appenderator/BatchAppenderatorTester.java | 76 +------------------ 1 file changed, 2 insertions(+), 74 deletions(-) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java index 0ed323eabdf9..af628ba2767c 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java @@ -24,8 +24,6 @@ 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.DynamicPartitionsSpec; -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.FileUtils; @@ -180,14 +178,9 @@ public BatchAppenderatorTester( new IndexSpec(), 0, false, - false, 0L, OffHeapMemorySegmentWriteOutMediumFactory.instance(), - false, - 0, - 1, IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, - 0L, basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory ); metrics = new FireDepartmentMetrics(); @@ -320,13 +313,8 @@ private static class TestIndexTuningConfig implements AppenderatorConfig private final IndexSpec indexSpec; private final File basePersistDirectory; private final int maxPendingPersists; - private final boolean forceGuaranteedRollup; private final boolean reportParseExceptions; private final long pushTimeout; - private final boolean logParseExceptions; - private final int maxParseExceptions; - private final int maxSavedParseExceptions; - private final long awaitSegmentAvailabilityTimeoutMillis; private final IndexSpec indexSpecForIntermediatePersists; @Nullable private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; @@ -338,15 +326,10 @@ public TestIndexTuningConfig( Boolean skipBytesInMemoryOverheadCheck, IndexSpec indexSpec, Integer maxPendingPersists, - Boolean forceGuaranteedRollup, Boolean reportParseExceptions, Long pushTimeout, @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - Boolean logParseExceptions, - Integer maxParseExceptions, - Integer maxSavedParseExceptions, Integer maxColumnsToMerge, - Long awaitSegmentAvailabilityTimeoutMillis, File basePersistDirectory ) { @@ -356,15 +339,10 @@ public TestIndexTuningConfig( this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck; this.indexSpec = indexSpec; this.maxPendingPersists = maxPendingPersists; - this.forceGuaranteedRollup = forceGuaranteedRollup; this.reportParseExceptions = reportParseExceptions; this.pushTimeout = pushTimeout; this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; - this.logParseExceptions = logParseExceptions; - this.maxParseExceptions = maxParseExceptions; - this.maxSavedParseExceptions = Math.min(1, maxSavedParseExceptions); this.maxColumnsToMerge = maxColumnsToMerge; - this.awaitSegmentAvailabilityTimeoutMillis = awaitSegmentAvailabilityTimeoutMillis; this.basePersistDirectory = basePersistDirectory; this.partitionsSpec = null; @@ -408,16 +386,6 @@ public PartitionsSpec getPartitionsSpec() return partitionsSpec; } - public PartitionsSpec getGivenOrDefaultPartitionsSpec() - { - if (partitionsSpec != null) { - return partitionsSpec; - } - return forceGuaranteedRollup - ? new HashedPartitionsSpec(null, null, null) - : new DynamicPartitionsSpec(null, null); - } - @Override public IndexSpec getIndexSpec() { @@ -436,11 +404,6 @@ public int getMaxPendingPersists() return maxPendingPersists; } - public boolean isForceGuaranteedRollup() - { - return forceGuaranteedRollup; - } - @Override public boolean isReportParseExceptions() { @@ -460,21 +423,6 @@ public int getMaxColumnsToMerge() return maxColumnsToMerge; } - public boolean isLogParseExceptions() - { - return logParseExceptions; - } - - public int getMaxParseExceptions() - { - return maxParseExceptions; - } - - public int getMaxSavedParseExceptions() - { - return maxSavedParseExceptions; - } - @Override public File getBasePersistDirectory() { @@ -487,11 +435,6 @@ public Period getIntermediatePersistPeriod() return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs } - public long getAwaitSegmentAvailabilityTimeoutMillis() - { - return awaitSegmentAvailabilityTimeoutMillis; - } - @Override public boolean equals(Object o) { @@ -508,18 +451,13 @@ public boolean equals(Object o) skipBytesInMemoryOverheadCheck == that.skipBytesInMemoryOverheadCheck && maxColumnsToMerge == that.maxColumnsToMerge && maxPendingPersists == that.maxPendingPersists && - forceGuaranteedRollup == that.forceGuaranteedRollup && reportParseExceptions == that.reportParseExceptions && pushTimeout == that.pushTimeout && - logParseExceptions == that.logParseExceptions && - maxParseExceptions == that.maxParseExceptions && - maxSavedParseExceptions == that.maxSavedParseExceptions && 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) && - Objects.equals(awaitSegmentAvailabilityTimeoutMillis, that.awaitSegmentAvailabilityTimeoutMillis); + Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory); } @Override @@ -536,14 +474,9 @@ public int hashCode() indexSpecForIntermediatePersists, basePersistDirectory, maxPendingPersists, - forceGuaranteedRollup, reportParseExceptions, pushTimeout, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions, - segmentWriteOutMediumFactory, - awaitSegmentAvailabilityTimeoutMillis + segmentWriteOutMediumFactory ); } @@ -560,14 +493,9 @@ public String toString() ", indexSpecForIntermediatePersists=" + indexSpecForIntermediatePersists + ", basePersistDirectory=" + basePersistDirectory + ", maxPendingPersists=" + maxPendingPersists + - ", forceGuaranteedRollup=" + forceGuaranteedRollup + ", reportParseExceptions=" + reportParseExceptions + ", pushTimeout=" + pushTimeout + - ", logParseExceptions=" + logParseExceptions + - ", maxParseExceptions=" + maxParseExceptions + - ", maxSavedParseExceptions=" + maxSavedParseExceptions + ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + - ", awaitSegmentAvailabilityTimeoutMillis=" + awaitSegmentAvailabilityTimeoutMillis + '}'; } }