From 58a3acc2c4c5d5d22098e15f7ff25d6e9baa8541 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 13 Dec 2022 18:54:42 +0530 Subject: [PATCH] Add InputStats to track bytes processed by a task (#13520) This commit adds a new class `InputStats` to track the total bytes processed by a task. The field `processedBytes` is published in task reports along with other row stats. Major changes: - Add class `InputStats` to track processed bytes - Add method `InputSourceReader.read(InputStats)` to read input rows while counting bytes. > Since we need to count the bytes, we could not just have a wrapper around `InputSourceReader` or `InputEntityReader` (the way `CountableInputSourceReader` does) because the `InputSourceReader` only deals with `InputRow`s and the byte information is already lost. - Classic batch: Use the new `InputSourceReader.read(inputStats)` in `AbstractBatchIndexTask` - Streaming: Increment `processedBytes` in `StreamChunkParser`. This does not use the new `InputSourceReader.read(inputStats)` method. - Extend `InputStats` with `RowIngestionMeters` so that bytes can be exposed in task reports Other changes: - Update tests to verify the value of `processedBytes` - Rename `MutableRowIngestionMeters` to `SimpleRowIngestionMeters` and remove duplicate class - Replace `CacheTestSegmentCacheManager` with `NoopSegmentCacheManager` - Refactor `KafkaIndexTaskTest` and `KinesisIndexTaskTest` --- .../data/input/BytesCountingInputEntity.java | 123 ++ .../druid/data/input/InputSourceReader.java | 7 +- .../apache/druid/data/input/InputStats.java | 30 + .../FirehoseToInputSourceReaderAdaptor.java | 3 +- .../impl/InputEntityIteratingReader.java | 7 +- .../impl/TimedShutoffInputSourceReader.java | 5 +- .../druid/java/util/common/FileUtils.java | 23 + .../input/BytesCountingInputEntityTest.java | 145 ++ .../impl/InputEntityIteratingReaderTest.java | 13 +- .../druid/data/input/impl/InputStatsImpl.java | 39 + .../apache/druid/math/expr/FunctionTest.java | 3 +- .../data/input/aliyun/OssInputSourceTest.java | 10 +- .../GoogleCloudStorageInputSourceTest.java | 10 +- .../inputsource/hdfs/HdfsInputSourceTest.java | 24 +- .../indexing/kafka/KafkaIndexTaskTest.java | 639 ++----- .../kinesis/KinesisIndexTaskTest.java | 1613 +++++------------ .../indexing/CountableInputSourceReader.java | 5 +- .../druid/msq/input/NilInputSource.java | 3 +- .../data/input/s3/S3InputSourceTest.java | 10 +- .../stats/DropwizardRowIngestionMeters.java | 18 + .../stats/TaskRealtimeMetricsMonitor.java | 2 +- .../common/task/AbstractBatchIndexTask.java | 2 +- .../parallel/ParallelIndexSupervisorTask.java | 20 +- .../input/DruidSegmentInputFormat.java | 22 +- .../indexing/input/DruidSegmentReader.java | 17 +- .../input/DruidTombstoneSegmentReader.java | 15 +- .../indexing/input/GeneratorInputSource.java | 3 +- .../seekablestream/StreamChunkParser.java | 19 +- ...penderatorDriverRealtimeIndexTaskTest.java | 56 +- .../task/CompactionTaskParallelRunTest.java | 45 +- .../common/task/CompactionTaskTest.java | 28 +- .../indexing/common/task/IndexTaskTest.java | 111 +- .../common/task/ParseExceptionReport.java | 71 + ...stractParallelIndexSupervisorTaskTest.java | 1 + ...ultiPhaseParallelIndexingRowStatsTest.java | 8 +- .../SinglePhaseParallelIndexingTest.java | 29 +- .../input/DruidSegmentReaderTest.java | 175 +- .../indexing/overlord/TaskLifecycleTest.java | 5 +- .../RecordSupplierInputSourceTest.java | 15 +- .../SeekableStreamIndexTaskTestBase.java | 217 ++- .../segment/generator/DataGenerator.java | 3 +- .../incremental/NoopRowIngestionMeters.java | 2 +- .../incremental/RowIngestionMeters.java | 16 +- .../incremental/RowIngestionMetersTotals.java | 13 +- ...ers.java => SimpleRowIngestionMeters.java} | 32 +- .../TransformingInputSourceReader.java | 5 +- ...RowIngestionMeters.java => RowMeters.java} | 71 +- ...java => SimpleRowIngestionMetersTest.java} | 23 +- .../druid/metadata/input/SqlReader.java | 4 +- .../SQLMetadataStorageActionHandlerTest.java | 24 +- .../druid/metadata/input/SqlEntityTest.java | 76 +- .../metadata/input/SqlInputSourceTest.java | 116 +- .../druid/metadata/input/SqlTestUtils.java | 45 +- ...ager.java => NoopSegmentCacheManager.java} | 20 +- .../firehose/SqlFirehoseFactoryTest.java | 54 +- .../SegmentLoadDropHandlerTest.java | 57 +- 56 files changed, 1783 insertions(+), 2369 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/data/input/BytesCountingInputEntity.java create mode 100644 core/src/main/java/org/apache/druid/data/input/InputStats.java create mode 100644 core/src/test/java/org/apache/druid/data/input/BytesCountingInputEntityTest.java create mode 100644 core/src/test/java/org/apache/druid/data/input/impl/InputStatsImpl.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/ParseExceptionReport.java rename processing/src/main/java/org/apache/druid/segment/incremental/{MutableRowIngestionMeters.java => SimpleRowIngestionMeters.java} (81%) rename processing/src/test/java/org/apache/druid/segment/incremental/{SimpleRowIngestionMeters.java => RowMeters.java} (50%) rename processing/src/test/java/org/apache/druid/segment/incremental/{MutableRowIngestionMetersTest.java => SimpleRowIngestionMetersTest.java} (58%) rename server/src/test/java/org/apache/druid/segment/loading/{CacheTestSegmentCacheManager.java => NoopSegmentCacheManager.java} (75%) diff --git a/core/src/main/java/org/apache/druid/data/input/BytesCountingInputEntity.java b/core/src/main/java/org/apache/druid/data/input/BytesCountingInputEntity.java new file mode 100644 index 00000000000..21485bfa02f --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/BytesCountingInputEntity.java @@ -0,0 +1,123 @@ +/* + * 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.data.input; + +import com.google.common.base.Predicate; +import com.google.common.io.CountingInputStream; +import org.apache.druid.java.util.common.FileUtils; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; + +/** + * Wrapper around an {@link InputEntity} that counts the number of bytes read. + */ +public class BytesCountingInputEntity implements InputEntity +{ + private final InputStats inputStats; + private final InputEntity baseInputEntity; + + public BytesCountingInputEntity(InputEntity baseInputEntity, InputStats inputStats) + { + this.baseInputEntity = baseInputEntity; + this.inputStats = inputStats; + } + + @Nullable + @Override + public URI getUri() + { + return baseInputEntity.getUri(); + } + + @Override + public InputStream open() throws IOException + { + return new BytesCountingInputStream(baseInputEntity.open(), inputStats); + } + + public InputEntity getBaseInputEntity() + { + return baseInputEntity; + } + + @Override + public CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) throws IOException + { + final CleanableFile cleanableFile = baseInputEntity.fetch(temporaryDirectory, fetchBuffer); + inputStats.incrementProcessedBytes(FileUtils.getFileSize(cleanableFile.file())); + return cleanableFile; + } + + @Override + public Predicate getRetryCondition() + { + return baseInputEntity.getRetryCondition(); + } + + /** + * Wraps an input stream, and counts the number of bytes read. + *

+ * Similar to {@link CountingInputStream} but does not reset count on call to + * {@link CountingInputStream#reset()}. + */ + private static class BytesCountingInputStream extends FilterInputStream + { + private final InputStats inputStats; + + BytesCountingInputStream(@Nullable InputStream in, InputStats inputStats) + { + super(in); + this.inputStats = inputStats; + } + + @Override + public int read() throws IOException + { + int result = in.read(); + if (result != -1) { + inputStats.incrementProcessedBytes(1); + } + return result; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException + { + int result = in.read(b, off, len); + if (result != -1) { + inputStats.incrementProcessedBytes(result); + } + return result; + } + + @Override + public long skip(long n) throws IOException + { + long result = in.skip(n); + inputStats.incrementProcessedBytes(result); + return result; + } + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java index 3e1d233fa3e..23cbcf32fd8 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java @@ -37,7 +37,12 @@ import java.io.IOException; @UnstableApi public interface InputSourceReader { - CloseableIterator read() throws IOException; + default CloseableIterator read() throws IOException + { + return read(null); + } + + CloseableIterator read(InputStats inputStats) throws IOException; CloseableIterator sample() throws IOException; } diff --git a/core/src/main/java/org/apache/druid/data/input/InputStats.java b/core/src/main/java/org/apache/druid/data/input/InputStats.java new file mode 100644 index 00000000000..cc9c970e7ef --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/InputStats.java @@ -0,0 +1,30 @@ +/* + * 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.data.input; + +/** + * Tracks bytes read from an input source. + */ +public interface InputStats +{ + void incrementProcessedBytes(long incrementByValue); + + long getProcessedBytes(); +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java index 697c64fe7a3..778ea3bef30 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java @@ -24,6 +24,7 @@ import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputStats; import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.File; @@ -48,7 +49,7 @@ public class FirehoseToInputSourceReaderAdaptor implements InputSourceReader } @Override - public CloseableIterator read() throws IOException + public CloseableIterator read(InputStats inputStats) throws IOException { return new CloseableIterator() { diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java index a76166a72d5..1d450e79ab7 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java @@ -19,6 +19,7 @@ package org.apache.druid.data.input.impl; +import org.apache.druid.data.input.BytesCountingInputEntity; import org.apache.druid.data.input.InputEntity; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputFormat; @@ -26,6 +27,7 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputStats; import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -69,12 +71,13 @@ public class InputEntityIteratingReader implements InputSourceReader } @Override - public CloseableIterator read() + public CloseableIterator read(InputStats inputStats) { return createIterator(entity -> { // InputEntityReader is stateful and so a new one should be created per entity. try { - final InputEntityReader reader = inputFormat.createReader(inputRowSchema, entity, temporaryDirectory); + final InputEntity entityToRead = inputStats == null ? entity : new BytesCountingInputEntity(entity, inputStats); + final InputEntityReader reader = inputFormat.createReader(inputRowSchema, entityToRead, temporaryDirectory); return reader.read(); } catch (IOException e) { diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java index 632832b3ba9..85e3780ac38 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java @@ -22,6 +22,7 @@ package org.apache.druid.data.input.impl; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputStats; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; @@ -47,10 +48,10 @@ public class TimedShutoffInputSourceReader implements InputSourceReader } @Override - public CloseableIterator read() throws IOException + public CloseableIterator read(InputStats inputStats) throws IOException { final ScheduledExecutorService shutdownExec = Execs.scheduledSingleThreaded("timed-shutoff-reader-%d"); - final CloseableIterator delegateIterator = delegate.read(); + final CloseableIterator delegateIterator = delegate.read(inputStats); return decorateShutdownTimeout(shutdownExec, delegateIterator); } diff --git a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java index d2c7a6d1a9b..43b6682d2c7 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java @@ -388,6 +388,29 @@ public class FileUtils } } + /** + * Computes the size of the file. If it is a directory, computes the size up + * to a depth of 1. + */ + public static long getFileSize(File file) + { + if (file == null) { + return 0; + } else if (file.isDirectory()) { + File[] children = file.listFiles(); + if (children == null) { + return 0; + } + long totalSize = 0; + for (File child : children) { + totalSize += child.length(); + } + return totalSize; + } else { + return file.length(); + } + } + /** * Creates a temporary directory inside the configured temporary space (java.io.tmpdir). Similar to the method * {@link com.google.common.io.Files#createTempDir()} from Guava, but has nicer error messages. diff --git a/core/src/test/java/org/apache/druid/data/input/BytesCountingInputEntityTest.java b/core/src/test/java/org/apache/druid/data/input/BytesCountingInputEntityTest.java new file mode 100644 index 00000000000..6b64fcdc4de --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/BytesCountingInputEntityTest.java @@ -0,0 +1,145 @@ +/* + * 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.data.input; + +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.impl.FileEntity; +import org.apache.druid.data.input.impl.InputStatsImpl; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; + +public class BytesCountingInputEntityTest +{ + @Rule + public TemporaryFolder folder = new TemporaryFolder(); + + private InputStats inputStats; + + @Before + public void setup() + { + inputStats = new InputStatsImpl(); + } + + @Test + public void testFetch() throws IOException + { + final int fileSize = 200; + final File sourceFile = folder.newFile("testWithFileEntity"); + writeBytesToFile(sourceFile, fileSize); + + final BytesCountingInputEntity inputEntity = new BytesCountingInputEntity(new FileEntity(sourceFile), inputStats); + inputEntity.fetch(folder.newFolder(), new byte[50]); + Assert.assertEquals(fileSize, inputStats.getProcessedBytes()); + } + + @Test + public void testFetchFromPartiallyReadFile() throws IOException + { + final int fileSize = 200; + final File sourceFile = folder.newFile("testWithFileEntity"); + writeBytesToFile(sourceFile, fileSize); + + final int bufferSize = 50; + final byte[] intermediateBuffer = new byte[bufferSize]; + + // Read the file partially + final BytesCountingInputEntity inputEntity = new BytesCountingInputEntity(new FileEntity(sourceFile), inputStats); + inputEntity.open().read(intermediateBuffer); + Assert.assertEquals(bufferSize, inputStats.getProcessedBytes()); + + // Read the whole file again + inputEntity.fetch(folder.newFolder(), intermediateBuffer); + Assert.assertEquals(fileSize + bufferSize, inputStats.getProcessedBytes()); + } + + @Test + public void testFetchFromDirectory() throws IOException + { + final File sourceDir = folder.newFolder("testWithDirectory"); + + final int fileSize1 = 100; + final File sourceFile1 = new File(sourceDir, "file1"); + writeBytesToFile(sourceFile1, fileSize1); + + final int fileSize2 = 200; + final File sourceFile2 = new File(sourceDir, "file2"); + writeBytesToFile(sourceFile2, fileSize2); + + final BytesCountingInputEntity inputEntity = new BytesCountingInputEntity(new FileEntity(sourceDir), inputStats); + inputEntity.fetch(folder.newFolder(), new byte[1000]); + Assert.assertEquals(fileSize1 + fileSize2, inputStats.getProcessedBytes()); + } + + @Test + public void testOpen() throws IOException + { + final int entitySize = 100; + + final BytesCountingInputEntity inputEntity = new BytesCountingInputEntity( + new ByteEntity(new byte[entitySize]), + inputStats + ); + inputEntity.open().read(new byte[200]); + Assert.assertEquals(entitySize, inputStats.getProcessedBytes()); + } + + @Test + public void testOpenWithSmallBuffer() throws IOException + { + final int entitySize = 100; + final int bufferSize = 50; + + final BytesCountingInputEntity inputEntity = new BytesCountingInputEntity( + new ByteEntity(new byte[entitySize]), + inputStats + ); + inputEntity.open().read(new byte[bufferSize]); + Assert.assertEquals(bufferSize, inputStats.getProcessedBytes()); + } + + private void writeBytesToFile(File sourceFile, int numBytes) throws IOException + { + if (!sourceFile.exists()) { + sourceFile.createNewFile(); + } + + final OutputStreamWriter outputStreamWriter = new OutputStreamWriter( + new FileOutputStream(sourceFile), + StandardCharsets.UTF_8 + ); + char[] chars = new char[numBytes]; + Arrays.fill(chars, ' '); + outputStreamWriter.write(chars); + outputStreamWriter.flush(); + outputStreamWriter.close(); + } + +} diff --git a/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java index 96ea16c8665..41d191b36b3 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java @@ -24,9 +24,11 @@ import com.google.common.collect.Iterables; import org.apache.druid.data.input.ColumnsFilter; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputStats; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -42,7 +44,7 @@ import java.nio.file.Files; import java.util.ArrayList; import java.util.List; -public class InputEntityIteratingReaderTest +public class InputEntityIteratingReaderTest extends InitializedNullHandlingTest { @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -52,6 +54,7 @@ public class InputEntityIteratingReaderTest { final int numFiles = 5; final List files = new ArrayList<>(); + long totalFileSize = 0; for (int i = 0; i < numFiles; i++) { final File file = temporaryFolder.newFile("test_" + i); files.add(file); @@ -59,8 +62,10 @@ public class InputEntityIteratingReaderTest writer.write(StringUtils.format("%d,%s,%d\n", 20190101 + i, "name_" + i, i)); writer.write(StringUtils.format("%d,%s,%d", 20190102 + i, "name_" + (i + 1), i + 1)); } + totalFileSize += file.length(); } - final InputEntityIteratingReader firehose = new InputEntityIteratingReader( + + final InputEntityIteratingReader reader = new InputEntityIteratingReader( new InputRowSchema( new TimestampSpec("time", "yyyyMMdd", null), new DimensionsSpec( @@ -79,7 +84,8 @@ public class InputEntityIteratingReaderTest temporaryFolder.newFolder() ); - try (CloseableIterator iterator = firehose.read()) { + final InputStats inputStats = new InputStatsImpl(); + try (CloseableIterator iterator = reader.read(inputStats)) { int i = 0; while (iterator.hasNext()) { InputRow row = iterator.next(); @@ -95,6 +101,7 @@ public class InputEntityIteratingReaderTest i++; } Assert.assertEquals(numFiles, i); + Assert.assertEquals(totalFileSize, inputStats.getProcessedBytes()); } } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/InputStatsImpl.java b/core/src/test/java/org/apache/druid/data/input/impl/InputStatsImpl.java new file mode 100644 index 00000000000..6fb732682d8 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/InputStatsImpl.java @@ -0,0 +1,39 @@ +/* + * 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.data.input.impl; + +import org.apache.druid.data.input.InputStats; + +public class InputStatsImpl implements InputStats +{ + private long processedBytes; + + @Override + public void incrementProcessedBytes(long incrementByValue) + { + processedBytes += incrementByValue; + } + + @Override + public long getProcessedBytes() + { + return processedBytes; + } +} diff --git a/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java b/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java index acf66972a15..7c1320e6589 100644 --- a/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java +++ b/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java @@ -40,7 +40,6 @@ import javax.annotation.Nullable; import java.math.BigDecimal; import java.math.RoundingMode; import java.nio.ByteBuffer; -import java.util.Locale; import java.util.Set; public class FunctionTest extends InitializedNullHandlingTest @@ -525,7 +524,7 @@ public class FunctionTest extends InitializedNullHandlingTest ); for (Pair argAndType : invalidArguments) { try { - assertExpr(String.format(Locale.ENGLISH, "round(d, %s)", argAndType.lhs), null); + assertExpr(StringUtils.format("round(d, %s)", argAndType.lhs), null); Assert.fail("Did not throw IllegalArgumentException"); } catch (ExpressionValidationException e) { diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java index 817167df38a..0b749fb8198 100644 --- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java +++ b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java @@ -44,10 +44,12 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.data.input.impl.CloudObjectLocation; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputStatsImpl; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.initialization.DruidModule; @@ -481,7 +483,8 @@ public class OssInputSourceTest extends InitializedNullHandlingTest temporaryFolder.newFolder() ); - CloseableIterator iterator = reader.read(); + final InputStats inputStats = new InputStatsImpl(); + CloseableIterator iterator = reader.read(inputStats); while (iterator.hasNext()) { InputRow nextRow = iterator.next(); @@ -490,6 +493,7 @@ public class OssInputSourceTest extends InitializedNullHandlingTest Assert.assertEquals("world", nextRow.getDimension("dim2").get(0)); } + Assert.assertEquals(2 * CONTENT.length, inputStats.getProcessedBytes()); EasyMock.verify(OSSCLIENT); } @@ -525,7 +529,8 @@ public class OssInputSourceTest extends InitializedNullHandlingTest temporaryFolder.newFolder() ); - CloseableIterator iterator = reader.read(); + final InputStats inputStats = new InputStatsImpl(); + CloseableIterator iterator = reader.read(inputStats); while (iterator.hasNext()) { InputRow nextRow = iterator.next(); @@ -534,6 +539,7 @@ public class OssInputSourceTest extends InitializedNullHandlingTest Assert.assertEquals("world", nextRow.getDimension("dim2").get(0)); } + Assert.assertEquals(2 * CONTENT.length, inputStats.getProcessedBytes()); EasyMock.verify(OSSCLIENT); } diff --git a/extensions-core/google-extensions/src/test/java/org/apache/druid/data/input/google/GoogleCloudStorageInputSourceTest.java b/extensions-core/google-extensions/src/test/java/org/apache/druid/data/input/google/GoogleCloudStorageInputSourceTest.java index c0648e7f4c8..7916e0a2752 100644 --- a/extensions-core/google-extensions/src/test/java/org/apache/druid/data/input/google/GoogleCloudStorageInputSourceTest.java +++ b/extensions-core/google-extensions/src/test/java/org/apache/druid/data/input/google/GoogleCloudStorageInputSourceTest.java @@ -36,10 +36,12 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.data.input.impl.CloudObjectLocation; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputStatsImpl; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.initialization.DruidModule; @@ -295,7 +297,8 @@ public class GoogleCloudStorageInputSourceTest extends InitializedNullHandlingTe null ); - CloseableIterator iterator = reader.read(); + final InputStats inputStats = new InputStatsImpl(); + CloseableIterator iterator = reader.read(inputStats); while (iterator.hasNext()) { InputRow nextRow = iterator.next(); @@ -303,6 +306,7 @@ public class GoogleCloudStorageInputSourceTest extends InitializedNullHandlingTe Assert.assertEquals("hello", nextRow.getDimension("dim1").get(0)); Assert.assertEquals("world", nextRow.getDimension("dim2").get(0)); } + Assert.assertEquals(2 * CONTENT.length, inputStats.getProcessedBytes()); } @Test @@ -339,7 +343,8 @@ public class GoogleCloudStorageInputSourceTest extends InitializedNullHandlingTe null ); - CloseableIterator iterator = reader.read(); + final InputStats inputStats = new InputStatsImpl(); + CloseableIterator iterator = reader.read(inputStats); while (iterator.hasNext()) { InputRow nextRow = iterator.next(); @@ -347,6 +352,7 @@ public class GoogleCloudStorageInputSourceTest extends InitializedNullHandlingTe Assert.assertEquals("hello", nextRow.getDimension("dim1").get(0)); Assert.assertEquals("world", nextRow.getDimension("dim2").get(0)); } + Assert.assertEquals(2 * CONTENT.length, inputStats.getProcessedBytes()); } private static void addExpectedPrefixObjects(URI prefix, List uris) throws IOException diff --git a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/inputsource/hdfs/HdfsInputSourceTest.java b/extensions-core/hdfs-storage/src/test/java/org/apache/druid/inputsource/hdfs/HdfsInputSourceTest.java index 4322e199f71..e1f112ad008 100644 --- a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/inputsource/hdfs/HdfsInputSourceTest.java +++ b/extensions-core/hdfs-storage/src/test/java/org/apache/druid/inputsource/hdfs/HdfsInputSourceTest.java @@ -31,9 +31,11 @@ import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputStatsImpl; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.storage.hdfs.HdfsStorageDruidModule; @@ -59,6 +61,7 @@ import java.io.OutputStreamWriter; import java.io.UncheckedIOException; import java.io.Writer; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -259,11 +262,13 @@ public class HdfsInputSourceTest extends InitializedNullHandlingTest private HdfsInputSource target; private Set paths; private Map timestampToValue; + private List fileContents; @Before public void setup() throws IOException { timestampToValue = new HashMap<>(); + fileContents = new ArrayList<>(); File dir = temporaryFolder.getRoot(); Configuration configuration = new Configuration(true); @@ -276,11 +281,10 @@ public class HdfsInputSourceTest extends InitializedNullHandlingTest i -> { char value = ALPHABET.charAt(i % ALPHABET.length()); timestampToValue.put((long) i, Character.toString(value)); - return createFile( - fileSystem, - String.valueOf(i), - i + KEY_VALUE_SEPARATOR + value - ); + + final String contents = i + KEY_VALUE_SEPARATOR + value; + fileContents.add(contents); + return createFile(fileSystem, String.valueOf(i), contents); } ) .collect(Collectors.toSet()); @@ -319,9 +323,10 @@ public class HdfsInputSourceTest extends InitializedNullHandlingTest public void readsSplitsCorrectly() throws IOException { InputSourceReader reader = target.formattableReader(INPUT_ROW_SCHEMA, INPUT_FORMAT, null); + final InputStats inputStats = new InputStatsImpl(); Map actualTimestampToValue = new HashMap<>(); - try (CloseableIterator iterator = reader.read()) { + try (CloseableIterator iterator = reader.read(inputStats)) { while (iterator.hasNext()) { InputRow row = iterator.next(); actualTimestampToValue.put(row.getTimestampFromEpoch(), row.getDimension(COLUMN).get(0)); @@ -329,6 +334,9 @@ public class HdfsInputSourceTest extends InitializedNullHandlingTest } Assert.assertEquals(timestampToValue, actualTimestampToValue); + + long totalFileSize = fileContents.stream().mapToLong(String::length).sum(); + Assert.assertEquals(totalFileSize, inputStats.getProcessedBytes()); } @Test @@ -395,10 +403,12 @@ public class HdfsInputSourceTest extends InitializedNullHandlingTest public void readsSplitsCorrectly() throws IOException { InputSourceReader reader = target.formattableReader(INPUT_ROW_SCHEMA, INPUT_FORMAT, null); + final InputStats inputStats = new InputStatsImpl(); - try (CloseableIterator iterator = reader.read()) { + try (CloseableIterator iterator = reader.read(inputStats)) { Assert.assertFalse(iterator.hasNext()); } + Assert.assertEquals(0, inputStats.getProcessedBytes()); } @Test diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 3a9000c3d6e..54f4523e024 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -36,10 +36,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; import org.apache.curator.test.TestingCluster; -import org.apache.druid.client.cache.CacheConfig; -import org.apache.druid.client.cache.CachePopulatorStats; -import org.apache.druid.client.cache.MapCache; -import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.data.input.InputEntity; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputFormat; @@ -55,34 +51,19 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.kafka.KafkaRecordEntity; import org.apache.druid.data.input.kafkainput.KafkaInputFormat; import org.apache.druid.data.input.kafkainput.KafkaStringHeaderFormat; -import org.apache.druid.discovery.DataNodeService; -import org.apache.druid.discovery.DruidNodeAnnouncer; -import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.LockGranularity; -import org.apache.druid.indexing.common.SegmentCacheManagerFactory; -import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; -import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; -import org.apache.druid.indexing.common.actions.TaskActionClientFactory; -import org.apache.druid.indexing.common.actions.TaskActionToolbox; -import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; -import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.task.IndexTaskTest; +import org.apache.druid.indexing.common.task.ParseExceptionReport; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; -import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.kafka.test.TestBroker; import org.apache.druid.indexing.overlord.DataSourceMetadata; -import org.apache.druid.indexing.overlord.MetadataTaskStorage; -import org.apache.druid.indexing.overlord.TaskLockbox; -import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; @@ -90,8 +71,6 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTestBase; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; import org.apache.druid.indexing.seekablestream.SettableByteEntity; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; -import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; -import org.apache.druid.indexing.test.TestDataSegmentKiller; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; @@ -102,14 +81,9 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; 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.java.util.metrics.MonitorScheduler; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; -import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; -import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; -import org.apache.druid.query.DirectQueryProcessingPool; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; @@ -130,42 +104,27 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryEngine; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; -import org.apache.druid.segment.handoff.SegmentHandoffNotifier; -import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; +import org.apache.druid.segment.incremental.RowMeters; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.join.NoopJoinableFactory; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.loading.LocalDataSegmentPusher; -import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; -import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.DataSegment; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.header.Header; -import org.easymock.EasyMock; import org.joda.time.Period; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -173,20 +132,15 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; import java.util.stream.Stream; @SuppressWarnings("unchecked") @@ -254,7 +208,6 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase private String topic; private List> records; - private final Set checkpointRequestsHash = new HashSet<>(); private static List> generateRecords(String topic) { @@ -302,12 +255,6 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase return "topic" + topicPostfix++; } - @Rule - public final TemporaryFolder tempFolder = new TemporaryFolder(); - - @Rule - public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); - public KafkaIndexTaskTest(LockGranularity lockGranularity) { super(lockGranularity); @@ -375,7 +322,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase public static void tearDownClass() throws Exception { taskExec.shutdown(); - taskExec.awaitTermination(9999, TimeUnit.DAYS); + taskExec.awaitTermination(20, TimeUnit.MINUTES); kafkaServer.close(); kafkaServer = null; @@ -414,11 +361,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); Assert.assertTrue(task.getRunner().getFireDepartmentMetrics().isProcessingDone()); // Check published metadata and segments in deep storage @@ -472,6 +415,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); final Collection segments = publishedSegments(); for (DataSegment segment : segments) { @@ -555,11 +499,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); Assert.assertTrue(task.getRunner().getFireDepartmentMetrics().isProcessingDone()); // Check published metadata and segments in deep storage @@ -608,11 +548,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); Assert.assertTrue(task.getRunner().getFireDepartmentMetrics().isProcessingDone()); // Check published metadata and segments in deep storage @@ -671,9 +607,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase runner.resume(); // Check metrics - Assert.assertEquals(buildSegments.getProcessed(), task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(buildSegments.getUnparseable(), task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(buildSegments.getThrownAway(), task.getRunner().getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(buildSegments, task.getRunner().getRowIngestionMeters().getTotals()); Assert.assertEquals(avg_1min.get("processed"), 0.0); Assert.assertEquals(avg_5min.get("processed"), 0.0); @@ -751,10 +685,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase ) ); - // Check metrics - Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); + long totalBytes = getTotalSizeOfRecords(0, 10) + getTotalSizeOfRecords(13, 15); + verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).unparseable(3).thrownAway(1).totalProcessed(8)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -886,10 +818,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase ) ); - // Check metrics - Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); + long totalBytes = getTotalSizeOfRecords(0, 10) + getTotalSizeOfRecords(13, 15); + verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).unparseable(3).thrownAway(1).totalProcessed(8)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -984,10 +914,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase ) ); - // Check metrics - Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 2)).totalProcessed(2)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -1106,13 +1033,9 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertEquals(TaskState.SUCCESS, normalReplicaFuture.get().getStatusCode()); Assert.assertEquals(TaskState.SUCCESS, staleReplicaFuture.get().getStatusCode()); - Assert.assertEquals(9, normalReplica.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, normalReplica.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, normalReplica.getRunner().getRowIngestionMeters().getThrownAway()); - - Assert.assertEquals(9, staleReplica.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, staleReplica.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, staleReplica.getRunner().getRowIngestionMeters().getThrownAway()); + long totalBytes = getTotalSizeOfRecords(0, 9); + verifyTaskMetrics(normalReplica, RowMeters.with().bytes(totalBytes).totalProcessed(9)); + verifyTaskMetrics(staleReplica, RowMeters.with().bytes(totalBytes).totalProcessed(9)); } @Test(timeout = 60_000L) @@ -1147,11 +1070,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAway(2).totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -1199,11 +1118,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAway(2).totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -1260,11 +1175,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAway(4).totalProcessed(1)); // Check published metadata final List publishedDescriptors = publishedDescriptors(); @@ -1351,11 +1262,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 4)).totalProcessed(4)); } @Test(timeout = 60_000L) @@ -1416,7 +1323,6 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase List> scanResultValues = scanData(task, interval); Assert.assertEquals(3, Iterables.size(scanResultValues)); - int i = 0; for (Map event : scanResultValues) { Assert.assertEquals("application/json", event.get("kafka.testheader.encoding")); Assert.assertEquals("y", event.get("dim2")); @@ -1427,11 +1333,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 4)).totalProcessed(4)); } @Test(timeout = 60_000L) @@ -1461,11 +1363,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().totalProcessed(0)); // Check published metadata Assert.assertEquals(ImmutableList.of(), publishedDescriptors()); @@ -1500,11 +1398,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -1550,11 +1444,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -1605,11 +1495,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.FAILED, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 6)).unparseable(1).totalProcessed(3)); // Check published metadata Assert.assertEquals(ImmutableList.of(), publishedDescriptors()); @@ -1650,12 +1536,11 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); Assert.assertNull(status.getErrorMsg()); - - // Check metrics - Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessedWithError()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); + final long totalRecordBytes = getTotalSizeOfRecords(2, 13); + verifyTaskMetrics(task, RowMeters.with() + .bytes(totalRecordBytes) + .unparseable(3).errors(3) + .thrownAway(1).totalProcessed(4)); // Check published metadata assertEqualsExceptVersion( @@ -1673,6 +1558,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( RowIngestionMeters.PROCESSED, 4, + RowIngestionMeters.PROCESSED_BYTES, (int) totalRecordBytes, RowIngestionMeters.PROCESSED_WITH_ERROR, 3, RowIngestionMeters.UNPARSEABLE, 3, RowIngestionMeters.THROWN_AWAY, 1 @@ -1680,9 +1566,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase ); Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - List parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.BUILD_SEGMENTS); + ParseExceptionReport parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); List expectedMessages = Arrays.asList( "Unable to parse value[notanumber] for field[met1]", @@ -1692,10 +1577,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase "Unable to parse row [unparseable] (Record: 1)", "Encountered row with timestamp[246140482-04-24T15:36:27.903Z] that cannot be represented as a long: [{timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}] (Record: 1)" ); - List actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); List expectedInputs = Arrays.asList( "{timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=20.0, met1=notanumber}", @@ -1705,10 +1587,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase "unparseable", "{timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}" ); - List actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); + Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); } @Test(timeout = 60_000L) @@ -1746,11 +1625,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); IndexTaskTest.checkTaskStatusErrorMsgForParseExceptionsExceeded(status); - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessedWithError()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + final long totalBytes = getTotalSizeOfRecords(2, 8); + verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).unparseable(3).totalProcessed(3)); // Check published metadata Assert.assertEquals(ImmutableList.of(), publishedDescriptors()); @@ -1762,6 +1638,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( RowIngestionMeters.PROCESSED, 3, + RowIngestionMeters.PROCESSED_BYTES, (int) totalBytes, RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.UNPARSEABLE, 3, RowIngestionMeters.THROWN_AWAY, 0 @@ -1769,28 +1646,17 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase ); Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - - List parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.BUILD_SEGMENTS); + ParseExceptionReport parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); List expectedMessages = Arrays.asList( "Unable to parse [] as the intermediateRow resulted in empty input row (Record: 1)", "Unable to parse row [unparseable] (Record: 1)" ); - List actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); - List expectedInputs = Arrays.asList( - "", - "unparseable" - ); - List actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); + List expectedInputs = Arrays.asList("", "unparseable"); + Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); } @Test(timeout = 60_000L) @@ -1839,13 +1705,9 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); - // Check metrics - Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + final long totalBytes = getTotalSizeOfRecords(2, 5); + verifyTaskMetrics(task1, RowMeters.with().bytes(totalBytes).totalProcessed(3)); + verifyTaskMetrics(task2, RowMeters.with().bytes(totalBytes).totalProcessed(3)); // Check published segments & metadata assertEqualsExceptVersion( @@ -1908,13 +1770,9 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase final ListenableFuture future2 = runTask(task2); Assert.assertEquals(TaskState.FAILED, future2.get().getStatusCode()); - // Check metrics - Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); + verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSizeOfRecords(3, 10)) + .unparseable(3).thrownAway(1).totalProcessed(3)); // Check published segments & metadata, should all be from the first task final List publishedDescriptors = publishedDescriptors(); @@ -1984,13 +1842,9 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase final ListenableFuture future2 = runTask(task2); Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); - // Check metrics - Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); + verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSizeOfRecords(3, 10)) + .unparseable(3).thrownAway(1).totalProcessed(3)); // Check published segments & metadata SegmentDescriptorAndExpectedDim1Values desc3 = sdd("2011/P1D", 1, ImmutableList.of("d", "e")); @@ -2026,11 +1880,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for tasks to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(5, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + long totalBytes = getTotalSizeOfRecords(2, 5) + getTotalSizeOfRecords(13, 15); + verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).totalProcessed(5)); // Check published segments & metadata SegmentDescriptorAndExpectedDim1Values desc1 = sdd("2010/P1D", 0, ImmutableList.of("c")); @@ -2094,13 +1945,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); - // Check metrics - Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); + verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSizeOfRecords(13, 14)).totalProcessed(1)); // Check published segments & metadata assertEqualsExceptVersion( @@ -2142,19 +1988,11 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase final ListenableFuture future1 = runTask(task1); // Insert some data, but not enough for the task to finish - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.limit(records, 4)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } + insertData(Iterables.limit(records, 4)); while (countEvents(task1) != 2) { Thread.sleep(25); } - Assert.assertEquals(2, countEvents(task1)); // Stop without publishing segment @@ -2184,26 +2022,13 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase final ListenableFuture future2 = runTask(task2); // Insert remaining data - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.skip(records, 4)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } + insertData(Iterables.skip(records, 4)); // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); - // Check metrics - Assert.assertEquals(2, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSizeOfRecords(2, 4)).totalProcessed(2)); + verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSizeOfRecords(4, 5)).totalProcessed(1)); // Check published segments & metadata assertEqualsExceptVersion( @@ -2253,14 +2078,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase final ListenableFuture future1 = runTask(task1); // Insert some data, but not enough for the task to finish - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.limit(records, 5)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } + insertData(Iterables.limit(records, 5)); while (task1.getRunner().getStatus() != Status.PAUSED) { Thread.sleep(10); @@ -2298,25 +2116,13 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for the task to start reading // Insert remaining data - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.skip(records, 5)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } + insertData(Iterables.skip(records, 5)); // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); - // Check metrics - Assert.assertEquals(5, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(4, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).totalProcessed(5)); + verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSizeOfRecords(6, 10)).totalProcessed(4)); // Check published segments & metadata assertEqualsExceptVersion( @@ -2360,20 +2166,11 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase final ListenableFuture future = runTask(task); // Insert some data, but not enough for the task to finish - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.limit(records, 4)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.flush(); - kafkaProducer.commitTransaction(); - } + insertData(Iterables.limit(records, 4)); while (countEvents(task) != 2) { Thread.sleep(25); } - Assert.assertEquals(2, countEvents(task)); Assert.assertEquals(Status.READING, task.getRunner().getStatus()); @@ -2385,14 +2182,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase ); Assert.assertEquals(Status.PAUSED, task.getRunner().getStatus()); // Insert remaining data - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.skip(records, 4)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } + insertData(Iterables.skip(records, 4)); try { future.get(10, TimeUnit.SECONDS); @@ -2408,11 +2198,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); Assert.assertEquals(task.getRunner().getEndOffsets(), task.getRunner().getCurrentOffsets()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -2537,11 +2323,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -2622,14 +2404,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase final ListenableFuture future = runTask(task); // Insert 2 records initially - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.limit(records, 2)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } + insertData(records.subList(0, 2)); awaitConsumedOffsets(task, ImmutableMap.of(0, 1L)); // Consume two real messages Assert.assertEquals(2, countEvents(task)); @@ -2641,25 +2416,10 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertEquals(2, Iterables.size(scanResultValues)); // Insert 3 more records and rollback - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.skip(Iterables.limit(records, 5), 2)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.flush(); - kafkaProducer.abortTransaction(); - } + insertData(records.subList(2, 5), true); // Insert up through first 8 items - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.skip(Iterables.limit(records, 8), 5)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } + insertData(records.subList(5, 8)); awaitConsumedOffsets(task, ImmutableMap.of(0, 9L)); // Consume 8 real messages + 2 txn controls Assert.assertEquals(2, countEvents(task)); @@ -2674,24 +2434,15 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertEquals(2, Iterables.size(scanResultValues1)); // Insert all remaining messages. One will get picked up. - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.skip(records, 8)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } + insertData(Iterables.skip(records, 8)); // Wait for task to exit and publish Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); Assert.assertEquals(task.getRunner().getEndOffsets(), task.getRunner().getCurrentOffsets()); - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getProcessedWithError()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); + long totalBytes = getTotalSizeOfRecords(0, 2) + getTotalSizeOfRecords(5, 11); + verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes) + .unparseable(3).errors(1).thrownAway(1).totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -2825,11 +2576,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().totalProcessed(0)); // Check published metadata and segments in deep storage assertEqualsExceptVersion(Collections.emptyList(), publishedDescriptors()); @@ -2908,6 +2655,19 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase .toList(); } + private long getRecordSize(ProducerRecord record) + { + return (record.key() == null ? 0L : record.key().length) + + (record.value() == null ? 0L : record.value().length); + } + + private long getTotalSizeOfRecords(int startIndexInclusive, int endIndexExclusive) + { + return records.subList(startIndexInclusive, endIndexExclusive).stream() + .mapToLong(this::getRecordSize) + .sum(); + } + private void insertData() throws ExecutionException, InterruptedException { insertData(records); @@ -2915,6 +2675,12 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase private void insertData(Iterable> records) throws ExecutionException, InterruptedException + { + insertData(records, false); + } + + private void insertData(Iterable> records, boolean rollbackTransaction) + throws ExecutionException, InterruptedException { try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { kafkaProducer.initTransactions(); @@ -2922,7 +2688,11 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } - kafkaProducer.commitTransaction(); + if (rollbackTransaction) { + kafkaProducer.abortTransaction(); + } else { + kafkaProducer.commitTransaction(); + } } } @@ -3017,7 +2787,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase ); } - private QueryRunnerFactoryConglomerate makeTimeseriesAndScanConglomerate() + @Override + protected QueryRunnerFactoryConglomerate makeQueryRunnerConglomerate() { return new DefaultQueryRunnerFactoryConglomerate( ImmutableMap., QueryRunnerFactory>builder() @@ -3050,158 +2821,13 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase { directory = tempFolder.newFolder(); final TestUtils testUtils = new TestUtils(); - RowIngestionMetersFactory rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory(); final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); for (Module module : new KafkaIndexTaskModule().getJacksonModules()) { objectMapper.registerModule(module); } objectMapper.registerModule(TEST_MODULE); - - final TaskConfig taskConfig = new TaskConfig( - new File(directory, "baseDir").getPath(), - new File(directory, "baseTaskDir").getPath(), - null, - 50000, - null, - true, - null, - null, - null, - false, - false, - TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null, - false - ); - final TestDerbyConnector derbyConnector = derby.getConnector(); - derbyConnector.createDataSourceTable(); - derbyConnector.createPendingSegmentsTable(); - derbyConnector.createSegmentTable(); - derbyConnector.createRulesTable(); - derbyConnector.createConfigTable(); - derbyConnector.createTaskTables(); - derbyConnector.createAuditTable(); - taskStorage = new MetadataTaskStorage( - derbyConnector, - new TaskStorageConfig(null), - new DerbyMetadataStorageActionHandlerFactory( - derbyConnector, - derby.metadataTablesConfigSupplier().get(), - objectMapper - ) - ); - metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( - testUtils.getTestObjectMapper(), - derby.metadataTablesConfigSupplier().get(), - derbyConnector - ); - taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); - final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( - taskLockbox, - taskStorage, - metadataStorageCoordinator, - emitter, - new SupervisorManager(null) - { - @Override - public boolean checkPointDataSourceMetadata( - String supervisorId, - int taskGroupId, - @Nullable DataSourceMetadata previousDataSourceMetadata - ) - { - log.info("Adding checkpoint hash to the set"); - checkpointRequestsHash.add( - Objects.hash( - supervisorId, - taskGroupId, - previousDataSourceMetadata - ) - ); - return true; - } - }, - objectMapper - ); - final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( - taskStorage, - taskActionToolbox, - new TaskAuditLogConfig(false) - ); - final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier() - { - @Override - public boolean registerSegmentHandoffCallback( - SegmentDescriptor descriptor, - Executor exec, - Runnable handOffRunnable - ) - { - if (doHandoff) { - // Simulate immediate handoff - exec.execute(handOffRunnable); - } - return true; - } - - @Override - public void start() - { - //Noop - } - - @Override - public void close() - { - //Noop - } - }; - final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); - dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); - dataSegmentPusherConfig.zip = true; - final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig); - - toolboxFactory = new TaskToolboxFactory( - taskConfig, - null, // taskExecutorNode - taskActionClientFactory, - emitter, - dataSegmentPusher, - new TestDataSegmentKiller(), - null, // DataSegmentMover - null, // DataSegmentArchiver - new TestDataSegmentAnnouncer(), - EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), - handoffNotifierFactory, - this::makeTimeseriesAndScanConglomerate, - DirectQueryProcessingPool.INSTANCE, - NoopJoinableFactory.INSTANCE, - () -> EasyMock.createMock(MonitorScheduler.class), - new SegmentCacheManagerFactory(testUtils.getTestObjectMapper()), - testUtils.getTestObjectMapper(), - testUtils.getTestIndexIO(), - MapCache.create(1024), - new CacheConfig(), - new CachePopulatorStats(), - testUtils.getIndexMergerV9Factory(), - EasyMock.createNiceMock(DruidNodeAnnouncer.class), - EasyMock.createNiceMock(DruidNode.class), - new LookupNodeService("tier"), - new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), - new SingleFileTaskReportFileWriter(reportsFile), - null, - AuthTestUtils.TEST_AUTHORIZER_MAPPER, - new NoopChatHandlerProvider(), - testUtils.getRowIngestionMetersFactory(), - new TestAppenderatorsManager(), - new NoopOverlordClient(), - null, - null, - null, - null, - "1" - ); + makeToolboxFactory(testUtils, emitter, doHandoff); } @Test(timeout = 60_000L) @@ -3211,39 +2837,27 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase maxParseExceptions = 1000; maxSavedParseExceptions = 2; - // Insert data - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); + // Prepare records and insert data + //multiple objects in one Kafka record will yield 2 rows in druid + String wellformed = toJsonString(true, "2049", "d2", "y", "10", "22.0", "2.0") + + toJsonString(true, "2049", "d3", "y", "10", "23.0", "3.0"); - //multiple objects in one Kafka record will yield 2 rows in druid - String wellformed = toJsonString(true, "2049", "d2", "y", "10", "22.0", "2.0") + - toJsonString(true, "2049", "d3", "y", "10", "23.0", "3.0"); + //multiple objects in one Kafka record but some objects are in ill-formed format + //as a result, the whole ProducerRecord will be discarded + String malformed = + "{\"timestamp\":2049, \"dim1\": \"d4\", \"dim2\":\"x\", \"dimLong\": 10, \"dimFloat\":\"24.0\", \"met1\":\"2.0\" }" + + "{\"timestamp\":2049, \"dim1\": \"d5\", \"dim2\":\"y\", \"dimLong\": 10, \"dimFloat\":\"24.0\", \"met1\":invalidFormat }" + + "{\"timestamp\":2049, \"dim1\": \"d6\", \"dim2\":\"z\", \"dimLong\": 10, \"dimFloat\":\"24.0\", \"met1\":\"3.0\" }"; - //multiple objects in one Kafka record but some objects are in ill-formed format - //as a result, the whole ProducerRecord will be discarded - String malformed = - "{\"timestamp\":2049, \"dim1\": \"d4\", \"dim2\":\"x\", \"dimLong\": 10, \"dimFloat\":\"24.0\", \"met1\":\"2.0\" }" - + - "{\"timestamp\":2049, \"dim1\": \"d5\", \"dim2\":\"y\", \"dimLong\": 10, \"dimFloat\":\"24.0\", \"met1\":invalidFormat }" - + - "{\"timestamp\":2049, \"dim1\": \"d6\", \"dim2\":\"z\", \"dimLong\": 10, \"dimFloat\":\"24.0\", \"met1\":\"3.0\" }"; - - ProducerRecord[] producerRecords = new ProducerRecord[]{ - // pretty formatted - new ProducerRecord<>(topic, 0, null, jbb(true, "2049", "d1", "y", "10", "20.0", "1.0")), - //well-formed - new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8(wellformed)), - //ill-formed - new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8(malformed)), - //a well-formed record after ill-formed to demonstrate that the ill-formed can be successfully skipped - new ProducerRecord<>(topic, 0, null, jbb(true, "2049", "d7", "y", "10", "20.0", "1.0")) - }; - for (ProducerRecord record : producerRecords) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } + List> producerRecords = Arrays.asList( + new ProducerRecord<>(topic, 0, null, jbb(true, "2049", "d1", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8(wellformed)), + new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8(malformed)), + //a well-formed record after ill-formed to demonstrate that the ill-formed can be successfully skipped + new ProducerRecord<>(topic, 0, null, jbb(true, "2049", "d7", "y", "10", "20.0", "1.0")) + ); + records = producerRecords; + insertData(); final KafkaIndexTask task = createTask( null, @@ -3266,12 +2880,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - // 4 records processed, 3 success, 1 failed - Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 4)).unparseable(1).totalProcessed(4)); // Check published metadata assertEqualsExceptVersion( @@ -3313,7 +2922,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase @Override public CloseableIterator read() throws IOException { - KafkaRecordEntity recordEntity = (KafkaRecordEntity) settableByteEntity.getEntity(); + KafkaRecordEntity recordEntity = settableByteEntity.getEntity(); return delegate.read().map( r -> { MapBasedInputRow row = (MapBasedInputRow) r; diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 99f939433f7..e8a7383b778 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -35,43 +35,24 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.name.Named; -import org.apache.druid.client.cache.CacheConfig; -import org.apache.druid.client.cache.CachePopulatorStats; -import org.apache.druid.client.cache.MapCache; -import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.discovery.DataNodeService; -import org.apache.druid.discovery.DruidNodeAnnouncer; -import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.LockGranularity; -import org.apache.druid.indexing.common.SegmentCacheManagerFactory; -import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; -import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; -import org.apache.druid.indexing.common.actions.TaskActionClientFactory; -import org.apache.druid.indexing.common.actions.TaskActionToolbox; -import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; -import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.task.IndexTaskTest; +import org.apache.druid.indexing.common.task.ParseExceptionReport; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; -import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisor; import org.apache.druid.indexing.overlord.DataSourceMetadata; -import org.apache.druid.indexing.overlord.MetadataTaskStorage; -import org.apache.druid.indexing.overlord.TaskLockbox; -import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTestBase; @@ -80,21 +61,14 @@ import org.apache.druid.indexing.seekablestream.SequenceMetadata; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; -import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; -import org.apache.druid.indexing.test.TestDataSegmentKiller; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; 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.java.util.metrics.MonitorScheduler; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; -import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; -import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; -import org.apache.druid.query.DirectQueryProcessingPool; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.filter.SelectorDimFilter; @@ -103,21 +77,11 @@ import org.apache.druid.query.timeseries.TimeseriesQueryEngine; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.handoff.SegmentHandoffNotifier; -import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.incremental.RowMeters; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.join.NoopJoinableFactory; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.loading.LocalDataSegmentPusher; -import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; -import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.joda.time.Period; @@ -126,12 +90,11 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.File; import java.io.IOException; @@ -139,19 +102,16 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.Executor; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.Predicate; import java.util.stream.Collectors; @@ -163,8 +123,44 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase private static final String STREAM = "stream"; private static final String SHARD_ID1 = "1"; private static final String SHARD_ID0 = "0"; - private static KinesisRecordSupplier recordSupplier; + private static final List RECORDS = Arrays.asList( + createRecord("1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")), + createRecord("1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")), + createRecord("1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")), + createRecord("1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")), + createRecord("1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")), + createRecord("1", "5", jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")), + createRecord("1", "6", new ByteEntity(StringUtils.toUtf8("unparseable"))), + createRecord("1", "7", new ByteEntity(StringUtils.toUtf8(""))), + createRecord("1", "8", new ByteEntity(StringUtils.toUtf8("{}"))), + createRecord("1", "9", jb("2013", "f", "y", "10", "20.0", "1.0")), + createRecord("1", "10", jb("2049", "f", "y", "notanumber", "20.0", "1.0")), + createRecord("1", "11", jb("2049", "f", "y", "10", "notanumber", "1.0")), + createRecord("1", "12", jb("2049", "f", "y", "10", "20.0", "notanumber")), + createRecord("0", "0", jb("2012", "g", "y", "10", "20.0", "1.0")), + createRecord("0", "1", jb("2011", "h", "y", "10", "20.0", "1.0")) + ); + + private static final List SINGLE_PARTITION_RECORDS = Arrays.asList( + createRecord("1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")), + createRecord("1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")), + createRecord("1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")), + createRecord("1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")), + createRecord("1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")), + createRecord("1", "5", jb("2012", "a", "y", "10", "20.0", "1.0")), + createRecord("1", "6", jb("2013", "b", "y", "10", "20.0", "1.0")), + createRecord("1", "7", jb("2010", "c", "y", "10", "20.0", "1.0")), + createRecord("1", "8", jb("2011", "d", "y", "10", "20.0", "1.0")), + createRecord("1", "9", jb("2011", "e", "y", "10", "20.0", "1.0")), + createRecord("1", "10", jb("2008", "a", "y", "10", "20.0", "1.0")), + createRecord("1", "11", jb("2009", "b", "y", "10", "20.0", "1.0")), + createRecord("1", "12", jb("2010", "c", "y", "10", "20.0", "1.0")), + createRecord("1", "13", jb("2012", "d", "y", "10", "20.0", "1.0")), + createRecord("1", "14", jb("2013", "e", "y", "10", "20.0", "1.0")) + ); + + private static KinesisRecordSupplier recordSupplier; private static ServiceEmitter emitter; @Parameterized.Parameters(name = "{0}") @@ -187,14 +183,6 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase private final Period intermediateHandoffPeriod = null; private int maxRecordsPerPoll; - private final Set checkpointRequestsHash = new HashSet<>(); - - @Rule - public final TemporaryFolder tempFolder = new TemporaryFolder(); - - @Rule - public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); - @BeforeClass public static void setupClass() { @@ -254,82 +242,59 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase public static void tearDownClass() throws Exception { taskExec.shutdown(); - taskExec.awaitTermination(9999, TimeUnit.DAYS); + taskExec.awaitTermination(20, TimeUnit.MINUTES); emitter.close(); } - // records can only be read once, hence we generate fresh records every time - private static List> generateRecords(int start) + private void waitUntil(KinesisIndexTask task, Predicate predicate) + throws InterruptedException { - final List> records = generateRecords(STREAM); - return records.subList(start, records.size()); + while (!predicate.test(task)) { + Thread.sleep(10); + } } - private static List> generateRecords(int start, int end) + private long getTotalSize(List records, int startIndexInclusive, int endIndexExclusive) { - return generateRecords(STREAM).subList(start, end); + return records.subList(startIndexInclusive, endIndexExclusive) + .stream().flatMap(record -> record.getData().stream()) + .mapToLong(entity -> entity.getBuffer().remaining()).sum(); } - private List> generateSinglePartitionRecords(int start, int end) + private static KinesisRecord clone(KinesisRecord record) { - return generateSinglePartitionRecords(STREAM).subList(start, end); - } - - private static List> generateRecords(String stream) - { - return ImmutableList.of( - new OrderedPartitionableRecord<>(stream, "1", "0", jbl("2008", "a", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "1", jbl("2009", "b", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "2", jbl("2010", "c", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "3", jbl("2011", "d", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "4", jbl("2011", "e", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>( - stream, - "1", - "5", - jbl("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0") - ), - new OrderedPartitionableRecord<>( - stream, - "1", - "6", - Collections.singletonList(new ByteEntity(StringUtils.toUtf8("unparseable"))) - ), - new OrderedPartitionableRecord<>( - stream, - "1", - "7", - Collections.singletonList(new ByteEntity(StringUtils.toUtf8(""))) - ), - new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(new ByteEntity(StringUtils.toUtf8("{}")))), - new OrderedPartitionableRecord<>(stream, "1", "9", jbl("2013", "f", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "10", jbl("2049", "f", "y", "notanumber", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "11", jbl("2049", "f", "y", "10", "notanumber", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "12", jbl("2049", "f", "y", "10", "20.0", "notanumber")), - new OrderedPartitionableRecord<>(stream, "0", "0", jbl("2012", "g", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "0", "1", jbl("2011", "h", "y", "10", "20.0", "1.0")) + return new KinesisRecord( + record.getStream(), + record.getPartitionId(), + record.getSequenceNumber(), + record.getData().stream() + .map(entity -> new ByteEntity(entity.getBuffer())) + .collect(Collectors.toList()) ); } - private static List> generateSinglePartitionRecords(String stream) + private static List> clone( + List records, + int start, + int end + ) { - return ImmutableList.of( - new OrderedPartitionableRecord<>(stream, "1", "0", jbl("2008", "a", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "1", jbl("2009", "b", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "2", jbl("2010", "c", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "3", jbl("2011", "d", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "4", jbl("2011", "e", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "5", jbl("2012", "a", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "6", jbl("2013", "b", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "7", jbl("2010", "c", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "8", jbl("2011", "d", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "9", jbl("2011", "e", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "10", jbl("2008", "a", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "11", jbl("2009", "b", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "12", jbl("2010", "c", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "13", jbl("2012", "d", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "14", jbl("2013", "e", "y", "10", "20.0", "1.0")) - ); + return clone(records).subList(start, end); + } + + /** + * Records can only be read once, hence we must use fresh records every time. + */ + private static List> clone( + List records + ) + { + return records.stream().map(KinesisIndexTaskTest::clone).collect(Collectors.toList()); + } + + private static KinesisRecord createRecord(String partitionId, String sequenceNumber, ByteEntity entity) + { + return new KinesisRecord(STREAM, partitionId, sequenceNumber, Collections.singletonList(entity)); } @Test(timeout = 120_000L) @@ -343,7 +308,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 5)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 2, 5)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -351,23 +317,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "4") ); Assert.assertTrue(task.supportsQueries()); @@ -378,10 +330,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 5)) + .totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -410,7 +360,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 5)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 2, 5)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -428,24 +379,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase ) ); final KinesisIndexTask task = createTask( - null, NEW_DATA_SCHEMA.withDimensionsSpec(dimensionsSpec), - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "4") ); final ListenableFuture future = runTask(task); @@ -473,7 +409,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 5)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 2, 5)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -481,7 +418,6 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, NEW_DATA_SCHEMA.withDimensionsSpec( new DimensionsSpec( ImmutableList.of( @@ -494,22 +430,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase ) ) ), - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "4") ); task.addToContext(Tasks.STORE_EMPTY_COLUMNS_KEY, false); final ListenableFuture future = runTask(task); @@ -536,7 +458,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 5)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 2, 5)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -544,24 +467,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, OLD_DATA_SCHEMA, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), - true, - null, - null, - null, - "awsEndpoint", - null, - null, - null, - null, - false - ) + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "4") ); final ListenableFuture future = runTask(task); @@ -570,11 +478,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); verifyAll(); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 5)) + .totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -609,10 +514,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(Collections.emptyList()) - .times(5) - .andReturn(generateRecords(13, 15)) - .once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(Collections.emptyList()).times(5) + .andReturn(clone(RECORDS, 13, 15)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -620,24 +524,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID0, "0"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID0, "1")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) - + 0, + ImmutableMap.of(SHARD_ID0, "0"), + ImmutableMap.of(SHARD_ID0, "1") ); final ListenableFuture future = runTask(task); @@ -646,10 +535,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); verifyAll(); - // Check metrics - Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 13, 15)) + .totalProcessed(2)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -670,7 +557,6 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase @Test(timeout = 120_000L) public void testIncrementalHandOff() throws Exception { - final String baseSequenceName = "sequence0"; // as soon as any segment has more than one record, incremental publishing should happen maxRowsPerSegment = 2; maxRecordsPerPoll = 1; @@ -683,10 +569,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(0, 5)) - .once() - .andReturn(generateRecords(4)) - .once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 0, 5)).once() + .andReturn(clone(RECORDS, 4, 15)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -709,28 +594,13 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase ImmutableMap.of(SHARD_ID1, "9", SHARD_ID0, "1") ); final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - baseSequenceName, - startPartitions, - endPartitions, - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + startPartitions.getPartitionSequenceNumberMap(), + endPartitions.getPartitionSequenceNumberMap() ); final ListenableFuture future = runTask(task); - while (task.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.PAUSED) { - Thread.sleep(10); - } + waitUntil(task, this::isTaskPaused); + final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); task.getRunner().setEndOffsets(currentOffsets, false); @@ -750,10 +620,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase ) ); - // Check metrics - Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + final long totalRecordBytes = getTotalSize(RECORDS, 0, 15) - getTotalSize(RECORDS, 10, 13); + verifyTaskMetrics(task, RowMeters.with().bytes(totalRecordBytes).unparseable(4).totalProcessed(8)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -782,7 +650,6 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase @Test(timeout = 120_000L) public void testIncrementalHandOffMaxTotalRows() throws Exception { - final String baseSequenceName = "sequence0"; // incremental publish should happen every 3 records maxRowsPerSegment = Integer.MAX_VALUE; maxTotalRows = 3L; @@ -795,74 +662,38 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(0, 3)) - .once() - .andReturn(generateRecords(2, 10)) - .once() - .andReturn(generateRecords(9, 11)); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 0, 3)).once() + .andReturn(clone(RECORDS, 2, 10)).once() + .andReturn(clone(RECORDS, 9, 11)); recordSupplier.close(); EasyMock.expectLastCall().once(); replayAll(); - // Insert data - final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( - STREAM, - ImmutableMap.of(SHARD_ID1, "0"), - ImmutableSet.of() - ); + final Map startOffsets = ImmutableMap.of(SHARD_ID1, "0"); // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering // of events fetched across two partitions from Kafka - final SeekableStreamEndSequenceNumbers checkpoint1 = new SeekableStreamEndSequenceNumbers<>( - STREAM, - ImmutableMap.of(SHARD_ID1, "2") - ); - final SeekableStreamEndSequenceNumbers checkpoint2 = new SeekableStreamEndSequenceNumbers<>( - STREAM, - ImmutableMap.of(SHARD_ID1, "9") - ); - final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( - STREAM, - ImmutableMap.of(SHARD_ID1, "10") - ); + final Map checkpointOffsets1 = ImmutableMap.of(SHARD_ID1, "2"); + final Map checkpointOffsets2 = ImmutableMap.of(SHARD_ID1, "9"); + final Map endOffsets = ImmutableMap.of(SHARD_ID1, "10"); - final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - baseSequenceName, - startPartitions, - endPartitions, - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) - ); + final KinesisIndexTask task = createTask(0, startOffsets, endOffsets); final ListenableFuture future = runTask(task); - while (task.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.PAUSED) { - Thread.sleep(10); - } + waitUntil(task, this::isTaskPaused); + final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); + Assert.assertEquals(checkpointOffsets1, currentOffsets); task.getRunner().setEndOffsets(currentOffsets, false); - while (task.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.PAUSED) { - Thread.sleep(10); - } + waitUntil(task, this::isTaskPaused); final Map nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), nextOffsets); + Assert.assertEquals(checkpointOffsets2, nextOffsets); task.getRunner().setEndOffsets(nextOffsets, false); @@ -876,7 +707,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase Objects.hash( NEW_DATA_SCHEMA.getDataSource(), 0, - new KinesisDataSourceMetadata(startPartitions) + new KinesisDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>(STREAM, startOffsets, Collections.emptySet()) + ) ) ) ); @@ -891,10 +724,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase ) ); - // Check metrics - Assert.assertEquals(6, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 0, 11)) + .errors(1).unparseable(4).totalProcessed(6)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -909,7 +740,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase publishedDescriptors() ); Assert.assertEquals( - new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "10"))), + new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, endOffsets)), newDataSchemaMetadata() ); } @@ -926,7 +757,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(0, 13)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(clone(RECORDS, 0, 13)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -934,7 +765,6 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, new KinesisIndexTaskIOConfig( 0, "sequence0", @@ -955,20 +785,15 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase final ListenableFuture future = runTask(task); - // Wait for the task to start reading - while (task.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.READING) { - Thread.sleep(10); - } + waitUntil(task, this::isTaskReading); // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); verifyAll(); - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 0, 5)) + .thrownAway(2).totalProcessed(3)); // Check published metadata assertEqualsExceptVersion( @@ -996,7 +821,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(0, 13)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(clone(RECORDS, 0, 13)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -1004,7 +829,6 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, new KinesisIndexTaskIOConfig( 0, "sequence0", @@ -1025,20 +849,15 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase final ListenableFuture future = runTask(task); - // Wait for the task to start reading - while (task.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.READING) { - Thread.sleep(10); - } + waitUntil(task, this::isTaskReading); // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); verifyAll(); - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 0, 5)) + .thrownAway(2).totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -1068,7 +887,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(0, 13)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(clone(RECORDS, 0, 13)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -1076,7 +895,6 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, NEW_DATA_SCHEMA.withTransformSpec( new TransformSpec( new SelectorDimFilter("dim1", "b", null), @@ -1085,40 +903,20 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase ) ) ), - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "0"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + ImmutableMap.of(SHARD_ID1, "0"), + ImmutableMap.of(SHARD_ID1, "4") ); final ListenableFuture future = runTask(task); - - // Wait for the task to start reading - while (task.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.READING) { - Thread.sleep(10); - } + waitUntil(task, this::isTaskReading); // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); verifyAll(); - // Check metrics - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 0, 5)) + .thrownAway(4).totalProcessed(1)); // Check published metadata assertEqualsExceptVersion(ImmutableList.of(sdd("2009/P1D", 0)), publishedDescriptors()); @@ -1146,7 +944,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 3)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(clone(RECORDS, 2, 3)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -1156,23 +954,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase // When start and end offsets are the same, it means we need to read one message (since in Kinesis, end offsets // are inclusive). final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "2") ); final ListenableFuture future = runTask(task); @@ -1182,10 +966,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); - // Check metrics - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 3)) + .totalProcessed(1)); // Check published metadata assertEqualsExceptVersion(ImmutableList.of(sdd("2010/P1D", 0)), publishedDescriptors()); @@ -1205,7 +987,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 13)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(clone(RECORDS, 2, 13)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -1213,23 +995,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "4") ); final ListenableFuture future = runTask(task); @@ -1239,10 +1007,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 5)) + .totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -1275,7 +1041,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 13)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(clone(RECORDS, 2, 13)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -1283,23 +1049,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "4") ); final ListenableFuture future = runTask(task); @@ -1309,10 +1061,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 5)) + .totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -1348,7 +1098,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 13)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(clone(RECORDS, 2, 13)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -1356,23 +1106,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "5")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "5") ); final ListenableFuture future = runTask(task); @@ -1382,10 +1118,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 6)) + .unparseable(1).totalProcessed(3)); // Check published metadata Assert.assertEquals(ImmutableList.of(), publishedDescriptors()); @@ -1408,7 +1142,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 13)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(clone(RECORDS, 2, 13)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -1416,23 +1150,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "12")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "12") ); final ListenableFuture future = runTask(task); @@ -1446,10 +1166,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertNull(status.getErrorMsg()); - // Check metrics - Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessedWithError()); - Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 13)) + .errors(3).unparseable(4).totalProcessed(4)); // Check published metadata assertEqualsExceptVersion( @@ -1469,6 +1187,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( RowIngestionMeters.PROCESSED, 4, + RowIngestionMeters.PROCESSED_BYTES, 763, RowIngestionMeters.PROCESSED_WITH_ERROR, 3, RowIngestionMeters.UNPARSEABLE, 4, RowIngestionMeters.THROWN_AWAY, 0 @@ -1476,9 +1195,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase ); Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - List parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.BUILD_SEGMENTS); + ParseExceptionReport parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); List expectedMessages = Arrays.asList( "Unable to parse value[notanumber] for field[met1]", @@ -1489,10 +1207,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase "Unable to parse row [unparseable] (Record: 1)", "Encountered row with timestamp[246140482-04-24T15:36:27.903Z] that cannot be represented as a long: [{timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}] (Record: 1)" ); - List actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); List expectedInputs = Arrays.asList( "{timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=20.0, met1=notanumber}", @@ -1503,10 +1218,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase "unparseable", "{timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}" ); - List actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); + Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); } @@ -1525,7 +1237,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 13)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 2, 13)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -1533,23 +1246,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "9")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "9") ); final ListenableFuture future = runTask(task); @@ -1561,11 +1260,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); IndexTaskTest.checkTaskStatusErrorMsgForParseExceptionsExceeded(status); - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessedWithError()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + long totalBytes = getTotalSize(RECORDS, 2, 8); + verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).unparseable(3).totalProcessed(3)); // Check published metadata Assert.assertEquals(ImmutableList.of(), publishedDescriptors()); @@ -1577,6 +1273,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( RowIngestionMeters.PROCESSED, 3, + RowIngestionMeters.PROCESSED_BYTES, (int) totalBytes, RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.UNPARSEABLE, 3, RowIngestionMeters.THROWN_AWAY, 0 @@ -1584,28 +1281,15 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase ); Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - - List parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.BUILD_SEGMENTS); + ParseExceptionReport parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); List expectedMessages = Arrays.asList( "Unable to parse [] as the intermediateRow resulted in empty input row (Record: 1)", "Unable to parse row [unparseable] (Record: 1)" ); - List actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); - - List expectedInputs = Arrays.asList( - "", - "unparseable" - ); - List actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); + Assert.assertEquals(Arrays.asList("", "unparseable"), parseExceptionReport.getInputs()); } @@ -1622,8 +1306,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase EasyMock.expectLastCall().anyTimes(); EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) - .andReturn(generateRecords(2, 13)).once() - .andReturn(generateRecords(2, 13)).once(); + .andReturn(clone(RECORDS, 2, 13)).once() + .andReturn(clone(RECORDS, 2, 13)).once(); recordSupplier.close(); EasyMock.expectLastCall().times(2); @@ -1631,42 +1315,14 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task1 = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "4") ); final KinesisIndexTask task2 = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "4") ); final ListenableFuture future1 = runTask(task1); @@ -1678,13 +1334,10 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); - // Check metrics - Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 5)) + .totalProcessed(3)); + verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 5)) + .totalProcessed(3)); // Check published segments & metadata assertEqualsExceptVersion( @@ -1714,9 +1367,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 13)) + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(clone(RECORDS, 2, 13)) .once() - .andReturn(generateRecords(3, 13)) + .andReturn(clone(RECORDS, 3, 13)) .once(); recordSupplier.close(); @@ -1725,42 +1378,14 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task1 = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "4") ); final KinesisIndexTask task2 = createTask( - null, - new KinesisIndexTaskIOConfig( - 1, - "sequence1", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "3"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "9")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 1, + ImmutableMap.of(SHARD_ID1, "3"), + ImmutableMap.of(SHARD_ID1, "9") ); // Run first task @@ -1772,13 +1397,10 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertEquals(TaskState.FAILED, future2.get().getStatusCode()); verifyAll(); - // Check metrics - Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(4, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 5)) + .totalProcessed(3)); + verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSize(RECORDS, 3, 10)) + .unparseable(4).totalProcessed(3)); // Check published segments & metadata, should all be from the first task assertEqualsExceptVersion( @@ -1806,9 +1428,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 13)) + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(clone(RECORDS, 2, 13)) .once() - .andReturn(generateRecords(3, 13)) + .andReturn(clone(RECORDS, 3, 13)) .once(); recordSupplier.close(); @@ -1817,42 +1439,20 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task1 = createTask( + 0, null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), - false, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + NEW_DATA_SCHEMA, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "4"), + false ); final KinesisIndexTask task2 = createTask( + 1, null, - new KinesisIndexTaskIOConfig( - 1, - "sequence1", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "3"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "9")), - false, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + NEW_DATA_SCHEMA, + ImmutableMap.of(SHARD_ID1, "3"), + ImmutableMap.of(SHARD_ID1, "9"), + false ); // Run first task @@ -1871,13 +1471,10 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); - // Check metrics - Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(4, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 5)) + .totalProcessed(3)); + verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSize(RECORDS, 3, 10)) + .unparseable(4).totalProcessed(3)); // Check published segments & metadata SegmentDescriptorAndExpectedDim1Values desc3 = sdd("2011/P1D", 1, ImmutableList.of("d", "e")); @@ -1899,7 +1496,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 2, 15)).once(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -1907,44 +1505,22 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence1", - new SeekableStreamStartSequenceNumbers<>( - STREAM, - ImmutableMap.of(SHARD_ID1, "2", SHARD_ID0, "0"), - ImmutableSet.of() - ), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4", SHARD_ID0, "1")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + ImmutableMap.of(SHARD_ID1, "2", SHARD_ID0, "0"), + ImmutableMap.of(SHARD_ID1, "4", SHARD_ID0, "1") ); final ListenableFuture future = runTask(task); - while (countEvents(task) < 5) { - Thread.sleep(10); - } + waitUntil(task, t -> countEvents(task) >= 5); // Wait for tasks to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); verifyAll(); - // Check metrics - Assert.assertEquals(5, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + long totalBytes = getTotalSize(RECORDS, 2, 5) + getTotalSize(RECORDS, 13, 15); + verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).totalProcessed(5)); // Check published segments & metadata assertEqualsExceptVersion( @@ -1975,10 +1551,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 13)) - .once() - .andReturn(generateRecords(13, 15)) - .once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 2, 13)).once() + .andReturn(clone(RECORDS, 13, 15)).once(); recordSupplier.close(); EasyMock.expectLastCall().times(2); @@ -1986,42 +1561,14 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task1 = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 0, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "4") ); final KinesisIndexTask task2 = createTask( - null, - new KinesisIndexTaskIOConfig( - 1, - "sequence1", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID0, "0"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID0, "1")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + 1, + ImmutableMap.of(SHARD_ID0, "0"), + ImmutableMap.of(SHARD_ID0, "1") ); final ListenableFuture future1 = runTask(task1); @@ -2032,13 +1579,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); - // Check metrics - Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(2, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 5)).totalProcessed(3)); + verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSize(RECORDS, 13, 15)).totalProcessed(2)); // Check published segments & metadata assertEqualsExceptVersion( @@ -2068,10 +1610,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase EasyMock.expectLastCall(); recordSupplier.seek(streamPartition, "2"); EasyMock.expectLastCall(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 4)) - .once() - .andReturn(Collections.emptyList()) - .anyTimes(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 2, 4)).once() + .andReturn(Collections.emptyList()).anyTimes(); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -2079,31 +1620,15 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task1 = createTask( + 0, "task1", - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "5")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "5") ); final ListenableFuture future1 = runTask(task1); - while (countEvents(task1) != 2) { - Thread.sleep(25); - } - + waitUntil(task1, t -> countEvents(t) == 2); Assert.assertEquals(2, countEvents(task1)); // Stop without publishing segment @@ -2119,7 +1644,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase EasyMock.expectLastCall(); recordSupplier.seek(streamPartition, "3"); EasyMock.expectLastCall(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(3, 6)).once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 3, 6)).once(); recordSupplier.assign(ImmutableSet.of()); EasyMock.expectLastCall(); recordSupplier.close(); @@ -2129,31 +1655,14 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase // Start a new task final KinesisIndexTask task2 = createTask( + 0, task1.getId(), - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "5")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "5") ); final ListenableFuture future2 = runTask(task2); - - while (countEvents(task2) < 3) { - Thread.sleep(25); - } - + waitUntil(task2, t -> countEvents(t) >= 3); Assert.assertEquals(3, countEvents(task2)); // Wait for task to exit @@ -2161,13 +1670,10 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); - // Check metrics - Assert.assertEquals(2, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 4)) + .totalProcessed(2)); + verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSize(RECORDS, 4, 6)) + .unparseable(1).totalProcessed(1)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -2189,7 +1695,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase { maxRowsPerSegment = 2; maxRecordsPerPoll = 1; - List> records = generateSinglePartitionRecords(STREAM); + List> records = + clone(SINGLE_PARTITION_RECORDS); recordSupplier.assign(EasyMock.anyObject()); EasyMock.expectLastCall().anyTimes(); @@ -2201,18 +1708,13 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase EasyMock.expectLastCall().once(); // simulate 1 record at a time - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(Collections.singletonList(records.get(0))) - .once() - .andReturn(Collections.singletonList(records.get(1))) - .once() - .andReturn(Collections.singletonList(records.get(2))) - .once() - .andReturn(Collections.singletonList(records.get(3))) - .once() - .andReturn(Collections.singletonList(records.get(4))) - .once() - .andReturn(Collections.emptyList()) - .anyTimes(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(Collections.singletonList(records.get(0))).once() + .andReturn(Collections.singletonList(records.get(1))).once() + .andReturn(Collections.singletonList(records.get(2))).once() + .andReturn(Collections.singletonList(records.get(3))).once() + .andReturn(Collections.singletonList(records.get(4))).once() + .andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(recordSupplier.getPartitionsTimeLag(EasyMock.anyString(), EasyMock.anyObject())) .andReturn(null) @@ -2222,37 +1724,19 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase final KinesisIndexTask task1 = createTask( + 0, "task1", - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "0"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "6")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) - ); - - final SeekableStreamEndSequenceNumbers checkpoint1 = new SeekableStreamEndSequenceNumbers<>( - STREAM, - ImmutableMap.of(SHARD_ID1, "4") + ImmutableMap.of(SHARD_ID1, "0"), + ImmutableMap.of(SHARD_ID1, "6") ); + final Map checkpointOffsets1 = ImmutableMap.of(SHARD_ID1, "4"); final ListenableFuture future1 = runTask(task1); - while (task1.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.PAUSED) { - Thread.sleep(10); - } + waitUntil(task1, this::isTaskPaused); + final Map currentOffsets = ImmutableMap.copyOf(task1.getRunner().getCurrentOffsets()); - Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); + Assert.assertEquals(checkpointOffsets1, currentOffsets); task1.getRunner().setEndOffsets(currentOffsets, false); // Stop without publishing segment @@ -2270,12 +1754,10 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(Collections.singletonList(records.get(5))) - .once() - .andReturn(Collections.singletonList(records.get(6))) - .once() - .andReturn(Collections.emptyList()) - .anyTimes(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(Collections.singletonList(records.get(5))).once() + .andReturn(Collections.singletonList(records.get(6))).once() + .andReturn(Collections.emptyList()).anyTimes(); recordSupplier.close(); EasyMock.expectLastCall(); @@ -2284,23 +1766,10 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase // Start a new task final KinesisIndexTask task2 = createTask( + 0, task1.getId(), - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "0"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "6")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + ImmutableMap.of(SHARD_ID1, "0"), + ImmutableMap.of(SHARD_ID1, "6") ); final ListenableFuture future2 = runTask(task2); @@ -2310,13 +1779,10 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); - // Check metrics - Assert.assertEquals(5, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(2, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSize(SINGLE_PARTITION_RECORDS, 0, 5)) + .totalProcessed(5)); + verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSize(SINGLE_PARTITION_RECORDS, 5, 7)) + .totalProcessed(2)); // Check published segments & metadata assertEqualsExceptVersion( @@ -2346,49 +1812,28 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase EasyMock.expectLastCall(); recordSupplier.seek(streamPartition, "2"); EasyMock.expectLastCall(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 5)) - .once() - .andReturn(Collections.emptyList()) - .anyTimes(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 2, 5)).once() + .andReturn(Collections.emptyList()).anyTimes(); replayAll(); final KinesisIndexTask task = createTask( + 0, "task1", - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "13")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, "13") ); final ListenableFuture future = runTask(task); - - - while (countEvents(task) != 3) { - Thread.sleep(25); - } - + waitUntil(task, t -> countEvents(t) == 3); Assert.assertEquals(3, countEvents(task)); - Assert.assertEquals(SeekableStreamIndexTaskRunner.Status.READING, task.getRunner().getStatus()); + Assert.assertTrue(isTaskReading(task)); task.getRunner().pause(); - while (task.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.PAUSED) { - Thread.sleep(10); - } - Assert.assertEquals(SeekableStreamIndexTaskRunner.Status.PAUSED, task.getRunner().getStatus()); + waitUntil(task, this::isTaskPaused); + Assert.assertTrue(isTaskPaused(task)); verifyAll(); @@ -2420,10 +1865,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); Assert.assertEquals(task.getRunner().getEndOffsets(), task.getRunner().getCurrentOffsets()); - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 5)) + .totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -2457,8 +1900,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(2, 13)) - .once(); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(clone(RECORDS, 2, 13)).once(); recordSupplier.close(); EasyMock.expectLastCall(); @@ -2475,7 +1918,6 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase OBJECT_MAPPER.writerFor(KinesisSupervisor.CHECKPOINTS_TYPE_REF).writeValueAsString(sequences) ); - final KinesisIndexTask task = createTask( "task1", NEW_DATA_SCHEMA, @@ -2501,11 +1943,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 5)) + .totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -2524,7 +1963,6 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase @Test(timeout = 5000L) public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception { - final String baseSequenceName = "sequence0"; // as soon as any segment has more than one record, incremental publishing should happen maxRowsPerSegment = 2; @@ -2534,10 +1972,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase EasyMock.expect(recordSupplier1.getEarliestSequenceNumber(EasyMock.anyObject())).andReturn("0").anyTimes(); recordSupplier1.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier1.poll(EasyMock.anyLong())).andReturn(generateSinglePartitionRecords(0, 5)) - .once() - .andReturn(generateSinglePartitionRecords(4, 10)) - .once(); + EasyMock.expect(recordSupplier1.poll(EasyMock.anyLong())) + .andReturn(clone(SINGLE_PARTITION_RECORDS, 0, 5)).once() + .andReturn(clone(SINGLE_PARTITION_RECORDS, 4, 10)).once(); recordSupplier1.close(); EasyMock.expectLastCall().once(); final KinesisRecordSupplier recordSupplier2 = mock(KinesisRecordSupplier.class); @@ -2546,64 +1983,22 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase EasyMock.expect(recordSupplier2.getEarliestSequenceNumber(EasyMock.anyObject())).andReturn("0").anyTimes(); recordSupplier2.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier2.poll(EasyMock.anyLong())).andReturn(generateSinglePartitionRecords(0, 5)) - .once() - .andReturn(generateSinglePartitionRecords(4, 10)) - .once(); + EasyMock.expect(recordSupplier2.poll(EasyMock.anyLong())) + .andReturn(clone(SINGLE_PARTITION_RECORDS, 0, 5)).once() + .andReturn(clone(SINGLE_PARTITION_RECORDS, 4, 10)).once(); recordSupplier2.close(); EasyMock.expectLastCall().once(); replayAll(); - final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( - STREAM, - ImmutableMap.of(SHARD_ID1, "0"), - ImmutableSet.of() - ); + final Map startOffsets = ImmutableMap.of(SHARD_ID1, "0"); + final Map checkpointOffsets1 = ImmutableMap.of(SHARD_ID1, "4"); + final Map checkpointOffsets2 = ImmutableMap.of(SHARD_ID1, "9"); + final Map endOffsets = ImmutableMap.of(SHARD_ID1, "100"); // simulating unlimited - final SeekableStreamEndSequenceNumbers checkpoint1 = new SeekableStreamEndSequenceNumbers<>( - STREAM, - ImmutableMap.of(SHARD_ID1, "4") - ); - - final SeekableStreamEndSequenceNumbers checkpoint2 = new SeekableStreamEndSequenceNumbers<>( - STREAM, - ImmutableMap.of(SHARD_ID1, "9") - ); - - final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( - STREAM, - ImmutableMap.of(SHARD_ID1, "100") // simulating unlimited - ); - final KinesisIndexTaskIOConfig ioConfig = new KinesisIndexTaskIOConfig( - 0, - baseSequenceName, - startPartitions, - endPartitions, - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ); - final KinesisIndexTask normalReplica = createTask( - null, - NEW_DATA_SCHEMA, - ioConfig, - null - ); + final KinesisIndexTask normalReplica = createTask(0, startOffsets, endOffsets); ((TestableKinesisIndexTask) normalReplica).setLocalSupplier(recordSupplier1); - final KinesisIndexTask staleReplica = createTask( - null, - NEW_DATA_SCHEMA, - ioConfig, - null - ); + final KinesisIndexTask staleReplica = createTask(0, startOffsets, endOffsets); ((TestableKinesisIndexTask) staleReplica).setLocalSupplier(recordSupplier2); final ListenableFuture normalReplicaFuture = runTask(normalReplica); // Simulating one replica is slower than the other @@ -2615,29 +2010,23 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase (AsyncFunction) this::runTask ); - while (normalReplica.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.PAUSED) { - Thread.sleep(10); - } + waitUntil(normalReplica, this::isTaskPaused); staleReplica.getRunner().pause(); - while (staleReplica.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.PAUSED) { - Thread.sleep(10); - } + waitUntil(staleReplica, this::isTaskPaused); + Map currentOffsets = ImmutableMap.copyOf(normalReplica.getRunner().getCurrentOffsets()); - Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); + Assert.assertEquals(checkpointOffsets1, currentOffsets); normalReplica.getRunner().setEndOffsets(currentOffsets, false); staleReplica.getRunner().setEndOffsets(currentOffsets, false); - while (normalReplica.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.PAUSED) { - Thread.sleep(10); - } - while (staleReplica.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.PAUSED) { - Thread.sleep(10); - } + waitUntil(normalReplica, this::isTaskPaused); + waitUntil(staleReplica, this::isTaskPaused); + currentOffsets = ImmutableMap.copyOf(normalReplica.getRunner().getCurrentOffsets()); - Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), currentOffsets); + Assert.assertEquals(checkpointOffsets2, currentOffsets); currentOffsets = ImmutableMap.copyOf(staleReplica.getRunner().getCurrentOffsets()); - Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), currentOffsets); + Assert.assertEquals(checkpointOffsets2, currentOffsets); normalReplica.getRunner().setEndOffsets(currentOffsets, true); staleReplica.getRunner().setEndOffsets(currentOffsets, true); @@ -2649,10 +2038,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertEquals(2, checkpointRequestsHash.size()); - // Check metrics - Assert.assertEquals(10, normalReplica.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, normalReplica.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, normalReplica.getRunner().getRowIngestionMeters().getThrownAway()); + long totalRecordBytes = getTotalSize(SINGLE_PARTITION_RECORDS, 0, 10); + verifyTaskMetrics(normalReplica, RowMeters.with().bytes(totalRecordBytes).totalProcessed(10)); + verifyTaskMetrics(staleReplica, RowMeters.with().bytes(totalRecordBytes).totalProcessed(10)); // Check published metadata assertEqualsExceptVersion( @@ -2766,7 +2154,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase ); EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) - .andReturn(generateRecords(2, 5)).once() + .andReturn(clone(RECORDS, 2, 5)).once() .andReturn(eosRecord).once(); recordSupplier.close(); @@ -2775,30 +2163,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>( - STREAM, - ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of() - ), - new SeekableStreamEndSequenceNumbers<>( - STREAM, - ImmutableMap.of(SHARD_ID1, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER) - ), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) - + 0, + ImmutableMap.of(SHARD_ID1, "2"), + ImmutableMap.of(SHARD_ID1, KinesisSequenceNumber.NO_END_SEQUENCE_NUMBER) ); final ListenableFuture future = runTask(task); @@ -2808,10 +2175,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase verifyAll(); - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 2, 5)) + .totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -2843,7 +2208,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(Collections.emptyList()).times(1, Integer.MAX_VALUE); + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())) + .andReturn(Collections.emptyList()) + .times(1, Integer.MAX_VALUE); recordSupplier.close(); EasyMock.expectLastCall().once(); @@ -2851,24 +2218,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase replayAll(); final KinesisIndexTask task = createTask( - null, - new KinesisIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID0, "0"), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID0, "1")), - true, - null, - null, - INPUT_FORMAT, - "awsEndpoint", - null, - null, - null, - null, - false - ) - + 0, + ImmutableMap.of(SHARD_ID0, "0"), + ImmutableMap.of(SHARD_ID0, "1") ); final ListenableFuture future = runTask(task); @@ -2884,10 +2236,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); verifyAll(); - // Check metrics - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + verifyTaskMetrics(task, RowMeters.with().totalProcessed(0)); // Check published metadata and segments in deep storage assertEqualsExceptVersion(Collections.emptyList(), publishedDescriptors()); @@ -2895,20 +2244,69 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase } private KinesisIndexTask createTask( - final String taskId, - final KinesisIndexTaskIOConfig ioConfig + int groupId, + Map startSequenceNumbers, + Map endSequenceNumbers ) throws JsonProcessingException { - return createTask(taskId, NEW_DATA_SCHEMA, ioConfig, null); + return createTask(groupId, null, startSequenceNumbers, endSequenceNumbers); } private KinesisIndexTask createTask( - final String taskId, - final DataSchema dataSchema, - final KinesisIndexTaskIOConfig ioConfig + int groupId, + String taskId, + DataSchema dataSchema, + Map startSequenceNumbers, + Map endSequenceNumbers, + boolean useTransaction ) throws JsonProcessingException { - return createTask(taskId, dataSchema, ioConfig, null); + return createTask( + taskId, + dataSchema, + new KinesisIndexTaskIOConfig( + groupId, + "sequence_" + groupId, + new SeekableStreamStartSequenceNumbers<>(STREAM, startSequenceNumbers, ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(STREAM, endSequenceNumbers), + useTransaction, + null, + null, + INPUT_FORMAT, + "awsEndpoint", + null, + null, + null, + null, + false + ), + null + ); + } + + private KinesisIndexTask createTask( + int groupId, + String taskId, + Map startSequenceNumbers, + Map endSequenceNumbers + ) throws JsonProcessingException + { + return createTask(groupId, taskId, NEW_DATA_SCHEMA, startSequenceNumbers, endSequenceNumbers, true); + } + + private KinesisIndexTask createTask( + DataSchema dataSchema, + Map startSequenceNumbers, + Map endSequenceNumbers + ) throws JsonProcessingException + { + return createTask(0, null, dataSchema, startSequenceNumbers, endSequenceNumbers, true); + } + + private KinesisIndexTask createTask(KinesisIndexTaskIOConfig ioConfig) + throws JsonProcessingException + { + return createTask(null, NEW_DATA_SCHEMA, ioConfig, null); } private KinesisIndexTask createTask( @@ -2918,11 +2316,9 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase @Nullable final Map context ) throws JsonProcessingException { - boolean resetOffsetAutomatically = false; - int maxRowsInMemory = 1000; final KinesisIndexTaskTuningConfig tuningConfig = new KinesisIndexTaskTuningConfig( null, - maxRowsInMemory, + 1000, null, null, maxRowsPerSegment, @@ -2934,7 +2330,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase null, reportParseExceptions, handoffConditionTimeout, - resetOffsetAutomatically, + false, true, null, null, @@ -2994,7 +2390,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase ); } - private QueryRunnerFactoryConglomerate makeTimeseriesOnlyConglomerate() + @Override + protected QueryRunnerFactoryConglomerate makeQueryRunnerConglomerate() { return new DefaultQueryRunnerFactoryConglomerate( ImmutableMap.of( @@ -3022,151 +2419,17 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase for (Module module : new KinesisIndexingServiceModule().getJacksonModules()) { objectMapper.registerModule(module); } - final TaskConfig taskConfig = new TaskConfig( - new File(directory, "baseDir").getPath(), - new File(directory, "baseTaskDir").getPath(), - null, - 50000, - null, - true, - null, - null, - null, - false, - false, - TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), - null, - false - ); - final TestDerbyConnector derbyConnector = derby.getConnector(); - derbyConnector.createDataSourceTable(); - derbyConnector.createPendingSegmentsTable(); - derbyConnector.createSegmentTable(); - derbyConnector.createRulesTable(); - derbyConnector.createConfigTable(); - derbyConnector.createTaskTables(); - derbyConnector.createAuditTable(); - taskStorage = new MetadataTaskStorage( - derbyConnector, - new TaskStorageConfig(null), - new DerbyMetadataStorageActionHandlerFactory( - derbyConnector, - derby.metadataTablesConfigSupplier().get(), - objectMapper - ) - ); - metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( - testUtils.getTestObjectMapper(), - derby.metadataTablesConfigSupplier().get(), - derbyConnector - ); - taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); - final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( - taskLockbox, - taskStorage, - metadataStorageCoordinator, - emitter, - new SupervisorManager(null) - { - @Override - public boolean checkPointDataSourceMetadata( - String supervisorId, - int taskGroupId, - @Nullable DataSourceMetadata checkpointMetadata - ) - { - LOG.info("Adding checkpoint hash to the set"); - checkpointRequestsHash.add( - Objects.hash( - supervisorId, - taskGroupId, - checkpointMetadata - ) - ); - return true; - } - }, - objectMapper - ); - final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( - taskStorage, - taskActionToolbox, - new TaskAuditLogConfig(false) - ); + makeToolboxFactory(testUtils, emitter, doHandoff); + } - final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier() - { - @Override - public boolean registerSegmentHandoffCallback( - SegmentDescriptor descriptor, - Executor exec, - Runnable handOffRunnable - ) - { - if (doHandoff) { - // Simulate immediate handoff - exec.execute(handOffRunnable); - } - return true; - } + private boolean isTaskPaused(KinesisIndexTask task) + { + return task.getRunner().getStatus() == SeekableStreamIndexTaskRunner.Status.PAUSED; + } - @Override - public void start() - { - //Noop - } - - @Override - public void close() - { - //Noop - } - }; - final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); - dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); - dataSegmentPusherConfig.zip = true; - final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig); - - toolboxFactory = new TaskToolboxFactory( - taskConfig, - null, // taskExecutorNode - taskActionClientFactory, - emitter, - dataSegmentPusher, - new TestDataSegmentKiller(), - null, // DataSegmentMover - null, // DataSegmentArchiver - new TestDataSegmentAnnouncer(), - EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), - handoffNotifierFactory, - this::makeTimeseriesOnlyConglomerate, - DirectQueryProcessingPool.INSTANCE, - NoopJoinableFactory.INSTANCE, - () -> EasyMock.createMock(MonitorScheduler.class), - new SegmentCacheManagerFactory(testUtils.getTestObjectMapper()), - testUtils.getTestObjectMapper(), - testUtils.getTestIndexIO(), - MapCache.create(1024), - new CacheConfig(), - new CachePopulatorStats(), - testUtils.getIndexMergerV9Factory(), - EasyMock.createNiceMock(DruidNodeAnnouncer.class), - EasyMock.createNiceMock(DruidNode.class), - new LookupNodeService("tier"), - new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), - new SingleFileTaskReportFileWriter(reportsFile), - null, - AuthTestUtils.TEST_AUTHORIZER_MAPPER, - new NoopChatHandlerProvider(), - testUtils.getRowIngestionMetersFactory(), - new TestAppenderatorsManager(), - new NoopOverlordClient(), - null, - null, - null, - null, - "1" - ); + private boolean isTaskReading(KinesisIndexTask task) + { + return task.getRunner().getStatus() == SeekableStreamIndexTaskRunner.Status.READING; } @JsonTypeName("index_kinesis") @@ -3209,4 +2472,30 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase } } + /** + * Utility class to keep the test code more readable. + */ + private static class KinesisRecord extends OrderedPartitionableRecord + { + private final List data; + + public KinesisRecord( + String stream, + String partitionId, + String sequenceNumber, + List data + ) + { + super(stream, partitionId, sequenceNumber, data); + this.data = data; + } + + @Nonnull + @Override + public List getData() + { + return data; + } + } + } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountableInputSourceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountableInputSourceReader.java index 54b8ac9a8cd..fce2de14b5b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountableInputSourceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountableInputSourceReader.java @@ -22,6 +22,7 @@ package org.apache.druid.msq.indexing; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputStats; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.msq.counters.ChannelCounters; @@ -42,9 +43,9 @@ public class CountableInputSourceReader implements InputSourceReader } @Override - public CloseableIterator read() throws IOException + public CloseableIterator read(InputStats inputStats) throws IOException { - return inputSourceReader.read().map(inputRow -> { + return inputSourceReader.read(inputStats).map(inputRow -> { channelCounters.incrementRowCount(); return inputRow; }); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSource.java index 58912a372c1..e003e11308a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/NilInputSource.java @@ -27,6 +27,7 @@ import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputStats; import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -75,7 +76,7 @@ public class NilInputSource implements InputSource return new InputSourceReader() { @Override - public CloseableIterator read() + public CloseableIterator read(InputStats inputStats) { return CloseableIterators.wrap(Collections.emptyIterator(), () -> {}); } diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java index 46e4a6d199a..0cd267a3487 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java @@ -53,10 +53,12 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.data.input.impl.CloudObjectLocation; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputStatsImpl; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.initialization.DruidModule; @@ -942,8 +944,10 @@ public class S3InputSourceTest extends InitializedNullHandlingTest temporaryFolder.newFolder() ); - CloseableIterator iterator = reader.read(); + final InputStats inputStats = new InputStatsImpl(); + CloseableIterator iterator = reader.read(inputStats); + Assert.assertEquals(CONTENT.length, inputStats.getProcessedBytes()); while (iterator.hasNext()) { InputRow nextRow = iterator.next(); Assert.assertEquals(NOW, nextRow.getTimestamp()); @@ -1036,8 +1040,10 @@ public class S3InputSourceTest extends InitializedNullHandlingTest temporaryFolder.newFolder() ); - CloseableIterator iterator = reader.read(); + final InputStats inputStats = new InputStatsImpl(); + CloseableIterator iterator = reader.read(inputStats); + Assert.assertEquals(CONTENT.length, inputStats.getProcessedBytes()); while (iterator.hasNext()) { InputRow nextRow = iterator.next(); Assert.assertEquals(NOW, nextRow.getTimestamp()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java index 318f1c46dbb..4f15f0f1598 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java @@ -34,6 +34,7 @@ public class DropwizardRowIngestionMeters implements RowIngestionMeters private static final String FIFTEEN_MINUTE_NAME = "15m"; private final Meter processed; + private final Meter processedBytes; private final Meter processedWithError; private final Meter unparseable; private final Meter thrownAway; @@ -42,6 +43,7 @@ public class DropwizardRowIngestionMeters implements RowIngestionMeters { MetricRegistry metricRegistry = new MetricRegistry(); this.processed = metricRegistry.meter(PROCESSED); + this.processedBytes = metricRegistry.meter(PROCESSED_BYTES); this.processedWithError = metricRegistry.meter(PROCESSED_WITH_ERROR); this.unparseable = metricRegistry.meter(UNPARSEABLE); this.thrownAway = metricRegistry.meter(THROWN_AWAY); @@ -59,6 +61,18 @@ public class DropwizardRowIngestionMeters implements RowIngestionMeters processed.mark(); } + @Override + public long getProcessedBytes() + { + return processedBytes.getCount(); + } + + @Override + public void incrementProcessedBytes(long incrementByValue) + { + processedBytes.mark(incrementByValue); + } + @Override public long getProcessedWithError() { @@ -100,6 +114,7 @@ public class DropwizardRowIngestionMeters implements RowIngestionMeters { return new RowIngestionMetersTotals( processed.getCount(), + processedBytes.getCount(), processedWithError.getCount(), thrownAway.getCount(), unparseable.getCount() @@ -113,18 +128,21 @@ public class DropwizardRowIngestionMeters implements RowIngestionMeters Map oneMinute = new HashMap<>(); oneMinute.put(PROCESSED, processed.getOneMinuteRate()); + oneMinute.put(PROCESSED_BYTES, processedBytes.getOneMinuteRate()); oneMinute.put(PROCESSED_WITH_ERROR, processedWithError.getOneMinuteRate()); oneMinute.put(UNPARSEABLE, unparseable.getOneMinuteRate()); oneMinute.put(THROWN_AWAY, thrownAway.getOneMinuteRate()); Map fiveMinute = new HashMap<>(); fiveMinute.put(PROCESSED, processed.getFiveMinuteRate()); + fiveMinute.put(PROCESSED_BYTES, processedBytes.getFiveMinuteRate()); fiveMinute.put(PROCESSED_WITH_ERROR, processedWithError.getFiveMinuteRate()); fiveMinute.put(UNPARSEABLE, unparseable.getFiveMinuteRate()); fiveMinute.put(THROWN_AWAY, thrownAway.getFiveMinuteRate()); Map fifteenMinute = new HashMap<>(); fifteenMinute.put(PROCESSED, processed.getFifteenMinuteRate()); + fifteenMinute.put(PROCESSED_BYTES, processedBytes.getFifteenMinuteRate()); fifteenMinute.put(PROCESSED_WITH_ERROR, processedWithError.getFifteenMinuteRate()); fifteenMinute.put(UNPARSEABLE, unparseable.getFifteenMinuteRate()); fifteenMinute.put(THROWN_AWAY, thrownAway.getFifteenMinuteRate()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java index 674b6e77bf7..91c842a05fb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java @@ -60,7 +60,7 @@ public class TaskRealtimeMetricsMonitor extends AbstractMonitor this.rowIngestionMeters = rowIngestionMeters; this.dimensions = ImmutableMap.copyOf(dimensions); previousFireDepartmentMetrics = new FireDepartmentMetrics(); - previousRowIngestionMetersTotals = new RowIngestionMetersTotals(0, 0, 0, 0); + previousRowIngestionMetersTotals = new RowIngestionMetersTotals(0, 0, 0, 0, 0); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 77c1c449999..6e6e12f2ad8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -217,7 +217,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask ) ); return new FilteringCloseableInputRowIterator( - inputSourceReader.read(), + inputSourceReader.read(ingestionMeters), rowFilter, ingestionMeters, parseExceptionHandler diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 434bbcdff63..ae2de54c3f1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -64,10 +64,10 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.rpc.HttpResponseException; import org.apache.druid.rpc.indexing.OverlordClient; -import org.apache.druid.segment.incremental.MutableRowIngestionMeters; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; +import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; @@ -1214,10 +1214,8 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen */ private Map getTaskCompletionReports(TaskStatus taskStatus, boolean segmentAvailabilityConfirmed) { - Pair, Map> rowStatsAndUnparseableEvents = doGetRowStatsAndUnparseableEvents( - "true", - true - ); + Pair, Map> rowStatsAndUnparseableEvents = + doGetRowStatsAndUnparseableEvents("true", true); return TaskReport.buildTaskReports( new IngestionStatsAndErrorsTaskReport( getId(), @@ -1538,6 +1536,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen Map buildSegmentsRowStatsMap = (Map) buildSegmentsRowStats; return new RowIngestionMetersTotals( ((Number) buildSegmentsRowStatsMap.get("processed")).longValue(), + ((Number) buildSegmentsRowStatsMap.get("processedBytes")).longValue(), ((Number) buildSegmentsRowStatsMap.get("processedWithError")).longValue(), ((Number) buildSegmentsRowStatsMap.get("thrownAway")).longValue(), ((Number) buildSegmentsRowStatsMap.get("unparseable")).longValue() @@ -1553,7 +1552,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen boolean includeUnparseable ) { - final MutableRowIngestionMeters buildSegmentsRowStats = new MutableRowIngestionMeters(); + final SimpleRowIngestionMeters buildSegmentsRowStats = new SimpleRowIngestionMeters(); List unparseableEvents = new ArrayList<>(); @@ -1594,7 +1593,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen Map completedSubtaskReports = (Map) currentRunner.getReports(); - final MutableRowIngestionMeters buildSegmentsRowStats = new MutableRowIngestionMeters(); + final SimpleRowIngestionMeters buildSegmentsRowStats = new SimpleRowIngestionMeters(); final List unparseableEvents = new ArrayList<>(); for (GeneratedPartitionsReport generatedPartitionsReport : completedSubtaskReports.values()) { Map taskReport = generatedPartitionsReport.getTaskReport(); @@ -1625,7 +1624,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen boolean includeUnparseable ) { - final MutableRowIngestionMeters buildSegmentsRowStats = new MutableRowIngestionMeters(); + final SimpleRowIngestionMeters buildSegmentsRowStats = new SimpleRowIngestionMeters(); for (String runningTaskId : runningTaskIds) { try { final Map report = getTaskReport(toolbox.getOverlordClient(), runningTaskId); @@ -1690,7 +1689,10 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen return totals; } - private Pair, Map> doGetRowStatsAndUnparseableEvents(String full, boolean includeUnparseable) + private Pair, Map> doGetRowStatsAndUnparseableEvents( + String full, + boolean includeUnparseable + ) { if (currentSubTaskHolder == null) { return Pair.of(ImmutableMap.of(), ImmutableMap.of()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentInputFormat.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentInputFormat.java index 732288e8541..98c9b40af5f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentInputFormat.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentInputFormat.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.input; import com.google.common.base.Preconditions; +import org.apache.druid.data.input.BytesCountingInputEntity; import org.apache.druid.data.input.InputEntity; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputFormat; @@ -56,19 +57,25 @@ public class DruidSegmentInputFormat implements InputFormat File temporaryDirectory ) { + final InputEntity baseInputEntity; + if (source instanceof BytesCountingInputEntity) { + baseInputEntity = ((BytesCountingInputEntity) source).getBaseInputEntity(); + } else { + baseInputEntity = source; + } + // this method handles the case when the entity comes from a tombstone or from a regular segment Preconditions.checkArgument( - source instanceof DruidSegmentInputEntity, + baseInputEntity instanceof DruidSegmentInputEntity, DruidSegmentInputEntity.class.getName() + " required, but " - + source.getClass().getName() + " provided." + + baseInputEntity.getClass().getName() + " provided." ); - final InputEntityReader retVal; - // Cast is safe here because of the precondition above passed - if (((DruidSegmentInputEntity) source).isFromTombstone()) { - retVal = new DruidTombstoneSegmentReader(source); + final DruidSegmentInputEntity druidSegmentEntity = (DruidSegmentInputEntity) baseInputEntity; + if (druidSegmentEntity.isFromTombstone()) { + return new DruidTombstoneSegmentReader(druidSegmentEntity); } else { - retVal = new DruidSegmentReader( + return new DruidSegmentReader( source, indexIO, inputRowSchema.getTimestampSpec(), @@ -78,6 +85,5 @@ public class DruidSegmentInputFormat implements InputFormat temporaryDirectory ); } - return retVal; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java index 87181b8cb49..decbf3b4038 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java @@ -23,6 +23,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Supplier; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; +import org.apache.druid.data.input.BytesCountingInputEntity; import org.apache.druid.data.input.ColumnsFilter; import org.apache.druid.data.input.InputEntity; import org.apache.druid.data.input.InputEntity.CleanableFile; @@ -57,6 +58,7 @@ import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; import org.apache.druid.utils.CloseableUtils; import org.apache.druid.utils.CollectionUtils; +import org.joda.time.Interval; import java.io.File; import java.io.IOException; @@ -71,12 +73,13 @@ import java.util.Set; public class DruidSegmentReader extends IntermediateRowParsingReader> { - private DruidSegmentInputEntity source; + private final InputEntity source; private final IndexIO indexIO; private final ColumnsFilter columnsFilter; private final InputRowSchema inputRowSchema; private final DimFilter dimFilter; private final File temporaryDirectory; + private final Interval intervalFilter; DruidSegmentReader( final InputEntity source, @@ -88,7 +91,7 @@ public class DruidSegmentReader extends IntermediateRowParsingReader cursors = storageAdapter.getAdapter().makeCursors( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidTombstoneSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidTombstoneSegmentReader.java index 2a80112df85..a8b98dd7d0f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidTombstoneSegmentReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidTombstoneSegmentReader.java @@ -20,10 +20,9 @@ package org.apache.druid.indexing.input; import com.google.common.annotations.VisibleForTesting; -import org.apache.druid.data.input.InputEntity; +import com.google.common.base.Preconditions; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.IntermediateRowParsingReader; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.util.List; @@ -35,16 +34,14 @@ import java.util.NoSuchElementException; */ public class DruidTombstoneSegmentReader extends IntermediateRowParsingReader> { - private DruidSegmentInputEntity source; - public DruidTombstoneSegmentReader( - InputEntity source + DruidSegmentInputEntity source ) { - this.source = (DruidSegmentInputEntity) source; - if (!this.source.isFromTombstone()) { - throw new IAE("DruidSegmentInputEntity must be created from a tombstone but is not."); - } + Preconditions.checkArgument( + source.isFromTombstone(), + "DruidSegmentInputEntity must be created from a tombstone." + ); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/GeneratorInputSource.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/GeneratorInputSource.java index 208666d15b9..4f5d587c03d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/GeneratorInputSource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/GeneratorInputSource.java @@ -30,6 +30,7 @@ import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.SplittableInputSource; @@ -148,7 +149,7 @@ public class GeneratorInputSource extends AbstractInputSource implements Splitta return new InputSourceReader() { @Override - public CloseableIterator read() + public CloseableIterator read(InputStats inputStats) { return CloseableIterators.withEmptyBaggage(new Iterator() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java index 7c1489c44e0..b6a8d971b8e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java @@ -108,11 +108,13 @@ class StreamChunkParser } } - private List parseWithParser(InputRowParser parser, List valueBytess) + private List parseWithParser(InputRowParser parser, List valueBytes) { final FluentIterable iterable = FluentIterable - .from(valueBytess) - .transformAndConcat(bytes -> parser.parseBatch(bytes.getBuffer())); + .from(valueBytes) + .transform(ByteEntity::getBuffer) + .transform(this::incrementProcessedBytes) + .transformAndConcat(parser::parseBatch); final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( CloseableIterators.withEmptyBaggage(iterable.iterator()), @@ -123,6 +125,16 @@ class StreamChunkParser return Lists.newArrayList(rowIterator); } + /** + * Increments the processed bytes with the number of bytes remaining in the + * given buffer. This method must be called before reading the buffer. + */ + private ByteBuffer incrementProcessedBytes(final ByteBuffer recordByteBuffer) + { + rowIngestionMeters.incrementProcessedBytes(recordByteBuffer.remaining()); + return recordByteBuffer; + } + private List parseWithInputFormat( SettableByteEntityReader byteEntityReader, List valueBytess @@ -130,6 +142,7 @@ class StreamChunkParser { final List rows = new ArrayList<>(); for (ByteEntity valueBytes : valueBytess) { + rowIngestionMeters.incrementProcessedBytes(valueBytes.getBuffer().remaining()); byteEntityReader.setEntity(valueBytes); try (FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( byteEntityReader.read(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 7bff995578d..29cc5df254b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -147,7 +147,6 @@ import java.util.ArrayDeque; import java.util.Arrays; import java.util.Collection; import java.util.Deque; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -158,7 +157,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; -import java.util.stream.Collectors; public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHandlingTest { @@ -690,25 +688,17 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); - List parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.BUILD_SEGMENTS); - + ParseExceptionReport parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); List expectedMessages = ImmutableList.of( "Unable to parse value[foo] for field[met1]" ); - List actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); List expectedInputs = ImmutableList.of( "{t=3000000, dim1=foo, met1=foo}" ); - List actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); + Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); } @Test(timeout = 60_000L) @@ -792,6 +782,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( RowIngestionMeters.PROCESSED, 2, + RowIngestionMeters.PROCESSED_BYTES, 0, RowIngestionMeters.PROCESSED_WITH_ERROR, 1, RowIngestionMeters.UNPARSEABLE, 2, RowIngestionMeters.THROWN_AWAY, 0 @@ -895,6 +886,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( RowIngestionMeters.PROCESSED, 2, + RowIngestionMeters.PROCESSED_BYTES, 0, RowIngestionMeters.PROCESSED_WITH_ERROR, 2, RowIngestionMeters.UNPARSEABLE, 2, RowIngestionMeters.THROWN_AWAY, 0 @@ -906,12 +898,10 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); - Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - List parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.BUILD_SEGMENTS); + ParseExceptionReport parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); List expectedMessages = Arrays.asList( "Timestamp[null] is unparseable! Event: {dim1=foo, met1=2.0, __fail__=x}", @@ -919,10 +909,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand "Unable to parse value[foo] for field[met1]", "Timestamp[null] is unparseable! Event: null" ); - List actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); List expectedInputs = Arrays.asList( "{dim1=foo, met1=2.0, __fail__=x}", @@ -930,11 +917,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand "{t=1521251960729, dim1=foo, met1=foo}", null ); - List actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); - + Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); Assert.assertEquals(IngestionState.COMPLETED, reportData.getIngestionState()); } @@ -1000,6 +983,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( RowIngestionMeters.PROCESSED, 1, + RowIngestionMeters.PROCESSED_BYTES, 0, RowIngestionMeters.PROCESSED_WITH_ERROR, 2, RowIngestionMeters.UNPARSEABLE, 2, RowIngestionMeters.THROWN_AWAY, 0 @@ -1007,9 +991,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand ); Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - List parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.BUILD_SEGMENTS); + ParseExceptionReport parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); List expectedMessages = ImmutableList.of( "Timestamp[null] is unparseable! Event: {dim1=foo, met1=2.0, __fail__=x}", @@ -1017,10 +1000,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand "Unable to parse value[foo] for field[met1]", "Timestamp[null] is unparseable! Event: null" ); - List actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); List expectedInputs = Arrays.asList( "{dim1=foo, met1=2.0, __fail__=x}", @@ -1028,12 +1008,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand "{t=1521251960729, dim1=foo, met1=foo}", null ); - List actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); - - + Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); Assert.assertEquals(IngestionState.BUILD_SEGMENTS, reportData.getIngestionState()); } @@ -1272,6 +1247,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand ImmutableMap.of( RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.PROCESSED, 0, + RowIngestionMeters.PROCESSED_BYTES, 0, RowIngestionMeters.UNPARSEABLE, 0, RowIngestionMeters.THROWN_AWAY, 0 ) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index 3e692ca9020..f91e67dc7c6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -52,7 +52,6 @@ import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervi import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.indexing.firehose.WindowedSegmentId; import org.apache.druid.indexing.input.DruidInputSource; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -62,7 +61,7 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.loading.SegmentCacheManager; +import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.DimensionRangeShardSpec; @@ -94,7 +93,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; -import java.util.concurrent.ExecutorService; @RunWith(Parameterized.class) public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervisorTaskTest @@ -880,46 +878,7 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis return baseToolbox; } else { return new TaskToolbox.Builder(baseToolbox) - .segmentCacheManager( - new SegmentCacheManager() - { - @Override - public boolean isSegmentCached(DataSegment segment) - { - throw new ISE("Expected no segment fetches by the compaction task"); - } - - @Override - public File getSegmentFiles(DataSegment segment) - { - throw new ISE("Expected no segment fetches by the compaction task"); - } - - @Override - public boolean reserve(DataSegment segment) - { - throw new ISE("Expected no segment fetches by the compaction task"); - } - - @Override - public boolean release(DataSegment segment) - { - throw new ISE("Expected no segment fetches by the compaction task"); - } - - @Override - public void cleanup(DataSegment segment) - { - throw new ISE("Expected no segment fetches by the compaction task"); - } - - @Override - public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) - { - throw new ISE("Expected no segment fetches by the compaction task"); - } - } - ) + .segmentCacheManager(new NoopSegmentCacheManager()) .build(); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index af92e821ee7..65bf7feba83 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -118,6 +118,7 @@ import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; +import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; @@ -151,7 +152,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Optional; -import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -1975,43 +1975,19 @@ public class CompactionTaskTest Map segments ) { - final SegmentCacheManager segmentCacheManager = new SegmentCacheManager() + final SegmentCacheManager segmentCacheManager = new NoopSegmentCacheManager() { - @Override - public boolean isSegmentCached(DataSegment segment) - { - throw new UnsupportedOperationException(); - } - @Override public File getSegmentFiles(DataSegment segment) { return Preconditions.checkNotNull(segments.get(segment)); } - @Override - public boolean reserve(DataSegment segment) - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean release(DataSegment segment) - { - throw new UnsupportedOperationException(); - } - @Override public void cleanup(DataSegment segment) { // Do nothing. } - - @Override - public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) - { - throw new UnsupportedOperationException(); - } }; return new TaskToolbox.Builder() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 095630a658c..469eaff4dab 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -123,13 +123,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; @RunWith(Parameterized.class) public class IndexTaskTest extends IngestionTestBase @@ -1521,22 +1519,12 @@ public class IndexTaskTest extends IngestionTestBase IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); - List parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.BUILD_SEGMENTS); + ParseExceptionReport parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); - List actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); - - List expectedInputs = ImmutableList.of( - "{time=unparseable, d=a, val=1}" - ); - List actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); + List expectedInputs = ImmutableList.of("{time=unparseable, d=a, val=1}"); + Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); } @Test @@ -1633,32 +1621,33 @@ public class IndexTaskTest extends IngestionTestBase TaskStatus status = runTask(indexTask).lhs; Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); - Assert.assertEquals(null, status.getErrorMsg()); + Assert.assertNull(status.getErrorMsg()); IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + final int processedBytes = useInputFormatApi ? 657 : 0; Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.DETERMINE_PARTITIONS, ImmutableMap.of( RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.PROCESSED, 4, + RowIngestionMeters.PROCESSED_BYTES, processedBytes, RowIngestionMeters.UNPARSEABLE, 4, RowIngestionMeters.THROWN_AWAY, 1 ), RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( - RowIngestionMeters.PROCESSED_WITH_ERROR, 3, RowIngestionMeters.PROCESSED, 1, + RowIngestionMeters.PROCESSED_BYTES, processedBytes, RowIngestionMeters.UNPARSEABLE, 4, RowIngestionMeters.THROWN_AWAY, 1 ) ); Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - List parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.BUILD_SEGMENTS); + ParseExceptionReport parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); List expectedMessages; if (useInputFormatApi) { @@ -1692,10 +1681,7 @@ public class IndexTaskTest extends IngestionTestBase ); } - List actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); List expectedInputs = Arrays.asList( "this is not JSON", @@ -1706,14 +1692,10 @@ public class IndexTaskTest extends IngestionTestBase "{time=2014-01-01T00:00:10Z, dim=b, dimLong=notnumber, dimFloat=3.0, val=1}", "{time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}" ); - List actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); + Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); - parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.DETERMINE_PARTITIONS); + parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.DETERMINE_PARTITIONS); if (useInputFormatApi) { expectedMessages = Arrays.asList( @@ -1740,10 +1722,7 @@ public class IndexTaskTest extends IngestionTestBase ); } - actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); expectedInputs = Arrays.asList( "this is not JSON", @@ -1751,11 +1730,7 @@ public class IndexTaskTest extends IngestionTestBase "{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}", "{time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}" ); - actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); - + Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); } @Test @@ -1877,6 +1852,7 @@ public class IndexTaskTest extends IngestionTestBase ImmutableMap.of( RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.PROCESSED, 0, + RowIngestionMeters.PROCESSED_BYTES, 0, RowIngestionMeters.UNPARSEABLE, 0, RowIngestionMeters.THROWN_AWAY, 0 ), @@ -1884,6 +1860,7 @@ public class IndexTaskTest extends IngestionTestBase ImmutableMap.of( RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.PROCESSED, 1, + RowIngestionMeters.PROCESSED_BYTES, useInputFormatApi ? 182 : 0, RowIngestionMeters.UNPARSEABLE, 3, RowIngestionMeters.THROWN_AWAY, useInputFormatApi ? 1 : 2 ) @@ -1891,24 +1868,16 @@ public class IndexTaskTest extends IngestionTestBase Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - List parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.BUILD_SEGMENTS); - - List actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); + ParseExceptionReport parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); List expectedInputs = Arrays.asList( "{time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}", "{time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1}", "{time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}" ); - List actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); + Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); } @Test @@ -2021,6 +1990,7 @@ public class IndexTaskTest extends IngestionTestBase ImmutableMap.of( RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.PROCESSED, 1, + RowIngestionMeters.PROCESSED_BYTES, useInputFormatApi ? 182 : 0, RowIngestionMeters.UNPARSEABLE, 3, RowIngestionMeters.THROWN_AWAY, useInputFormatApi ? 1 : 2 ), @@ -2028,6 +1998,7 @@ public class IndexTaskTest extends IngestionTestBase ImmutableMap.of( RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.PROCESSED, 0, + RowIngestionMeters.PROCESSED_BYTES, 0, RowIngestionMeters.UNPARSEABLE, 0, RowIngestionMeters.THROWN_AWAY, 0 ) @@ -2035,24 +2006,16 @@ public class IndexTaskTest extends IngestionTestBase Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - List parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.DETERMINE_PARTITIONS); - - List actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); + ParseExceptionReport parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.DETERMINE_PARTITIONS); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); List expectedInputs = Arrays.asList( "{time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}", "{time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1}", "{time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}" ); - List actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); + Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); } @Test @@ -2208,22 +2171,14 @@ public class IndexTaskTest extends IngestionTestBase IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); - List parseExceptionReports = (List) reportData - .getUnparseableEvents() - .get(RowIngestionMeters.BUILD_SEGMENTS); - - List actualMessages = parseExceptionReports.stream().map((r) -> { - return ((List) r.get("details")).get(0); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedMessages, actualMessages); + ParseExceptionReport parseExceptionReport = + ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); + Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); List expectedInputs = ImmutableList.of( "{column_1=2014-01-01T00:00:10Z, column_2=a, column_3=1}" ); - List actualInputs = parseExceptionReports.stream().map((r) -> { - return (String) r.get("input"); - }).collect(Collectors.toList()); - Assert.assertEquals(expectedInputs, actualInputs); + Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ParseExceptionReport.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ParseExceptionReport.java new file mode 100644 index 00000000000..cd5efbd88bc --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ParseExceptionReport.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; + +/** + * Utility class encapsulating a parse exception report used in tests. + */ +public class ParseExceptionReport +{ + private final List inputs; + private final List errorMessages; + + private ParseExceptionReport(List inputs, List errorMessages) + { + this.inputs = inputs; + this.errorMessages = errorMessages; + } + + @SuppressWarnings("unchecked") + public static ParseExceptionReport forPhase( + IngestionStatsAndErrorsTaskReportData reportData, + String phase + ) + { + List> events = + (List>) reportData.getUnparseableEvents().get(phase); + + final List inputs = new ArrayList<>(); + final List errorMessages = new ArrayList<>(); + events.forEach(event -> { + inputs.add((String) event.get("input")); + errorMessages.add(((List) event.get("details")).get(0)); + }); + + return new ParseExceptionReport(inputs, errorMessages); + } + + public List getInputs() + { + return inputs; + } + + public List getErrorMessages() + { + return errorMessages; + } + +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index b7cc3159ff2..d2255c31f0b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -828,6 +828,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase ); Map emptyAverageMinuteMap = ImmutableMap.of( "processed", 0.0, + "processedBytes", 0.0, "unparseable", 0.0, "thrownAway", 0.0, "processedWithError", 0.0 diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingRowStatsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingRowStatsTest.java index 3c1a7fb5e53..0cf7fdd147b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingRowStatsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingRowStatsTest.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; +import org.apache.druid.segment.incremental.RowMeters; import org.joda.time.Interval; import org.junit.Before; import org.junit.Ignore; @@ -132,13 +133,13 @@ public class MultiPhaseParallelIndexingRowStatsTest extends AbstractMultiPhasePa false ); - final RowIngestionMetersTotals expectedTotals = new RowIngestionMetersTotals(200, 0, 0, 0); + final RowIngestionMetersTotals expectedTotals = RowMeters.with().totalProcessed(200); final Map expectedReports = maxNumConcurrentSubTasks <= 1 ? buildExpectedTaskReportSequential( task.getId(), ImmutableList.of(), - new RowIngestionMetersTotals(0, 0, 0, 0), + RowMeters.with().totalProcessed(0), expectedTotals ) : buildExpectedTaskReportParallel( @@ -163,7 +164,6 @@ public class MultiPhaseParallelIndexingRowStatsTest extends AbstractMultiPhasePa INTERVAL_TO_INDEX, inputDir, "test_*", - //new DimensionRangePartitionsSpec(targetRowsPerSegment, null, DIMS, false), new SingleDimensionPartitionsSpec(targetRowsPerSegment, null, DIM1, false), 10, false, @@ -172,7 +172,7 @@ public class MultiPhaseParallelIndexingRowStatsTest extends AbstractMultiPhasePa Map expectedReports = buildExpectedTaskReportParallel( task.getId(), ImmutableList.of(), - new RowIngestionMetersTotals(200, 0, 0, 0) + new RowIngestionMetersTotals(200, 0, 0, 0, 0) ); Map actualReports = runTaskAndGetReports(task, TaskState.SUCCESS); compareTaskReports(expectedReports, actualReports); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index b35e660b909..ddf26dd072c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -437,6 +437,7 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv Collections.emptyList() ); Map actualReports = task.doGetLiveReports("full"); + final long processedBytes = useInputFormatApi ? 335 : 0; Map expectedReports = buildExpectedTaskReportParallel( task.getId(), ImmutableList.of( @@ -453,11 +454,7 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv 1L ) ), - new RowIngestionMetersTotals( - 10, - 1, - 1, - 1) + new RowIngestionMetersTotals(10, processedBytes, 1, 1, 1) ); compareTaskReports(expectedReports, actualReports); } @@ -492,12 +489,8 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv final ParallelIndexSupervisorTask executedTask = (ParallelIndexSupervisorTask) taskContainer.getTask(); Map actualReports = executedTask.doGetLiveReports("full"); - RowIngestionMetersTotals expectedTotals = new RowIngestionMetersTotals( - 10, - 1, - 1, - 1 - ); + final long processedBytes = useInputFormatApi ? 335 : 0; + RowIngestionMetersTotals expectedTotals = new RowIngestionMetersTotals(10, processedBytes, 1, 1, 1); List expectedUnparseableEvents = ImmutableList.of( new ParseExceptionReport( "{ts=2017unparseable}", @@ -518,7 +511,7 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv expectedReports = buildExpectedTaskReportSequential( task.getId(), expectedUnparseableEvents, - new RowIngestionMetersTotals(0, 0, 0, 0), + new RowIngestionMetersTotals(0, 0, 0, 0, 0), expectedTotals ); } else { @@ -532,7 +525,6 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv } compareTaskReports(expectedReports, actualReports); - System.out.println(actualReports); } @Test @@ -885,17 +877,6 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv boolean appendToExisting, boolean splittableInputSource ) - { - return newTask(interval, segmentGranularity, appendToExisting, splittableInputSource, false); - } - - private ParallelIndexSupervisorTask newTask( - @Nullable Interval interval, - Granularity segmentGranularity, - boolean appendToExisting, - boolean splittableInputSource, - boolean isReplace - ) { return newTask( interval, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java index ff4b50cee48..e65b7c8c71a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java @@ -24,19 +24,23 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.druid.common.config.NullHandlingTest; +import org.apache.druid.data.input.BytesCountingInputEntity; import org.apache.druid.data.input.ColumnsFilter; +import org.apache.druid.data.input.InputEntity; import org.apache.druid.data.input.InputEntity.CleanableFile; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.DoubleDimensionSchema; -import org.apache.druid.data.input.impl.FileEntity; +import org.apache.druid.data.input.impl.InputStatsImpl; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.hll.HyperLogLogCollector; import org.apache.druid.hll.HyperLogLogHash; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.BaseSequence.IteratorMaker; @@ -54,7 +58,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.loading.SegmentCacheManager; +import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.TombstoneShardSpec; @@ -68,11 +72,11 @@ import org.junit.rules.TemporaryFolder; import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutorService; import static org.junit.Assert.assertThrows; @@ -82,11 +86,13 @@ public class DruidSegmentReaderTest extends NullHandlingTest public TemporaryFolder temporaryFolder = new TemporaryFolder(); private File segmentDirectory; + private long segmentSize; private final IndexIO indexIO = TestHelper.getTestIndexIO(); private DimensionsSpec dimensionsSpec; private List metrics; - private List rows; + + private InputStats inputStats; @Before public void setUp() throws IOException @@ -102,7 +108,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest new CountAggregatorFactory("cnt"), new HyperUniquesAggregatorFactory("met_s", "strCol") ); - rows = ImmutableList.of( + final List rows = ImmutableList.of( new MapBasedInputRow( DateTimes.of("2000"), ImmutableList.of("strCol", "dblCol"), @@ -121,7 +127,8 @@ public class DruidSegmentReaderTest extends NullHandlingTest ) ); - createTestSetup(); + inputStats = new InputStatsImpl(); + persistSegment(rows); } @Test @@ -169,6 +176,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest ), readRows(reader) ); + Assert.assertEquals(segmentSize, inputStats.getProcessedBytes()); } @Test @@ -184,7 +192,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest metrics = ImmutableList.of(); List columnNames = ImmutableList.of("longCol", "a", "b"); - rows = ImmutableList.of( + final List rows = ImmutableList.of( new MapBasedInputRow( DateTimes.utc(1667115726217L), columnNames, @@ -217,10 +225,14 @@ public class DruidSegmentReaderTest extends NullHandlingTest ) ); - createTestSetup(); + persistSegment(rows); + final InputStats inputStats = new InputStatsImpl(); final DruidSegmentReader reader = new DruidSegmentReader( - makeInputEntityWithParams(Intervals.of("2022-10-30/2022-10-31"), columnNames, null), + new BytesCountingInputEntity( + makeInputEntity(Intervals.of("2022-10-30/2022-10-31"), segmentDirectory, columnNames, null), + inputStats + ), indexIO, new TimestampSpec("__time", "iso", null), dimensionsSpec, @@ -233,10 +245,8 @@ public class DruidSegmentReaderTest extends NullHandlingTest temporaryFolder.newFolder() ); - List expectedRows = new ArrayList<>(); - expectedRows.add(rows.get(2)); - expectedRows.add(rows.get(1)); - Assert.assertEquals(expectedRows, readRows(reader)); + Assert.assertEquals(Arrays.asList(rows.get(2), rows.get(1)), readRows(reader)); + Assert.assertEquals(segmentSize, inputStats.getProcessedBytes()); } @Test @@ -247,32 +257,27 @@ public class DruidSegmentReaderTest extends NullHandlingTest ); Assert.assertFalse(reader.intermediateRowIterator().hasNext()); - Assert.assertEquals( - Collections.emptyList(), - readRows(reader) - ); - } - - @Test - public void testDruidTombstoneSegmentReaderBadEntity() - { - assertThrows(ClassCastException.class, () -> { - new DruidTombstoneSegmentReader( - new FileEntity(null)); - }); + Assert.assertTrue(readRows(reader).isEmpty()); } @Test public void testDruidTombstoneSegmentReaderNotCreatedFromTombstone() { - Exception exception = assertThrows(IllegalArgumentException.class, () -> { - new DruidTombstoneSegmentReader(makeInputEntity(Intervals.of("2000/P1D"))); - }); - String expectedMessage = - "DruidSegmentInputEntity must be created from a tombstone but is not."; - String actualMessage = exception.getMessage(); - Assert.assertEquals(expectedMessage, actualMessage); - + Exception exception = assertThrows( + IllegalArgumentException.class, + () -> new DruidTombstoneSegmentReader( + makeInputEntity( + Intervals.of("2000/P1D"), + segmentDirectory, + Collections.emptyList(), + Collections.emptyList() + ) + ) + ); + Assert.assertEquals( + "DruidSegmentInputEntity must be created from a tombstone.", + exception.getMessage() + ); } @Test @@ -320,6 +325,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest ), readRows(reader) ); + Assert.assertEquals(segmentSize, inputStats.getProcessedBytes()); } @Test @@ -362,6 +368,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest ), readRows(reader) ); + Assert.assertEquals(segmentSize, inputStats.getProcessedBytes()); } @Test @@ -405,6 +412,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest ), readRows(reader) ); + Assert.assertEquals(segmentSize, inputStats.getProcessedBytes()); } @Test @@ -446,6 +454,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest ), readRows(reader) ); + Assert.assertEquals(segmentSize, inputStats.getProcessedBytes()); } @Test @@ -482,6 +491,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest ), readRows(reader) ); + Assert.assertEquals(segmentSize, inputStats.getProcessedBytes()); } @Test @@ -529,6 +539,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest ), readRows(reader) ); + Assert.assertEquals(segmentSize, inputStats.getProcessedBytes()); } @Test @@ -576,6 +587,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest ), readRows(reader) ); + Assert.assertEquals(segmentSize, inputStats.getProcessedBytes()); } @Test @@ -623,6 +635,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest ), readRows(reader) ); + Assert.assertEquals(segmentSize, inputStats.getProcessedBytes()); } @Test @@ -670,14 +683,17 @@ public class DruidSegmentReaderTest extends NullHandlingTest Assert.assertTrue("Sequence is not closed", isSequenceClosed.booleanValue()); } - private DruidSegmentInputEntity makeInputEntity(final Interval interval) + private InputEntity makeInputEntity(final Interval interval) { - return makeInputEntity(interval, segmentDirectory, ImmutableList.of("strCol", "dblCol"), ImmutableList.of("cnt", "met_s")); - } - - private DruidSegmentInputEntity makeInputEntityWithParams(final Interval interval, final List dimensions, final List metrics) - { - return makeInputEntity(interval, segmentDirectory, dimensions, metrics); + return new BytesCountingInputEntity( + makeInputEntity( + interval, + segmentDirectory, + ImmutableList.of("strCol", "dblCol"), + ImmutableList.of("cnt", "met_s") + ), + inputStats + ); } public static DruidSegmentInputEntity makeInputEntity( @@ -688,43 +704,13 @@ public class DruidSegmentReaderTest extends NullHandlingTest ) { return new DruidSegmentInputEntity( - new SegmentCacheManager() + new NoopSegmentCacheManager() { - @Override - public boolean isSegmentCached(DataSegment segment) - { - throw new UnsupportedOperationException("unused"); - } - @Override public File getSegmentFiles(DataSegment segment) { return segmentDirectory; } - - @Override - public void cleanup(DataSegment segment) - { - throw new UnsupportedOperationException("unused"); - } - - @Override - public boolean reserve(DataSegment segment) - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean release(DataSegment segment) - { - throw new UnsupportedOperationException(); - } - - @Override - public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) - { - throw new UnsupportedOperationException(); - } }, DataSegment.builder() .dataSource("ds") @@ -741,45 +727,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest public static DruidSegmentInputEntity makeTombstoneInputEntity(final Interval interval) { return new DruidSegmentInputEntity( - new SegmentCacheManager() - { - @Override - public boolean isSegmentCached(DataSegment segment) - { - throw new UnsupportedOperationException("unused"); - } - - @Override - public File getSegmentFiles(DataSegment segment) - { - throw new UnsupportedOperationException("unused"); - } - - @Override - public void cleanup(DataSegment segment) - { - throw new UnsupportedOperationException("unused"); - } - - @Override - public boolean reserve(DataSegment segment) - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean release(DataSegment segment) - { - throw new UnsupportedOperationException(); - } - - @Override - public void loadSegmentIntoPageCache(DataSegment segment, ExecutorService exec) - { - throw new UnsupportedOperationException(); - - } - }, + new NoopSegmentCacheManager(), DataSegment.builder() .dataSource("ds") .interval(Intervals.of("2000/P1D")) @@ -792,7 +740,6 @@ public class DruidSegmentReaderTest extends NullHandlingTest ); } - private List readRows(DruidSegmentReader reader) throws IOException { final List rows = new ArrayList<>(); @@ -815,7 +762,6 @@ public class DruidSegmentReaderTest extends NullHandlingTest return rows; } - private static HyperLogLogCollector makeHLLC(final String... values) { final HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); @@ -825,7 +771,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest return collector; } - private void createTestSetup() throws IOException + private void persistSegment(List rows) throws IOException { final IncrementalIndex incrementalIndex = IndexBuilder.create() @@ -850,6 +796,7 @@ public class DruidSegmentReaderTest extends NullHandlingTest new IndexSpec(), null ); + segmentSize = FileUtils.getFileSize(segmentDirectory); } finally { incrementalIndex.close(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index fd2b68ef39e..3cabeb6c674 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -42,6 +42,7 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputRowParser; @@ -293,7 +294,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest return new InputSourceReader() { @Override - public CloseableIterator read() + public CloseableIterator read(InputStats inputStats) { return new CloseableIterator() { @@ -345,7 +346,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest return new InputSourceReader() { @Override - public CloseableIterator read() + public CloseableIterator read(InputStats inputStats) { final Iterator inputRowIterator = IDX_TASK_INPUT_ROWS.iterator(); return CloseableIterators.withEmptyBaggage(inputRowIterator); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSourceTest.java index 34464030727..cd714969e8c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSourceTest.java @@ -27,9 +27,11 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputStatsImpl; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; @@ -88,7 +90,8 @@ public class RecordSupplierInputSourceTest extends InitializedNullHandlingTest ); int read = 0; - try (CloseableIterator iterator = reader.read()) { + final InputStats inputStats = new InputStatsImpl(); + try (CloseableIterator iterator = reader.read(inputStats)) { for (; read < NUM_ROWS && iterator.hasNext(); read++) { final InputRow inputRow = iterator.next(); Assert.assertEquals(DateTimes.of(TIMESTAMP_STRING), inputRow.getTimestamp()); @@ -96,6 +99,7 @@ public class RecordSupplierInputSourceTest extends InitializedNullHandlingTest } } + Assert.assertTrue(inputStats.getProcessedBytes() > NUM_ROWS * supplier.getMinRowSize()); Assert.assertEquals(NUM_ROWS, read); Assert.assertTrue(supplier.isClosed()); } @@ -120,11 +124,13 @@ public class RecordSupplierInputSourceTest extends InitializedNullHandlingTest ); int read = 0; - try (CloseableIterator iterator = reader.read()) { + final InputStats inputStats = new InputStatsImpl(); + try (CloseableIterator iterator = reader.read(inputStats)) { for (; read < NUM_ROWS && iterator.hasNext(); read++) { iterator.next(); } } + Assert.assertEquals(0, inputStats.getProcessedBytes()); Assert.assertEquals(0, read); Assert.assertTrue(supplier.isClosed()); } @@ -255,5 +261,10 @@ public class RecordSupplierInputSourceTest extends InitializedNullHandlingTest { throw new UnsupportedOperationException(); } + + private long getMinRowSize() + { + return TIMESTAMP_STRING.length() + (NUM_COLS - 1) * STR_LEN; + } } } 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 e93510ad740..06179d5c495 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 @@ -31,6 +31,10 @@ import com.google.common.collect.ImmutableSet; import com.google.common.io.Files; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.client.cache.MapCache; +import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.ByteEntity; @@ -42,19 +46,36 @@ import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.discovery.DataNodeService; +import org.apache.druid.discovery.DruidNodeAnnouncer; +import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.SegmentCacheManagerFactory; +import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; +import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.common.actions.TaskActionToolbox; +import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; +import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.MetadataTaskStorage; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; +import org.apache.druid.indexing.test.TestDataSegmentKiller; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -62,9 +83,16 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.metrics.MonitorScheduler; +import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.EntryExistsException; +import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.query.DirectQueryProcessingPool; import org.apache.druid.query.Druids; import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -77,16 +105,32 @@ import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.column.DictionaryEncodedColumn; +import org.apache.druid.segment.handoff.SegmentHandoffNotifier; +import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; +import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.join.NoopJoinableFactory; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.loading.LocalDataSegmentPusher; +import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.realtime.appenderator.StreamAppenderator; +import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.DataSegment; import org.apache.druid.utils.CompressionUtils; import org.assertj.core.api.Assertions; +import org.easymock.EasyMock; import org.easymock.EasyMockSupport; import org.joda.time.Interval; import org.junit.Assert; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.lang.reflect.InvocationTargetException; @@ -97,13 +141,22 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; +import java.util.concurrent.Executor; import java.util.stream.Collectors; -public class SeekableStreamIndexTaskTestBase extends EasyMockSupport +public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport { + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Rule + public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); + protected static final ObjectMapper OBJECT_MAPPER; protected static final DataSchema OLD_DATA_SCHEMA; protected static final DataSchema NEW_DATA_SCHEMA = new DataSchema( @@ -147,6 +200,7 @@ public class SeekableStreamIndexTaskTestBase extends EasyMockSupport protected TaskStorage taskStorage; protected TaskLockbox taskLockbox; protected IndexerMetadataStorageCoordinator metadataStorageCoordinator; + protected final Set checkpointRequestsHash = new HashSet<>(); static { OBJECT_MAPPER = new TestUtils().getTestObjectMapper(); @@ -494,6 +548,167 @@ public class SeekableStreamIndexTaskTestBase extends EasyMockSupport metadataStorageCoordinator = null; } + protected void verifyTaskMetrics( + SeekableStreamIndexTask task, + RowIngestionMetersTotals expectedTotals + ) + { + Assert.assertEquals(expectedTotals, task.getRunner().getRowIngestionMeters().getTotals()); + } + + protected abstract QueryRunnerFactoryConglomerate makeQueryRunnerConglomerate(); + + protected void makeToolboxFactory(TestUtils testUtils, ServiceEmitter emitter, boolean doHandoff) + throws IOException + { + final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); + directory = tempFolder.newFolder(); + final TaskConfig taskConfig = new TaskConfig( + new File(directory, "baseDir").getPath(), + new File(directory, "baseTaskDir").getPath(), + null, + 50000, + null, + true, + null, + null, + null, + false, + false, + TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name(), + null, + false + ); + final TestDerbyConnector derbyConnector = derby.getConnector(); + derbyConnector.createDataSourceTable(); + derbyConnector.createPendingSegmentsTable(); + derbyConnector.createSegmentTable(); + derbyConnector.createRulesTable(); + derbyConnector.createConfigTable(); + derbyConnector.createTaskTables(); + derbyConnector.createAuditTable(); + taskStorage = new MetadataTaskStorage( + derbyConnector, + new TaskStorageConfig(null), + new DerbyMetadataStorageActionHandlerFactory( + derbyConnector, + derby.metadataTablesConfigSupplier().get(), + objectMapper + ) + ); + metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( + objectMapper, + derby.metadataTablesConfigSupplier().get(), + derbyConnector + ); + taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( + taskLockbox, + taskStorage, + metadataStorageCoordinator, + emitter, + new SupervisorManager(null) + { + @Override + public boolean checkPointDataSourceMetadata( + String supervisorId, + int taskGroupId, + @Nullable DataSourceMetadata previousDataSourceMetadata + ) + { + // log.info("Adding checkpoint hash to the set"); + checkpointRequestsHash.add( + Objects.hash( + supervisorId, + taskGroupId, + previousDataSourceMetadata + ) + ); + return true; + } + }, + objectMapper + ); + final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( + taskStorage, + taskActionToolbox, + new TaskAuditLogConfig(false) + ); + final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier() + { + @Override + public boolean registerSegmentHandoffCallback( + SegmentDescriptor descriptor, + Executor exec, + Runnable handOffRunnable + ) + { + if (doHandoff) { + // Simulate immediate handoff + exec.execute(handOffRunnable); + } + return true; + } + + @Override + public void start() + { + //Noop + } + + @Override + public void close() + { + //Noop + } + }; + final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); + dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); + dataSegmentPusherConfig.zip = true; + final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig); + + toolboxFactory = new TaskToolboxFactory( + taskConfig, + null, // taskExecutorNode + taskActionClientFactory, + emitter, + dataSegmentPusher, + new TestDataSegmentKiller(), + null, // DataSegmentMover + null, // DataSegmentArchiver + new TestDataSegmentAnnouncer(), + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), + handoffNotifierFactory, + this::makeQueryRunnerConglomerate, + DirectQueryProcessingPool.INSTANCE, + NoopJoinableFactory.INSTANCE, + () -> EasyMock.createMock(MonitorScheduler.class), + new SegmentCacheManagerFactory(objectMapper), + objectMapper, + testUtils.getTestIndexIO(), + MapCache.create(1024), + new CacheConfig(), + new CachePopulatorStats(), + testUtils.getIndexMergerV9Factory(), + EasyMock.createNiceMock(DruidNodeAnnouncer.class), + EasyMock.createNiceMock(DruidNode.class), + new LookupNodeService("tier"), + new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), + new SingleFileTaskReportFileWriter(reportsFile), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + new NoopChatHandlerProvider(), + testUtils.getRowIngestionMetersFactory(), + new TestAppenderatorsManager(), + new NoopOverlordClient(), + null, + null, + null, + null, + "1" + ); + } + protected class SegmentDescriptorAndExpectedDim1Values { final SegmentDescriptor segmentDescriptor; diff --git a/processing/src/main/java/org/apache/druid/segment/generator/DataGenerator.java b/processing/src/main/java/org/apache/druid/segment/generator/DataGenerator.java index a78d8bbd716..31578093e77 100644 --- a/processing/src/main/java/org/apache/druid/segment/generator/DataGenerator.java +++ b/processing/src/main/java/org/apache/druid/segment/generator/DataGenerator.java @@ -99,8 +99,7 @@ public class DataGenerator for (ColumnValueGenerator generator : columnGenerators) { event.put(generator.getSchema().getName(), generator.generateRowValue()); } - MapBasedInputRow row = new MapBasedInputRow(nextTimestamp(), dimensionNames, event); - return row; + return new MapBasedInputRow(nextTimestamp(), dimensionNames, event); } /** diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java index 3ce11386c94..bff4f2e6de3 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java @@ -29,7 +29,7 @@ import java.util.Map; */ public class NoopRowIngestionMeters implements RowIngestionMeters { - private static final RowIngestionMetersTotals EMPTY_TOTALS = new RowIngestionMetersTotals(0, 0, 0, 0); + private static final RowIngestionMetersTotals EMPTY_TOTALS = new RowIngestionMetersTotals(0, 0, 0, 0, 0); @Override public long getProcessed() diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java index efefca7fa08..81c128744e7 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.incremental; +import org.apache.druid.data.input.InputStats; import org.apache.druid.guice.annotations.ExtensionPoint; import java.util.Map; @@ -30,12 +31,13 @@ import java.util.Map; * RowIngestionMeters to avoid unnecessary overhead from maintaining these moving averages. */ @ExtensionPoint -public interface RowIngestionMeters +public interface RowIngestionMeters extends InputStats { String BUILD_SEGMENTS = "buildSegments"; String DETERMINE_PARTITIONS = "determinePartitions"; String PROCESSED = "processed"; + String PROCESSED_BYTES = "processedBytes"; String PROCESSED_WITH_ERROR = "processedWithError"; String UNPARSEABLE = "unparseable"; String THROWN_AWAY = "thrownAway"; @@ -43,6 +45,18 @@ public interface RowIngestionMeters long getProcessed(); void incrementProcessed(); + @Override + default void incrementProcessedBytes(long incrementByValue) + { + + } + + @Override + default long getProcessedBytes() + { + return 0; + } + long getProcessedWithError(); void incrementProcessedWithError(); diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java index 89bac759199..2002bb24ac0 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java @@ -27,6 +27,7 @@ import java.util.Objects; public class RowIngestionMetersTotals { private final long processed; + private final long processedBytes; private final long processedWithError; private final long thrownAway; private final long unparseable; @@ -34,12 +35,14 @@ public class RowIngestionMetersTotals @JsonCreator public RowIngestionMetersTotals( @JsonProperty("processed") long processed, + @JsonProperty("processedBytes") long processedBytes, @JsonProperty("processedWithError") long processedWithError, @JsonProperty("thrownAway") long thrownAway, @JsonProperty("unparseable") long unparseable ) { this.processed = processed; + this.processedBytes = processedBytes; this.processedWithError = processedWithError; this.thrownAway = thrownAway; this.unparseable = unparseable; @@ -51,6 +54,12 @@ public class RowIngestionMetersTotals return processed; } + @JsonProperty + public long getProcessedBytes() + { + return processedBytes; + } + @JsonProperty public long getProcessedWithError() { @@ -80,6 +89,7 @@ public class RowIngestionMetersTotals } RowIngestionMetersTotals that = (RowIngestionMetersTotals) o; return processed == that.processed + && processedBytes == that.processedBytes && processedWithError == that.processedWithError && thrownAway == that.thrownAway && unparseable == that.unparseable; @@ -88,7 +98,7 @@ public class RowIngestionMetersTotals @Override public int hashCode() { - return Objects.hash(processed, processedWithError, thrownAway, unparseable); + return Objects.hash(processed, processedBytes, processedWithError, thrownAway, unparseable); } @Override @@ -96,6 +106,7 @@ public class RowIngestionMetersTotals { return "RowIngestionMetersTotals{" + "processed=" + processed + + ", processedBytes=" + processedBytes + ", processedWithError=" + processedWithError + ", thrownAway=" + thrownAway + ", unparseable=" + unparseable + diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/MutableRowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java similarity index 81% rename from processing/src/main/java/org/apache/druid/segment/incremental/MutableRowIngestionMeters.java rename to processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java index 0888c19c594..10293e4e24a 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/MutableRowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java @@ -21,20 +21,13 @@ package org.apache.druid.segment.incremental; import java.util.Map; -public class MutableRowIngestionMeters implements RowIngestionMeters +public class SimpleRowIngestionMeters implements RowIngestionMeters { private long processed; private long processedWithError; private long unparseable; private long thrownAway; - - public MutableRowIngestionMeters() - { - this.processed = 0; - this.processedWithError = 0; - this.unparseable = 0; - this.thrownAway = 0; - } + private long processedBytes; @Override public long getProcessed() @@ -48,6 +41,18 @@ public class MutableRowIngestionMeters implements RowIngestionMeters processed++; } + @Override + public void incrementProcessedBytes(long increment) + { + processedBytes += increment; + } + + @Override + public long getProcessedBytes() + { + return processedBytes; + } + @Override public long getProcessedWithError() { @@ -87,7 +92,13 @@ public class MutableRowIngestionMeters implements RowIngestionMeters @Override public RowIngestionMetersTotals getTotals() { - return new RowIngestionMetersTotals(processed, processedWithError, thrownAway, unparseable); + return new RowIngestionMetersTotals( + processed, + processedBytes, + processedWithError, + thrownAway, + unparseable + ); } @Override @@ -102,5 +113,6 @@ public class MutableRowIngestionMeters implements RowIngestionMeters this.processedWithError += rowIngestionMetersTotals.getProcessedWithError(); this.unparseable += rowIngestionMetersTotals.getUnparseable(); this.thrownAway += rowIngestionMetersTotals.getThrownAway(); + this.processedBytes += rowIngestionMetersTotals.getProcessedBytes(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputSourceReader.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputSourceReader.java index 4039cbdb6e2..fe1353c32d3 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputSourceReader.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputSourceReader.java @@ -22,6 +22,7 @@ package org.apache.druid.segment.transform; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputStats; import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.IOException; @@ -38,9 +39,9 @@ public class TransformingInputSourceReader implements InputSourceReader } @Override - public CloseableIterator read() throws IOException + public CloseableIterator read(InputStats inputStats) throws IOException { - return delegate.read().map(transformer::transform); + return delegate.read(inputStats).map(transformer::transform); } @Override diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java b/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java similarity index 50% rename from processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java rename to processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java index 3fd14e0316e..02b1d290003 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java @@ -19,72 +19,51 @@ package org.apache.druid.segment.incremental; -import java.util.Map; - -public class SimpleRowIngestionMeters implements RowIngestionMeters +/** + * Utility class to build {@link RowIngestionMetersTotals}, used in tests. + */ +public class RowMeters { - private long processed; + private long processedBytes; private long processedWithError; private long unparseable; private long thrownAway; - @Override - public long getProcessed() + /** + * Creates a new {@link RowMeters}, that can be used to build an instance of + * {@link RowIngestionMetersTotals}. + */ + public static RowMeters with() { - return processed; + return new RowMeters(); } - @Override - public void incrementProcessed() + public RowMeters bytes(long processedBytes) { - processed++; + this.processedBytes = processedBytes; + return this; } - @Override - public long getProcessedWithError() + public RowMeters errors(long processedWithError) { - return processedWithError; + this.processedWithError = processedWithError; + return this; } - @Override - public void incrementProcessedWithError() + public RowMeters unparseable(long unparseable) { - processedWithError++; + this.unparseable = unparseable; + return this; } - @Override - public long getUnparseable() + public RowMeters thrownAway(long thrownAway) { - return unparseable; + this.thrownAway = thrownAway; + return this; } - @Override - public void incrementUnparseable() + public RowIngestionMetersTotals totalProcessed(long processed) { - unparseable++; - } - - @Override - public long getThrownAway() - { - return thrownAway; - } - - @Override - public void incrementThrownAway() - { - thrownAway++; - } - - @Override - public RowIngestionMetersTotals getTotals() - { - return new RowIngestionMetersTotals(processed, processedWithError, thrownAway, unparseable); - } - - @Override - public Map getMovingAverages() - { - throw new UnsupportedOperationException(); + return new RowIngestionMetersTotals(processed, processedBytes, processedWithError, thrownAway, unparseable); } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/MutableRowIngestionMetersTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java similarity index 58% rename from processing/src/test/java/org/apache/druid/segment/incremental/MutableRowIngestionMetersTest.java rename to processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java index 5d32c5994b2..5f46129c170 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/MutableRowIngestionMetersTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java @@ -22,25 +22,26 @@ package org.apache.druid.segment.incremental; import org.junit.Assert; import org.junit.Test; -public class MutableRowIngestionMetersTest +public class SimpleRowIngestionMetersTest { @Test public void testIncrement() { - MutableRowIngestionMeters mutableRowIngestionMeters = new MutableRowIngestionMeters(); - mutableRowIngestionMeters.incrementProcessed(); - mutableRowIngestionMeters.incrementProcessedWithError(); - mutableRowIngestionMeters.incrementUnparseable(); - mutableRowIngestionMeters.incrementThrownAway(); - Assert.assertEquals(mutableRowIngestionMeters.getTotals(), new RowIngestionMetersTotals(1, 1, 1, 1)); + SimpleRowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); + rowIngestionMeters.incrementProcessed(); + rowIngestionMeters.incrementProcessedBytes(5); + rowIngestionMeters.incrementProcessedWithError(); + rowIngestionMeters.incrementUnparseable(); + rowIngestionMeters.incrementThrownAway(); + Assert.assertEquals(rowIngestionMeters.getTotals(), new RowIngestionMetersTotals(1, 5, 1, 1, 1)); } @Test public void testAddRowIngestionMetersTotals() { - MutableRowIngestionMeters mutableRowIngestionMeters = new MutableRowIngestionMeters(); - RowIngestionMetersTotals rowIngestionMetersTotals = new RowIngestionMetersTotals(10, 1, 0, 1); - mutableRowIngestionMeters.addRowIngestionMetersTotals(rowIngestionMetersTotals); - Assert.assertEquals(mutableRowIngestionMeters.getTotals(), rowIngestionMetersTotals); + SimpleRowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); + RowIngestionMetersTotals rowIngestionMetersTotals = new RowIngestionMetersTotals(10, 0, 1, 0, 1); + rowIngestionMeters.addRowIngestionMetersTotals(rowIngestionMetersTotals); + Assert.assertEquals(rowIngestionMeters.getTotals(), rowIngestionMetersTotals); } } diff --git a/server/src/main/java/org/apache/druid/metadata/input/SqlReader.java b/server/src/main/java/org/apache/druid/metadata/input/SqlReader.java index 0a503eb87ab..27c1f612c8b 100644 --- a/server/src/main/java/org/apache/druid/metadata/input/SqlReader.java +++ b/server/src/main/java/org/apache/druid/metadata/input/SqlReader.java @@ -44,7 +44,7 @@ import java.util.Map; public class SqlReader extends IntermediateRowParsingReader> { private final InputRowSchema inputRowSchema; - private final SqlEntity source; + private final InputEntity source; private final File temporaryDirectory; private final ObjectMapper objectMapper; @@ -57,7 +57,7 @@ public class SqlReader extends IntermediateRowParsingReader> ) { this.inputRowSchema = inputRowSchema; - this.source = (SqlEntity) source; + this.source = source; this.temporaryDirectory = temporaryDirectory; this.objectMapper = objectMapper; } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index f89ae9b0ebd..a7c19ff978b 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -44,15 +44,11 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; -import org.skife.jdbi.v2.Handle; -import org.skife.jdbi.v2.tweak.HandleCallback; import java.sql.ResultSet; -import java.sql.SQLException; import java.util.HashMap; import java.util.HashSet; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Random; import java.util.UUID; @@ -553,18 +549,14 @@ public class SQLMetadataStorageActionHandlerTest private Integer getUnmigratedTaskCount() { return handler.getConnector().retryWithHandle( - new HandleCallback() - { - @Override - public Integer withHandle(Handle handle) throws SQLException - { - String sql = String.format(Locale.ENGLISH, - "SELECT COUNT(*) FROM %s WHERE type is NULL or group_id is NULL", - entryTable); - ResultSet resultSet = handle.getConnection().createStatement().executeQuery(sql); - resultSet.next(); - return resultSet.getInt(1); - } + handle -> { + String sql = StringUtils.format( + "SELECT COUNT(*) FROM %s WHERE type is NULL or group_id is NULL", + entryTable + ); + ResultSet resultSet = handle.getConnection().createStatement().executeQuery(sql); + resultSet.next(); + return resultSet.getInt(1); } ); } diff --git a/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java b/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java index 46a171b50c9..ccd71cfaff5 100644 --- a/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java +++ b/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java @@ -23,6 +23,8 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.io.IOUtils; import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.segment.TestHelper; import org.junit.Assert; @@ -35,6 +37,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; +import java.util.Collections; public class SqlEntityTest { @@ -47,10 +50,6 @@ public class SqlEntityTest String VALID_SQL = "SELECT timestamp,a,b FROM FOOS_TABLE"; String INVALID_SQL = "DONT SELECT timestamp,a,b FROM FOOS_TABLE"; - String resultJson = "[{\"a\":\"0\"," - + "\"b\":\"0\"," - + "\"timestamp\":\"2011-01-12T00:00:00.000Z\"" - + "}]"; @Before public void setUp() @@ -65,11 +64,8 @@ public class SqlEntityTest { derbyConnector = derbyConnectorRule.getConnector(); SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - testUtils.createAndUpdateTable(TABLE_NAME_1, 1); - File tmpFile = File.createTempFile( - "testQueryResults", - "" - ); + final InputRow expectedRow = testUtils.createTableWithRows(TABLE_NAME_1, 1).get(0); + File tmpFile = File.createTempFile("testQueryResults", ""); InputEntity.CleanableFile queryResult = SqlEntity.openCleanableFile( VALID_SQL, testUtils.getDerbyFirehoseConnector(), @@ -79,56 +75,34 @@ public class SqlEntityTest ); InputStream queryInputStream = new FileInputStream(queryResult.file()); String actualJson = IOUtils.toString(queryInputStream, StandardCharsets.UTF_8); - - Assert.assertEquals(actualJson, resultJson); + String expectedJson = mapper.writeValueAsString( + Collections.singletonList(((MapBasedInputRow) expectedRow).getEvent()) + ); + Assert.assertEquals(actualJson, expectedJson); testUtils.dropTable(TABLE_NAME_1); } - @Test(expected = IOException.class) - public void testFailOnInvalidQuery() throws IOException - { - derbyConnector = derbyConnectorRule.getConnector(); - SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - testUtils.createAndUpdateTable(TABLE_NAME_1, 1); - File tmpFile = File.createTempFile( - "testQueryResults", - "" - ); - InputEntity.CleanableFile queryResult = SqlEntity.openCleanableFile( - INVALID_SQL, - testUtils.getDerbyFirehoseConnector(), - mapper, - true, - tmpFile - ); - - Assert.assertTrue(tmpFile.exists()); - } - @Test public void testFileDeleteOnInvalidQuery() throws IOException { - //The test parameters here are same as those used for testFailOnInvalidQuery(). - //The only difference is that this test checks if the temporary file is deleted upon failure. derbyConnector = derbyConnectorRule.getConnector(); SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - testUtils.createAndUpdateTable(TABLE_NAME_1, 1); - File tmpFile = File.createTempFile( - "testQueryResults", - "" + testUtils.createTableWithRows(TABLE_NAME_1, 1); + File tmpFile = File.createTempFile("testQueryResults", ""); + Assert.assertTrue(tmpFile.exists()); + + Assert.assertThrows( + IOException.class, + () -> SqlEntity.openCleanableFile( + INVALID_SQL, + testUtils.getDerbyFirehoseConnector(), + mapper, + true, + tmpFile + ) ); - try { - SqlEntity.openCleanableFile( - INVALID_SQL, - testUtils.getDerbyFirehoseConnector(), - mapper, - true, - tmpFile - ); - } - // Lets catch the exception so as to test temporary file deletion. - catch (IOException e) { - Assert.assertFalse(tmpFile.exists()); - } + + // Verify that the temporary file is cleaned up + Assert.assertFalse(tmpFile.exists()); } } diff --git a/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java b/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java index e3659517aeb..a83dec676bf 100644 --- a/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java +++ b/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java @@ -23,8 +23,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.commons.dbcp2.BasicDataSource; import org.apache.druid.data.input.ColumnsFilter; @@ -34,11 +34,11 @@ import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.Row; +import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputStatsImpl; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; @@ -57,7 +57,6 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Comparator; import java.util.List; import java.util.Objects; import java.util.Set; @@ -67,14 +66,8 @@ import java.util.stream.Stream; public class SqlInputSourceTest { private static final List FIREHOSE_TMP_DIRS = new ArrayList<>(); - private final String TABLE_NAME_1 = "FOOS_TABLE_1"; - private final String TABLE_NAME_2 = "FOOS_TABLE_2"; - - private final List SQLLIST1 = ImmutableList.of("SELECT timestamp,a,b FROM FOOS_TABLE_1"); - private final List SQLLIST2 = ImmutableList.of( - "SELECT timestamp,a,b FROM FOOS_TABLE_1", - "SELECT timestamp,a,b FROM FOOS_TABLE_2" - ); + private final String TABLE_1 = "FOOS_TABLE_1"; + private final String TABLE_2 = "FOOS_TABLE_2"; private static final InputRowSchema INPUT_ROW_SCHEMA = new InputRowSchema( new TimestampSpec("timestamp", "auto", null), @@ -83,6 +76,7 @@ public class SqlInputSourceTest ), ColumnsFilter.all() ); + @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); private final ObjectMapper mapper = TestHelper.makeSmileMapper(); @@ -104,25 +98,6 @@ public class SqlInputSourceTest } } - private void assertResult(List rows, List sqls) - { - Assert.assertEquals(10 * sqls.size(), rows.size()); - rows.sort(Comparator.comparing(Row::getTimestamp) - .thenComparingInt(r -> Integer.valueOf(r.getDimension("a").get(0))) - .thenComparingInt(r -> Integer.valueOf(r.getDimension("b").get(0)))); - int rowCount = 0; - for (int i = 0; i < 10; i++) { - for (int j = 0; j < sqls.size(); j++) { - final Row row = rows.get(rowCount); - String timestampSt = StringUtils.format("2011-01-12T00:0%s:00.000Z", i); - Assert.assertEquals(timestampSt, row.getTimestamp().toString()); - Assert.assertEquals(i, Integer.valueOf(row.getDimension("a").get(0)).intValue()); - Assert.assertEquals(i, Integer.valueOf(row.getDimension("b").get(0)).intValue()); - rowCount++; - } - } - } - private File createFirehoseTmpDir(String dirSuffix) throws IOException { final File firehoseTempDir = File.createTempFile( @@ -141,7 +116,8 @@ public class SqlInputSourceTest mapper.registerSubtypes(TestSerdeFirehoseConnector.class); final SqlInputSourceTest.TestSerdeFirehoseConnector testSerdeFirehoseConnector = new SqlInputSourceTest.TestSerdeFirehoseConnector( new MetadataStorageConnectorConfig()); - final SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST1, true, testSerdeFirehoseConnector, mapper); + final SqlInputSource sqlInputSource = + new SqlInputSource(SqlTestUtils.selectFrom(TABLE_1), true, testSerdeFirehoseConnector, mapper); final String valueString = mapper.writeValueAsString(sqlInputSource); final SqlInputSource inputSourceFromJson = mapper.readValue(valueString, SqlInputSource.class); Assert.assertEquals(sqlInputSource, inputSourceFromJson); @@ -152,17 +128,26 @@ public class SqlInputSourceTest { derbyConnector = derbyConnectorRule.getConnector(); SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - testUtils.createAndUpdateTable(TABLE_NAME_1, 10); + final List expectedRows = testUtils.createTableWithRows(TABLE_1, 10); final File tempDir = createFirehoseTmpDir("testSingleSplit"); - SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST1, true, testUtils.getDerbyFirehoseConnector(), mapper); + final InputStats inputStats = new InputStatsImpl(); + + SqlInputSource sqlInputSource = new SqlInputSource( + SqlTestUtils.selectFrom(TABLE_1), + true, + testUtils.getDerbyFirehoseConnector(), + mapper + ); InputSourceReader sqlReader = sqlInputSource.fixedFormatReader(INPUT_ROW_SCHEMA, tempDir); - CloseableIterator resultIterator = sqlReader.read(); - final List rows = new ArrayList<>(); - while (resultIterator.hasNext()) { - rows.add(resultIterator.next()); - } - assertResult(rows, SQLLIST1); - testUtils.dropTable(TABLE_NAME_1); + CloseableIterator resultIterator = sqlReader.read(inputStats); + final List rows = Lists.newArrayList(resultIterator); + + // Records for each split are written to a temp file as a json array + // file size = 1B (array open char) + 10 records * 60B (including trailing comma) + Assert.assertEquals(601, inputStats.getProcessedBytes()); + Assert.assertEquals(expectedRows, rows); + + testUtils.dropTable(TABLE_1); } @@ -171,19 +156,27 @@ public class SqlInputSourceTest { derbyConnector = derbyConnectorRule.getConnector(); SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - testUtils.createAndUpdateTable(TABLE_NAME_1, 10); - testUtils.createAndUpdateTable(TABLE_NAME_2, 10); + final List expectedRowsTable1 = testUtils.createTableWithRows(TABLE_1, 10); + final List expectedRowsTable2 = testUtils.createTableWithRows(TABLE_2, 10); final File tempDir = createFirehoseTmpDir("testMultipleSplit"); - SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST2, true, testUtils.getDerbyFirehoseConnector(), mapper); + SqlInputSource sqlInputSource = new SqlInputSource( + SqlTestUtils.selectFrom(TABLE_1, TABLE_2), + true, + testUtils.getDerbyFirehoseConnector(), + mapper + ); + + final InputStats inputStats = new InputStatsImpl(); InputSourceReader sqlReader = sqlInputSource.fixedFormatReader(INPUT_ROW_SCHEMA, tempDir); - CloseableIterator resultIterator = sqlReader.read(); - final List rows = new ArrayList<>(); - while (resultIterator.hasNext()) { - rows.add(resultIterator.next()); - } - assertResult(rows, SQLLIST2); - testUtils.dropTable(TABLE_NAME_1); - testUtils.dropTable(TABLE_NAME_2); + CloseableIterator resultIterator = sqlReader.read(inputStats); + final List rows = Lists.newArrayList(resultIterator); + + Assert.assertEquals(expectedRowsTable1, rows.subList(0, 10)); + Assert.assertEquals(expectedRowsTable2, rows.subList(10, 20)); + Assert.assertEquals(1202, inputStats.getProcessedBytes()); + + testUtils.dropTable(TABLE_1); + testUtils.dropTable(TABLE_2); } @Test @@ -191,10 +184,12 @@ public class SqlInputSourceTest { derbyConnector = derbyConnectorRule.getConnector(); SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST2, true, testUtils.getDerbyFirehoseConnector(), mapper); + final List sqls = SqlTestUtils.selectFrom(TABLE_1, TABLE_2); + SqlInputSource sqlInputSource = + new SqlInputSource(sqls, true, testUtils.getDerbyFirehoseConnector(), mapper); InputFormat inputFormat = EasyMock.createMock(InputFormat.class); Stream> sqlSplits = sqlInputSource.createSplits(inputFormat, null); - Assert.assertEquals(SQLLIST2, sqlSplits.map(InputSplit::get).collect(Collectors.toList())); + Assert.assertEquals(sqls, sqlSplits.map(InputSplit::get).collect(Collectors.toList())); Assert.assertEquals(2, sqlInputSource.estimateNumSplits(inputFormat, null)); } @@ -203,22 +198,23 @@ public class SqlInputSourceTest { derbyConnector = derbyConnectorRule.getConnector(); SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - testUtils.createAndUpdateTable(TABLE_NAME_1, 10); + final List expectedRows = testUtils.createTableWithRows(TABLE_1, 10); try { final File tempDir = createFirehoseTmpDir("testSingleSplit"); - SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST1, true, testUtils.getDerbyFirehoseConnector(), mapper); + SqlInputSource sqlInputSource = + new SqlInputSource(SqlTestUtils.selectFrom(TABLE_1), true, testUtils.getDerbyFirehoseConnector(), mapper); InputSourceReader sqlReader = sqlInputSource.fixedFormatReader(INPUT_ROW_SCHEMA, tempDir); CloseableIterator resultIterator = sqlReader.sample(); - final List rows = new ArrayList<>(); + final List rows = new ArrayList<>(); while (resultIterator.hasNext()) { InputRowListPlusRawValues row = resultIterator.next(); Assert.assertNull(row.getParseException()); - rows.add(row); + rows.addAll(row.getInputRows()); } - assertResult(rows.stream().flatMap(r -> r.getInputRows().stream()).collect(Collectors.toList()), SQLLIST1); + Assert.assertEquals(expectedRows, rows); } finally { - testUtils.dropTable(TABLE_NAME_1); + testUtils.dropTable(TABLE_1); } } diff --git a/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java b/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java index 462df2321a1..2e99bfbb301 100644 --- a/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java +++ b/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java @@ -23,6 +23,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.commons.dbcp2.BasicDataSource; +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.StringUtils; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; @@ -33,7 +36,14 @@ import org.skife.jdbi.v2.Batch; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.tweak.HandleCallback; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; public class SqlTestUtils { @@ -88,7 +98,7 @@ public class SqlTestUtils } } - public void createAndUpdateTable(final String tableName, int numEntries) + public List createTableWithRows(final String tableName, int numEntries) { derbyConnector.createTable( tableName, @@ -104,20 +114,30 @@ public class SqlTestUtils ) ); + final List rowsToCreate = IntStream.range(0, numEntries).mapToObj(i -> { + final String timestamp = StringUtils.format("2011-01-12T00:%02d:00.000Z", i); + final Map event = new LinkedHashMap<>(); + event.put("a", "a " + i); + event.put("b", "b" + i); + event.put("timestamp", timestamp); + return new MapBasedInputRow(DateTimes.of(timestamp), Arrays.asList("timestamp", "a", "b"), event); + }).collect(Collectors.toList()); + derbyConnector.getDBI().withHandle( (handle) -> { Batch batch = handle.createBatch(); - for (int i = 0; i < numEntries; i++) { - String timestampSt = StringUtils.format("2011-01-12T00:0%s:00.000Z", i); - batch.add(StringUtils.format("INSERT INTO %1$s (timestamp, a, b) VALUES ('%2$s', '%3$s', '%4$s')", - tableName, timestampSt, - i, i + for (InputRow row : rowsToCreate) { + batch.add(StringUtils.format( + "INSERT INTO %1$s (timestamp, a, b) VALUES ('%2$s', '%3$s', '%4$s')", + tableName, row.getTimestamp(), row.getDimension("a").get(0), row.getDimension("b").get(0) )); } batch.execute(); return null; } ); + + return rowsToCreate; } public void dropTable(final String tableName) @@ -135,4 +155,17 @@ public class SqlTestUtils { return derbyFirehoseConnector; } + + /** + * Builds a {@code SELECT timestamp, a, b FROM tableName} query for each of + * the given tables. + */ + public static List selectFrom(String... tableNames) + { + final List selects = new ArrayList<>(); + for (String tableName : tableNames) { + selects.add(StringUtils.format("SELECT timestamp, a, b FROM %s", tableName)); + } + return selects; + } } diff --git a/server/src/test/java/org/apache/druid/segment/loading/CacheTestSegmentCacheManager.java b/server/src/test/java/org/apache/druid/segment/loading/NoopSegmentCacheManager.java similarity index 75% rename from server/src/test/java/org/apache/druid/segment/loading/CacheTestSegmentCacheManager.java rename to server/src/test/java/org/apache/druid/segment/loading/NoopSegmentCacheManager.java index d546e0eb9f2..8c2ef05d35d 100644 --- a/server/src/test/java/org/apache/druid/segment/loading/CacheTestSegmentCacheManager.java +++ b/server/src/test/java/org/apache/druid/segment/loading/NoopSegmentCacheManager.java @@ -19,27 +19,22 @@ package org.apache.druid.segment.loading; -import org.apache.druid.java.util.common.MapUtils; import org.apache.druid.timeline.DataSegment; import java.io.File; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; import java.util.concurrent.ExecutorService; /** - * + * Test implementation of {@link SegmentCacheManager} which throws an + * {@link UnsupportedOperationException} on invocation of any method. */ -public class CacheTestSegmentCacheManager implements SegmentCacheManager +public class NoopSegmentCacheManager implements SegmentCacheManager { - private final Set segmentsInTrash = new HashSet<>(); @Override public boolean isSegmentCached(DataSegment segment) { - Map loadSpec = segment.getLoadSpec(); - return new File(MapUtils.getString(loadSpec, "cacheDir")).exists(); + throw new UnsupportedOperationException(); } @Override @@ -63,12 +58,7 @@ public class CacheTestSegmentCacheManager implements SegmentCacheManager @Override public void cleanup(DataSegment segment) { - segmentsInTrash.add(segment); - } - - public Set getSegmentsInTrash() - { - return segmentsInTrash; + throw new UnsupportedOperationException(); } @Override diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java index bd92c16c96b..7893aefd2a0 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java @@ -20,8 +20,8 @@ package org.apache.druid.segment.realtime.firehose; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.Row; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputRowParser; @@ -29,7 +29,6 @@ import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.metadata.input.SqlTestUtils; import org.apache.druid.segment.TestHelper; @@ -44,21 +43,14 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Comparator; import java.util.List; public class SqlFirehoseFactoryTest { private static final List FIREHOSE_TMP_DIRS = new ArrayList<>(); private static File TEST_DIR; - private final String TABLE_NAME_1 = "FOOS_TABLE_1"; - private final String TABLE_NAME_2 = "FOOS_TABLE_2"; - - private final List SQLLIST1 = ImmutableList.of("SELECT timestamp,a,b FROM FOOS_TABLE_1"); - private final List SQLLIST2 = ImmutableList.of( - "SELECT timestamp,a,b FROM FOOS_TABLE_1", - "SELECT timestamp,a,b FROM FOOS_TABLE_2" - ); + private static final String TABLE_NAME_1 = "FOOS_TABLE_1"; + private static final String TABLE_NAME_2 = "FOOS_TABLE_2"; @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); @@ -93,25 +85,6 @@ public class SqlFirehoseFactoryTest } } - private void assertResult(List rows, List sqls) - { - Assert.assertEquals(10 * sqls.size(), rows.size()); - rows.sort(Comparator.comparing(Row::getTimestamp) - .thenComparingInt(r -> Integer.valueOf(r.getDimension("a").get(0))) - .thenComparingInt(r -> Integer.valueOf(r.getDimension("b").get(0)))); - int rowCount = 0; - for (int i = 0; i < 10; i++) { - for (int j = 0; j < sqls.size(); j++) { - final Row row = rows.get(rowCount); - String timestampSt = StringUtils.format("2011-01-12T00:0%s:00.000Z", i); - Assert.assertEquals(timestampSt, row.getTimestamp().toString()); - Assert.assertEquals(i, Integer.valueOf(row.getDimension("a").get(0)).intValue()); - Assert.assertEquals(i, Integer.valueOf(row.getDimension("b").get(0)).intValue()); - rowCount++; - } - } - } - private void assertNumRemainingCacheFiles(File firehoseTmpDir, int expectedNumFiles) { final String[] files = firehoseTmpDir.list(); @@ -136,10 +109,10 @@ public class SqlFirehoseFactoryTest { derbyConnector = derbyConnectorRule.getConnector(); SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - testUtils.createAndUpdateTable(TABLE_NAME_1, 10); + final List expectedRows = testUtils.createTableWithRows(TABLE_NAME_1, 10); final SqlFirehoseFactory factory = new SqlFirehoseFactory( - SQLLIST1, + SqlTestUtils.selectFrom(TABLE_NAME_1), 0L, 0L, 0L, @@ -157,7 +130,7 @@ public class SqlFirehoseFactoryTest } } - assertResult(rows, SQLLIST1); + Assert.assertEquals(expectedRows, rows); assertNumRemainingCacheFiles(firehoseTmpDir, 0); testUtils.dropTable(TABLE_NAME_1); } @@ -168,10 +141,10 @@ public class SqlFirehoseFactoryTest { derbyConnector = derbyConnectorRule.getConnector(); SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - testUtils.createAndUpdateTable(TABLE_NAME_1, 10); + final List expectedRows = testUtils.createTableWithRows(TABLE_NAME_1, 10); final SqlFirehoseFactory factory = new SqlFirehoseFactory( - SQLLIST1, + SqlTestUtils.selectFrom(TABLE_NAME_1), 0L, null, null, @@ -190,7 +163,7 @@ public class SqlFirehoseFactoryTest } } - assertResult(rows, SQLLIST1); + Assert.assertEquals(expectedRows, rows); assertNumRemainingCacheFiles(firehoseTmpDir, 0); testUtils.dropTable(TABLE_NAME_1); } @@ -201,12 +174,12 @@ public class SqlFirehoseFactoryTest { derbyConnector = derbyConnectorRule.getConnector(); SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - testUtils.createAndUpdateTable(TABLE_NAME_1, 10); - testUtils.createAndUpdateTable(TABLE_NAME_2, 10); + final List expectedRowsTable1 = testUtils.createTableWithRows(TABLE_NAME_1, 10); + final List expectedRowsTable2 = testUtils.createTableWithRows(TABLE_NAME_2, 10); final SqlFirehoseFactory factory = new SqlFirehoseFactory( - SQLLIST2, + SqlTestUtils.selectFrom(TABLE_NAME_1, TABLE_NAME_2), null, null, 0L, @@ -224,7 +197,8 @@ public class SqlFirehoseFactoryTest } } - assertResult(rows, SQLLIST2); + Assert.assertEquals(expectedRowsTable1, rows.subList(0, 10)); + Assert.assertEquals(expectedRowsTable2, rows.subList(10, 20)); assertNumRemainingCacheFiles(firehoseTmpDir, 2); testUtils.dropTable(TABLE_NAME_1); testUtils.dropTable(TABLE_NAME_2); diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index 31f39c0f28b..d6d4d2374df 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -26,14 +26,15 @@ import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.MapUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.loading.CacheTestSegmentCacheManager; import org.apache.druid.segment.loading.CacheTestSegmentLoader; -import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.loading.NoopSegmentCacheManager; +import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; @@ -84,13 +85,12 @@ public class SegmentLoadDropHandlerTest private TestStorageLocation testStorageLocation; private AtomicInteger announceCount; private ConcurrentSkipListSet segmentsAnnouncedByMe; - private CacheTestSegmentCacheManager segmentCacheManager; - private SegmentLoader segmentLoader; + private SegmentCacheManager segmentCacheManager; + private Set segmentsRemovedFromCache; private SegmentManager segmentManager; private List scheduledRunnable; private SegmentLoaderConfig segmentLoaderConfig; private SegmentLoaderConfig noAnnouncerSegmentLoaderConfig; - private SegmentLoaderConfig segmentLoaderConfigNoLocations; private ScheduledExecutorFactory scheduledExecutorFactory; private List locations; @@ -122,9 +122,24 @@ public class SegmentLoadDropHandlerTest scheduledRunnable = new ArrayList<>(); - segmentCacheManager = new CacheTestSegmentCacheManager(); - segmentLoader = new CacheTestSegmentLoader(); - segmentManager = new SegmentManager(segmentLoader); + segmentsRemovedFromCache = new HashSet<>(); + segmentCacheManager = new NoopSegmentCacheManager() + { + @Override + public boolean isSegmentCached(DataSegment segment) + { + Map loadSpec = segment.getLoadSpec(); + return new File(MapUtils.getString(loadSpec, "cacheDir")).exists(); + } + + @Override + public void cleanup(DataSegment segment) + { + segmentsRemovedFromCache.add(segment); + } + }; + + segmentManager = new SegmentManager(new CacheTestSegmentLoader()); segmentsAnnouncedByMe = new ConcurrentSkipListSet<>(); announceCount = new AtomicInteger(0); @@ -230,28 +245,6 @@ public class SegmentLoadDropHandlerTest } }; - segmentLoaderConfigNoLocations = new SegmentLoaderConfig() - { - @Override - public int getNumLoadingThreads() - { - return 5; - } - - @Override - public int getAnnounceIntervalMillis() - { - return 50; - } - - - @Override - public int getDropSegmentDelayMillis() - { - return 0; - } - }; - scheduledExecutorFactory = new ScheduledExecutorFactory() { @Override @@ -314,7 +307,7 @@ public class SegmentLoadDropHandlerTest } Assert.assertTrue(segmentsAnnouncedByMe.contains(segment)); - Assert.assertFalse("segment files shouldn't be deleted", segmentCacheManager.getSegmentsInTrash().contains(segment)); + Assert.assertFalse("segment files shouldn't be deleted", segmentsRemovedFromCache.contains(segment)); segmentLoadDropHandler.stop(); } @@ -353,7 +346,7 @@ public class SegmentLoadDropHandlerTest } Assert.assertTrue(segmentsAnnouncedByMe.contains(segment)); - Assert.assertFalse("segment files shouldn't be deleted", segmentCacheManager.getSegmentsInTrash().contains(segment)); + Assert.assertFalse("segment files shouldn't be deleted", segmentsRemovedFromCache.contains(segment)); segmentLoadDropHandler.stop(); }