From a109a4be9b9929381ccd855834a34b68e07c0c32 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Thu, 5 Apr 2018 11:26:42 -0700 Subject: [PATCH 01/31] This commit introduces a new tuning config called 'maxBytesInMemory' for ingestion tasks Currently a config called 'maxRowsInMemory' is present which affects how much memory gets used for indexing.If this value is not optimal for your JVM heap size, it could lead to OutOfMemoryError sometimes. A lower value will lead to frequent persists which might be bad for query performance and a higher value will limit number of persists but require more jvm heap space and could lead to OOM. 'maxBytesInMemory' is an attempt to solve this problem. It limits the total number of bytes kept in memory before persisting. * The default value is 1/3(Runtime.maxMemory()) * To maintain the current behaviour set 'maxBytesInMemory' to -1 * If both 'maxRowsInMemory' and 'maxBytesInMemory' are present, both of them will be respected i.e. the first one to go above threshold will trigger persist --- .../indexing/kafka/KafkaTuningConfig.java | 15 +++ .../kafka/supervisor/KafkaSupervisorSpec.java | 1 + .../KafkaSupervisorTuningConfig.java | 3 + .../indexing/kafka/KafkaIndexTaskTest.java | 4 +- .../indexing/kafka/KafkaTuningConfigTest.java | 2 +- .../kafka/supervisor/KafkaSupervisorTest.java | 2 +- .../RealtimeAppenderatorTuningConfig.java | 17 ++++ .../common/index/YeOldePlumberSchool.java | 1 + .../druid/indexing/common/task/IndexTask.java | 16 +++- ...penderatorDriverRealtimeIndexTaskTest.java | 1 + .../common/task/CompactionTaskTest.java | 1 + .../indexing/common/task/IndexTaskTest.java | 17 ++-- .../common/task/RealtimeIndexTaskTest.java | 1 + .../indexing/common/task/TaskSerdeTest.java | 38 +++++++- .../indexing/overlord/TaskLifecycleTest.java | 55 ++++++++++- .../io/druid/segment/DimensionIndexer.java | 8 ++ .../druid/segment/DoubleDimensionIndexer.java | 6 ++ .../druid/segment/FloatDimensionIndexer.java | 6 ++ .../druid/segment/LongDimensionIndexer.java | 6 ++ .../druid/segment/StringDimensionIndexer.java | 11 +++ .../segment/incremental/IncrementalIndex.java | 48 +++++++++- .../incremental/OffheapIncrementalIndex.java | 2 + .../incremental/OnheapIncrementalIndex.java | 60 ++++++++++-- .../OnheapIncrementalIndexBenchmark.java | 30 +++--- .../incremental/TimeAndDimsKeySizeTest.java | 94 +++++++++++++++++++ .../indexing/RealtimeTuningConfig.java | 18 ++++ .../appenderator/AppenderatorConfig.java | 2 + .../appenderator/AppenderatorImpl.java | 16 +++- .../realtime/plumber/RealtimePlumber.java | 2 + .../druid/segment/realtime/plumber/Sink.java | 22 ++++- .../segment/realtime/RealtimeManagerTest.java | 5 + .../appenderator/AppenderatorPlumberTest.java | 1 + .../appenderator/AppenderatorTest.java | 83 +++++++++++++++- .../appenderator/AppenderatorTester.java | 22 ++++- ...DefaultOfflineAppenderatorFactoryTest.java | 1 + .../plumber/RealtimePlumberSchoolTest.java | 4 + .../segment/realtime/plumber/SinkTest.java | 2 + .../cli/validate/DruidJsonValidatorTest.java | 1 + 38 files changed, 577 insertions(+), 47 deletions(-) create mode 100644 processing/src/test/java/io/druid/segment/incremental/TimeAndDimsKeySizeTest.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index 21ec1ed9f1ca..2b4ad4017e14 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -38,6 +38,7 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; private final int maxRowsInMemory; + private final long maxBytesInMemory; private final int maxRowsPerSegment; private final Period intermediatePersistPeriod; private final File basePersistDirectory; @@ -54,6 +55,7 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig @JsonCreator public KafkaTuningConfig( @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, @JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod, @JsonProperty("basePersistDirectory") @Nullable File basePersistDirectory, @@ -73,6 +75,7 @@ public KafkaTuningConfig( this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; this.maxRowsPerSegment = maxRowsPerSegment == null ? DEFAULT_MAX_ROWS_PER_SEGMENT : maxRowsPerSegment; + this.maxBytesInMemory = maxBytesInMemory == null ? defaults.getMaxBytesInMemory() : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaults.getIntermediatePersistPeriod() : intermediatePersistPeriod; @@ -98,6 +101,7 @@ public static KafkaTuningConfig copyOf(KafkaTuningConfig config) { return new KafkaTuningConfig( config.maxRowsInMemory, + config.maxBytesInMemory, config.maxRowsPerSegment, config.intermediatePersistPeriod, config.basePersistDirectory, @@ -119,6 +123,13 @@ public int getMaxRowsInMemory() return maxRowsInMemory; } + @Override + @JsonProperty + public long getMaxBytesInMemory() + { + return maxBytesInMemory; + } + @JsonProperty public int getMaxRowsPerSegment() { @@ -201,6 +212,7 @@ public KafkaTuningConfig withBasePersistDirectory(File dir) { return new KafkaTuningConfig( maxRowsInMemory, + maxBytesInMemory, maxRowsPerSegment, intermediatePersistPeriod, dir, @@ -227,6 +239,7 @@ public boolean equals(Object o) KafkaTuningConfig that = (KafkaTuningConfig) o; return maxRowsInMemory == that.maxRowsInMemory && maxRowsPerSegment == that.maxRowsPerSegment && + maxBytesInMemory == this.maxBytesInMemory && maxPendingPersists == that.maxPendingPersists && reportParseExceptions == that.reportParseExceptions && handoffConditionTimeout == that.handoffConditionTimeout && @@ -244,6 +257,7 @@ public int hashCode() return Objects.hash( maxRowsInMemory, maxRowsPerSegment, + maxBytesInMemory, intermediatePersistPeriod, basePersistDirectory, maxPendingPersists, @@ -262,6 +276,7 @@ public String toString() return "KafkaTuningConfig{" + "maxRowsInMemory=" + maxRowsInMemory + ", maxRowsPerSegment=" + maxRowsPerSegment + + ",maxBytesInMemory=" + maxBytesInMemory + ", intermediatePersistPeriod=" + intermediatePersistPeriod + ", basePersistDirectory=" + basePersistDirectory + ", maxPendingPersists=" + maxPendingPersists + diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index 1db4580c23f9..d3e89a925a65 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -90,6 +90,7 @@ public KafkaSupervisorSpec( null, null, null, + null, null ); this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index c6e5a3fe8d65..7ebf71415a78 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -40,6 +40,7 @@ public class KafkaSupervisorTuningConfig extends KafkaTuningConfig public KafkaSupervisorTuningConfig( @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -62,6 +63,7 @@ public KafkaSupervisorTuningConfig( { super( maxRowsInMemory, + maxBytesInMemory, maxRowsPerSegment, intermediatePersistPeriod, basePersistDirectory, @@ -125,6 +127,7 @@ public String toString() return "KafkaSupervisorTuningConfig{" + "maxRowsInMemory=" + getMaxRowsInMemory() + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + + ", maxBytesInMemory=" + getMaxBytesInMemory() + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + ", maxPendingPersists=" + getMaxPendingPersists() + diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 9fb0495284f2..edb4d406cab0 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -1777,7 +1777,7 @@ private KafkaIndexTask createTask( ) { final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( - 1000, + 1000, null, maxRowsPerSegment, new Period("P1Y"), null, @@ -1816,7 +1816,7 @@ private KafkaIndexTask createTask( ) { final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( - 1000, + 1000, null, maxRowsPerSegment, new Period("P1Y"), null, diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java index d18a34651de3..435b08dd05d9 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java @@ -102,7 +102,7 @@ public void testSerdeWithNonDefaults() throws Exception public void testCopyOf() { KafkaTuningConfig original = new KafkaTuningConfig( - 1, + 1, null, 2, new Period("PT3S"), new File("/tmp/xxx"), diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index f1c8ce97c9fe..34d3958a13ae 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -184,7 +184,7 @@ public void setupTest() taskQueue = createMock(TaskQueue.class); tuningConfig = new KafkaSupervisorTuningConfig( - 1000, + 1000, null, 50000, new Period("P1Y"), new File("/test"), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index a7084f69d436..41550f345bed 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -39,6 +39,7 @@ public class RealtimeAppenderatorTuningConfig implements TuningConfig, Appendera { private static final int defaultMaxRowsInMemory = 75000; private static final int defaultMaxRowsPerSegment = 5_000_000; + private static final long defaultMaxBytesInMemory = getDefaultMaxBytesInMemory(); private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); private static final int defaultMaxPendingPersists = 0; private static final ShardSpec defaultShardSpec = NoneShardSpec.instance(); @@ -52,8 +53,15 @@ private static File createNewBasePersistDirectory() return Files.createTempDir(); } + private static long getDefaultMaxBytesInMemory() + { + return (Runtime.getRuntime().maxMemory()) / 3; + } + + private final int maxRowsInMemory; private final int maxRowsPerSegment; + private final long maxBytesInMemory; private final Period intermediatePersistPeriod; private final File basePersistDirectory; private final int maxPendingPersists; @@ -69,6 +77,7 @@ private static File createNewBasePersistDirectory() public RealtimeAppenderatorTuningConfig( @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, + @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @JsonProperty("maxPendingPersists") Integer maxPendingPersists, @@ -82,6 +91,7 @@ public RealtimeAppenderatorTuningConfig( { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; this.maxRowsPerSegment = maxRowsPerSegment == null ? defaultMaxRowsPerSegment : maxRowsPerSegment; + this.maxBytesInMemory = maxBytesInMemory == null ? defaultMaxBytesInMemory : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaultIntermediatePersistPeriod : intermediatePersistPeriod; @@ -109,6 +119,12 @@ public int getMaxRowsInMemory() return maxRowsInMemory; } + @Override + public long getMaxBytesInMemory() + { + return maxBytesInMemory; + } + @JsonProperty public int getMaxRowsPerSegment() { @@ -181,6 +197,7 @@ public RealtimeAppenderatorTuningConfig withBasePersistDirectory(File dir) return new RealtimeAppenderatorTuningConfig( maxRowsInMemory, maxRowsPerSegment, + maxBytesInMemory, intermediatePersistPeriod, dir, maxPendingPersists, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java index 0bba29fbe881..3bc5aefb33dc 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java @@ -105,6 +105,7 @@ public Plumber findPlumber( config.getShardSpec(), version, config.getMaxRowsInMemory(), + config.getMaxBytesInMemory(), config.isReportParseExceptions() ); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 08f857ea4212..a9f78fb3ee4d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -927,6 +927,7 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi { private static final int DEFAULT_MAX_ROWS_IN_MEMORY = 75_000; private static final int DEFAULT_MAX_TOTAL_ROWS = 20_000_000; + private static final long DEFAULT_MAX_BYTES_IN_MEMORY = Runtime.getRuntime().maxMemory() / 3; private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; private static final boolean DEFAULT_FORCE_EXTENDABLE_SHARD_SPECS = false; @@ -938,6 +939,7 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi private final Integer targetPartitionSize; private final int maxRowsInMemory; + private final long maxBytesInMemory; private final Long maxTotalRows; private final Integer numShards; private final IndexSpec indexSpec; @@ -954,6 +956,7 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi public IndexTuningConfig( @JsonProperty("targetPartitionSize") @Nullable Integer targetPartitionSize, @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows, @JsonProperty("rowFlushBoundary") @Nullable Integer rowFlushBoundary_forBackCompatibility, // DEPRECATED @JsonProperty("numShards") @Nullable Integer numShards, @@ -972,6 +975,7 @@ public IndexTuningConfig( this( targetPartitionSize, maxRowsInMemory != null ? maxRowsInMemory : rowFlushBoundary_forBackCompatibility, + maxBytesInMemory != null ? maxBytesInMemory : DEFAULT_MAX_BYTES_IN_MEMORY, maxTotalRows, numShards, indexSpec, @@ -987,12 +991,13 @@ public IndexTuningConfig( private IndexTuningConfig() { - this(null, null, null, null, null, null, null, null, null, null, null, null); + this(null, null, null, null, null, null, null, null, null, null, null, null, null); } private IndexTuningConfig( @Nullable Integer targetPartitionSize, @Nullable Integer maxRowsInMemory, + @Nullable Long maxBytesInMemory, @Nullable Long maxTotalRows, @Nullable Integer numShards, @Nullable IndexSpec indexSpec, @@ -1012,6 +1017,7 @@ private IndexTuningConfig( this.targetPartitionSize = initializeTargetPartitionSize(numShards, targetPartitionSize); this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory == null ? DEFAULT_MAX_BYTES_IN_MEMORY : maxBytesInMemory; this.maxTotalRows = initializeMaxTotalRows(numShards, maxTotalRows); this.numShards = numShards == null || numShards.equals(-1) ? null : numShards; this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; @@ -1059,6 +1065,7 @@ public IndexTuningConfig withBasePersistDirectory(File dir) return new IndexTuningConfig( targetPartitionSize, maxRowsInMemory, + maxBytesInMemory, maxTotalRows, numShards, indexSpec, @@ -1085,6 +1092,13 @@ public int getMaxRowsInMemory() return maxRowsInMemory; } + @JsonProperty + @Override + public long getMaxBytesInMemory() + { + return maxBytesInMemory; + } + @JsonProperty public Long getMaxTotalRows() { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index abd451a9ed30..4bb79ba6a316 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1038,6 +1038,7 @@ private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( null, null, null, + null, reportParseExceptions, handoffTimeout, null, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java index 27ecd4bb66a2..d97a0d363468 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java @@ -231,6 +231,7 @@ private static IndexTuningConfig createTuningConfig() 1000000L, null, null, + null, new IndexSpec( new RoaringBitmapSerdeFactory(true), CompressionStrategy.LZ4, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 8106c79122e8..5c4be2942b90 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -581,7 +581,7 @@ public void testWithSmallMaxTotalRows() throws Exception Granularities.MINUTE, null ), - createTuningConfig(2, 2, 2L, null, false, false, true), + createTuningConfig(2, 2, null, 2L, null, false, false, true), false ), null @@ -623,7 +623,7 @@ public void testPerfectRollup() throws Exception true, null ), - createTuningConfig(3, 2, 2L, null, false, true, true), + createTuningConfig(3, 2, null, 2L, null, false, true, true), false ), null @@ -664,7 +664,7 @@ public void testBestEffortRollup() throws Exception true, null ), - createTuningConfig(3, 2, 2L, null, false, false, true), + createTuningConfig(3, 2, null, 2L, null, false, false, true), false ), null @@ -734,7 +734,7 @@ public void testIgnoreParseException() throws Exception 0 ), null, - createTuningConfig(2, null, null, null, false, false, false), // ignore parse exception, + createTuningConfig(2, null, null, null, null, false, false, false), // ignore parse exception, false ); @@ -787,7 +787,7 @@ public void testReportParseException() throws Exception 0 ), null, - createTuningConfig(2, null, null, null, false, false, true), // report parse exception + createTuningConfig(2, null, null, null, null, false, false, true), // report parse exception false ); @@ -846,7 +846,7 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception 0 ), null, - createTuningConfig(2, 1, null, null, false, true, true), // report parse exception + createTuningConfig(2, 1, null, null, null, false, true, true), // report parse exception false ); @@ -916,7 +916,7 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception 0 ), null, - createTuningConfig(2, null, null, null, false, false, true), // report parse exception + createTuningConfig(2, null, null, null, null, false, false, true), // report parse exception false ); @@ -1119,6 +1119,7 @@ private static IndexTuningConfig createTuningConfig( targetPartitionSize, 1, null, + null, numShards, forceExtendableShardSpecs, forceGuaranteedRollup, @@ -1129,6 +1130,7 @@ private static IndexTuningConfig createTuningConfig( private static IndexTuningConfig createTuningConfig( Integer targetPartitionSize, Integer maxRowsInMemory, + Long maxBytesInMemory, Long maxTotalRows, Integer numShards, boolean forceExtendableShardSpecs, @@ -1139,6 +1141,7 @@ private static IndexTuningConfig createTuningConfig( return new IndexTask.IndexTuningConfig( targetPartitionSize, maxRowsInMemory, + maxBytesInMemory, maxTotalRows, null, numShards, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 916c925e0d02..a8f10f9c9b42 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -916,6 +916,7 @@ private RealtimeIndexTask makeRealtimeTask( ); RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( 1000, + null, new Period("P1Y"), new Period("PT10M"), null, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index ff5a17d8ba44..5c1ab4a7c9f4 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -190,7 +190,23 @@ public void testIndexTaskSerde() throws Exception jsonMapper ), new IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), - new IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, false, null, null, null, null) + new IndexTuningConfig( + 10000, + 10, + null, + null, + 9999, + null, + indexSpec, + 3, + true, + true, + false, + null, + null, + null, + null + ) ), null ); @@ -253,7 +269,23 @@ public void testIndexTaskwithResourceSerde() throws Exception jsonMapper ), new IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), - new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null, null) + new IndexTuningConfig( + 10000, + 10, + null, + null, + null, + null, + indexSpec, + 3, + true, + true, + false, + null, + null, + null, + null + ) ), null ); @@ -495,7 +527,7 @@ public Plumber findPlumber( ), new RealtimeTuningConfig( - 1, + 1, null, new Period("PT10M"), null, null, diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index c17452f654aa..52823c83aa88 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -667,7 +667,23 @@ public void testIndexTask() throws Exception mapper ), new IndexIOConfig(new MockFirehoseFactory(false), false), - new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null, null) + new IndexTuningConfig( + 10000, + 10, + null, + null, + null, + null, + indexSpec, + 3, + true, + true, + false, + null, + null, + null, + null + ) ), null ); @@ -725,7 +741,23 @@ public void testIndexTaskFailure() throws Exception mapper ), new IndexIOConfig(new MockExceptionalFirehoseFactory(), false), - new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null, null) + new IndexTuningConfig( + 10000, + 10, + null, + null, + null, + null, + indexSpec, + 3, + true, + true, + false, + null, + null, + null, + null + ) ), null ); @@ -1090,7 +1122,23 @@ public void testResumeTasks() throws Exception mapper ), new IndexIOConfig(new MockFirehoseFactory(false), false), - new IndexTuningConfig(10000, 10, null, null, null, indexSpec, null, false, null, null, null, null, null, null) + new IndexTuningConfig( + 10000, + 10, + null, + null, + null, + null, + indexSpec, + null, + false, + null, + null, + null, + null, + null, + null + ) ), null ); @@ -1199,6 +1247,7 @@ private RealtimeIndexTask newRealtimeIndexTask() ); RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( 1000, + null, new Period("P1Y"), null, //default window period of 10 minutes null, // base persist dir ignored by Realtime Index task diff --git a/processing/src/main/java/io/druid/segment/DimensionIndexer.java b/processing/src/main/java/io/druid/segment/DimensionIndexer.java index cbc298c2f246..07409d51ec2d 100644 --- a/processing/src/main/java/io/druid/segment/DimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/DimensionIndexer.java @@ -127,6 +127,14 @@ public interface DimensionIndexer */ EncodedKeyComponentType processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boolean reportParseExceptions); + /** + * Gives the estimated size in bytes for the given key + * + * @param key dimension value array from a TimeAndDims key + * + * @return the estimated size in bytes of the key + */ + long estimateEncodedKeyComponentSize(EncodedKeyComponentType key); /** * Given an encoded value that was ordered by associated actual value, return the equivalent diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java index a0a7f30e31f3..d61a4605165d 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java @@ -49,6 +49,12 @@ public Double processRowValsToUnsortedEncodedKeyComponent(Object dimValues, bool return ret == null ? DimensionHandlerUtils.ZERO_DOUBLE : ret; } + @Override + public long estimateEncodedKeyComponentSize(Double key) + { + return Double.BYTES; + } + @Override public Double getUnsortedEncodedValueFromSorted(Double sortedIntermediateValue) { diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java index 695535cdac5a..651a25321b26 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java @@ -50,6 +50,12 @@ public Float processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boole return ret == null ? DimensionHandlerUtils.ZERO_FLOAT : ret; } + @Override + public long estimateEncodedKeyComponentSize(Float key) + { + return Float.BYTES; + } + @Override public Float getUnsortedEncodedValueFromSorted(Float sortedIntermediateValue) { diff --git a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java index 187c3fe2d291..1e5a378a4899 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java @@ -50,6 +50,12 @@ public Long processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boolea return ret == null ? DimensionHandlerUtils.ZERO_LONG : ret; } + @Override + public long estimateEncodedKeyComponentSize(Long key) + { + return Long.BYTES; + } + @Override public Long getUnsortedEncodedValueFromSorted(Long sortedIntermediateValue) { diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java index 68457c5f4982..911153613b89 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -265,6 +265,17 @@ public int[] processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boole return encodedDimensionValues; } + @Override + public long estimateEncodedKeyComponentSize(int[] key) + { + // the size is calculated conservatively, first key may be accounted for twice + long estimatedSize = (key.length * Integer.BYTES); + estimatedSize += Arrays.stream(key) + .filter(element -> dimLookup.getValue(element) != null) + .mapToLong(element -> dimLookup.getValue(element).length() * Character.BYTES).sum(); + return estimatedSize; + } + public Integer getSortedEncodedValueFromUnsorted(Integer unsortedIntermediateValue) { return sortedLookup().getSortedIdFromUnsortedId(unsortedIntermediateValue); diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 332781b9beeb..fb09dbc97c53 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -95,6 +95,7 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; /** */ @@ -239,6 +240,7 @@ public ColumnCapabilities getColumnCapabilities(String columnName) private final List dimensionDescsList; private final Map columnCapabilities; private final AtomicInteger numEntries = new AtomicInteger(); + private final AtomicLong sizeInBytes = new AtomicLong(); // This is modified on add() in a critical section. private final ThreadLocal in = new ThreadLocal<>(); @@ -247,7 +249,7 @@ public ColumnCapabilities getColumnCapabilities(String columnName) /** * Setting deserializeComplexMetrics to false is necessary for intermediate aggregation such as groupBy that * should not deserialize input columns using ComplexMetricSerde for aggregators that return complex metrics. - * + *

* Set concurrentEventAdd to true to indicate that adding of input row should be thread-safe (for example, groupBy * where the multiple threads can add concurrently to the IncrementalIndex). * @@ -333,6 +335,7 @@ public static class Builder private boolean concurrentEventAdd; private boolean sortFacts; private int maxRowCount; + private long maxBytesInMemory; public Builder() { @@ -342,6 +345,7 @@ public Builder() concurrentEventAdd = false; sortFacts = true; maxRowCount = 0; + maxBytesInMemory = 0; } public Builder setIndexSchema(final IncrementalIndexSchema incrementalIndexSchema) @@ -398,6 +402,12 @@ public Builder setMaxRowCount(final int maxRowCount) return this; } + public Builder setMaxBytesInMemory(final long maxBytesInMemory) + { + this.maxBytesInMemory = maxBytesInMemory; + return this; + } + public IncrementalIndex buildOnheap() { if (maxRowCount <= 0) { @@ -410,7 +420,8 @@ public IncrementalIndex buildOnheap() reportParseExceptions, concurrentEventAdd, sortFacts, - maxRowCount + maxRowCount, + maxBytesInMemory ); } @@ -457,6 +468,7 @@ protected abstract Integer addToFacts( boolean reportParseExceptions, InputRow row, AtomicInteger numEntries, + AtomicLong sizeInBytes, TimeAndDims key, ThreadLocal rowContainer, Supplier rowSupplier, @@ -528,6 +540,7 @@ public int add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeE reportParseExceptions, row, numEntries, + sizeInBytes, key, in, rowSupplier, @@ -549,6 +562,7 @@ TimeAndDims toTimeAndDims(InputRow row) Object[] dims; List overflow = null; + long dimsKeySize = 0; synchronized (dimensionDescs) { dims = new Object[dimensionDescs.size()]; for (String dimension : rowDimensions) { @@ -580,6 +594,7 @@ TimeAndDims toTimeAndDims(InputRow row) row.getRaw(dimension), reportParseExceptions ); + dimsKeySize += indexer.estimateEncodedKeyComponentSize(dimsKey); // Set column capabilities as data is coming in if (!capabilities.hasMultipleValues() && dimsKey != null && handler.getLengthOfEncodedKeyComponent(dimsKey) > 1) { @@ -622,7 +637,7 @@ TimeAndDims toTimeAndDims(InputRow row) if (row.getTimestamp() != null) { truncated = gran.bucketStart(row.getTimestamp()).getMillis(); } - return new TimeAndDims(Math.max(truncated, minTimestamp), dims, dimensionDescsList); + return new TimeAndDims(Math.max(truncated, minTimestamp), dims, dimensionDescsList, dimsKeySize); } private synchronized void updateMaxIngestedTime(DateTime eventTime) @@ -642,6 +657,11 @@ public int size() return numEntries.get(); } + public long sizeInBytes() + { + return sizeInBytes.get(); + } + private long getMinTimeMillis() { return getFacts().getMinTimeMillis(); @@ -1002,6 +1022,7 @@ public static final class TimeAndDims * {@link RollupFactsHolder} needs concurrent collections, that are not present in fastutil. */ private int rowIndex; + private long dimsKeySize; TimeAndDims( long timestamp, @@ -1025,6 +1046,19 @@ public static final class TimeAndDims this.rowIndex = rowIndex; } + TimeAndDims( + long timestamp, + Object[] dims, + List dimensionDescsList, + long dimsKeySize + ) + { + this.timestamp = timestamp; + this.dims = dims; + this.dimensionDescsList = dimensionDescsList; + this.dimsKeySize = dimsKeySize; + } + public long getTimestamp() { return timestamp; @@ -1045,6 +1079,14 @@ private void setRowIndex(int rowIndex) this.rowIndex = rowIndex; } + public long estimateBytesInMemory() + { + //timestamp + dims length + dimensionDescsList shared pointer + long sizeInBytes = Long.BYTES + Integer.BYTES * dims.length + Long.BYTES + Long.BYTES; + sizeInBytes += dimsKeySize; + return sizeInBytes; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java index d377634e0bc8..c5553578403d 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -40,6 +40,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; /** */ @@ -144,6 +145,7 @@ protected Integer addToFacts( boolean reportParseExceptions, InputRow row, AtomicInteger numEntries, + AtomicLong sizeInBytes, TimeAndDims key, ThreadLocal rowContainer, Supplier rowSupplier, diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 1a377ea826be..45defbfc7566 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -37,21 +37,30 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; /** */ public class OnheapIncrementalIndex extends IncrementalIndex { private static final Logger log = new Logger(OnheapIncrementalIndex.class); - + /** + * overhead per Map$Entry object + * KeyHash + next pointer + key pointer + value pointer + safe extra + */ + private static final int ROUGH_OVERHEAD_PER_MAP_ENTRY = Long.BYTES * 5 + Integer.BYTES; + private static final long defaultMaxBytesInMemory = Runtime.getRuntime().maxMemory() / 3; private final ConcurrentHashMap aggregators = new ConcurrentHashMap<>(); private final FactsHolder facts; private final AtomicInteger indexIncrement = new AtomicInteger(0); + private long maxBytesPerRowForAggregators = 0; protected final int maxRowCount; + protected final long maxBytesInMemory; private volatile Map selectors; private String outOfRowsReason = null; @@ -62,14 +71,30 @@ public class OnheapIncrementalIndex extends IncrementalIndex boolean reportParseExceptions, boolean concurrentEventAdd, boolean sortFacts, - int maxRowCount + int maxRowCount, + long maxBytesInMemory ) { super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions, concurrentEventAdd); this.maxRowCount = maxRowCount; - + this.maxBytesInMemory = maxBytesInMemory == 0 ? defaultMaxBytesInMemory : maxBytesInMemory; this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) : new PlainFactsHolder(sortFacts); + if (maxBytesInMemory != -1) { + maxBytesPerRowForAggregators = getMaxBytesPerRowForAggregators(incrementalIndexSchema); + } + } + + private long getMaxBytesPerRowForAggregators(IncrementalIndexSchema incrementalIndexSchema) + { + long maxAggregatorIntermediateSize = Integer.BYTES * incrementalIndexSchema.getMetrics().length; + maxAggregatorIntermediateSize += Arrays.stream(incrementalIndexSchema.getMetrics()) + .reduce( + 0, + (sum, aggregator) -> sum += aggregator.getMaxIntermediateSize(), + (sum1, sum2) -> sum1 + sum2 + ); + return maxAggregatorIntermediateSize; } @Override @@ -107,6 +132,7 @@ protected Integer addToFacts( boolean reportParseExceptions, InputRow row, AtomicInteger numEntries, + AtomicLong sizeInBytes, TimeAndDims key, ThreadLocal rowContainer, Supplier rowSupplier, @@ -130,13 +156,18 @@ protected Integer addToFacts( // Last ditch sanity checks if (numEntries.get() >= maxRowCount - && facts.getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX - && !skipMaxRowsInMemoryCheck) { - throw new IndexSizeExceededException("Maximum number of rows [%d] reached", maxRowCount); + || maxBytesInMemory != -1 && sizeInBytes.get() >= maxBytesInMemory + && facts.getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX + && !skipMaxRowsInMemoryCheck) { + throw new IndexSizeExceededException("Maximum number of rows [%d] or max size in bytes [%d] reached", maxRowCount, maxBytesInMemory); } final int prev = facts.putIfAbsent(key, rowIndex); if (TimeAndDims.EMPTY_ROW_INDEX == prev) { numEntries.incrementAndGet(); + if (maxBytesInMemory != -1) { + long estimatedRowSize = estimateRowSizeInBytes(key) + maxBytesPerRowForAggregators; + sizeInBytes.addAndGet(estimatedRowSize); + } } else { // We lost a race aggs = concurrentGet(prev); @@ -150,6 +181,11 @@ protected Integer addToFacts( return numEntries.get(); } + private long estimateRowSizeInBytes(TimeAndDims key) + { + return ROUGH_OVERHEAD_PER_MAP_ENTRY + key.estimateBytesInMemory(); + } + @Override public int getLastRowIndex() { @@ -237,10 +273,18 @@ protected void concurrentRemove(int offset) @Override public boolean canAppendRow() { - final boolean canAdd = size() < maxRowCount; - if (!canAdd) { + final boolean countCheck = size() < maxRowCount; + boolean sizeCheck = true; + if (maxBytesInMemory != -1) { + sizeCheck = sizeInBytes() < maxBytesInMemory; + } + final boolean canAdd = countCheck && sizeCheck; + if (!countCheck) { outOfRowsReason = StringUtils.format("Maximum number of rows [%d] reached", maxRowCount); } + if (!sizeCheck) { + outOfRowsReason = StringUtils.format("Maximum size in bytes [%d] reached", maxBytesInMemory); + } return canAdd; } diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index c4f5fe3f1737..2f859c941991 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -76,6 +76,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; /** * Extending AbstractBenchmark means only runs if explicitly called @@ -118,7 +119,8 @@ public MapIncrementalIndex( boolean reportParseExceptions, boolean concurrentEventAdd, boolean sortFacts, - int maxRowCount + int maxRowCount, + long maxBytesInMemory ) { super( @@ -127,7 +129,8 @@ public MapIncrementalIndex( reportParseExceptions, concurrentEventAdd, sortFacts, - maxRowCount + maxRowCount, + maxBytesInMemory ); } @@ -135,20 +138,22 @@ public MapIncrementalIndex( long minTimestamp, Granularity gran, AggregatorFactory[] metrics, - int maxRowCount + int maxRowCount, + long maxBytesInMemory ) { super( new IncrementalIndexSchema.Builder() - .withMinTimestamp(minTimestamp) - .withQueryGranularity(gran) - .withMetrics(metrics) - .build(), - true, - true, - false, - true, - maxRowCount + .withMinTimestamp(minTimestamp) + .withQueryGranularity(gran) + .withMetrics(metrics) + .build(), + true, + true, + false, + true, + maxRowCount, + maxBytesInMemory ); } @@ -172,6 +177,7 @@ protected Integer addToFacts( boolean reportParseExceptions, InputRow row, AtomicInteger numEntries, + AtomicLong sizeInBytes, TimeAndDims key, ThreadLocal rowContainer, Supplier rowSupplier, diff --git a/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsKeySizeTest.java b/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsKeySizeTest.java new file mode 100644 index 000000000000..e8b59652faf4 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsKeySizeTest.java @@ -0,0 +1,94 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.incremental; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import io.druid.data.input.MapBasedInputRow; +import io.druid.query.aggregation.CountAggregatorFactory; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Map; + +/** + */ +public class TimeAndDimsKeySizeTest +{ + @Test + public void testTimeAndDimsKeySizeBasic() + { + IncrementalIndex index = new IncrementalIndex.Builder() + .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) + .setMaxRowCount(10000) + .setMaxBytesInMemory(1000) + .buildOnheap(); + long time = System.currentTimeMillis(); + IncrementalIndex.TimeAndDims td1 = index.toTimeAndDims(toMapRow(time, "billy", "A", "joe", "B")); + Assert.assertEquals(44, td1.estimateBytesInMemory()); + } + + @Test + public void testTimeAndDimsKeySizeArr() + { + IncrementalIndex index = new IncrementalIndex.Builder() + .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) + .setMaxRowCount(10000) + .setMaxBytesInMemory(1000) + .buildOnheap(); + long time = System.currentTimeMillis(); + IncrementalIndex.TimeAndDims td1 = index.toTimeAndDims(toMapRow( + time + 1, + "billy", + "A", + "joe", + Arrays.asList("A", "B") + )); + Assert.assertEquals(50, td1.estimateBytesInMemory()); + } + + @Test + public void testTimeAndDimsKeySizeComplex() + { + IncrementalIndex index = new IncrementalIndex.Builder() + .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) + .setMaxRowCount(10000) + .setMaxBytesInMemory(1000) + .buildOnheap(); + long time = System.currentTimeMillis(); + IncrementalIndex.TimeAndDims td1 = index.toTimeAndDims(toMapRow( + time + 1, + "billy", + "nelson", + "joe", + Arrays.asList("123", "abcdef") + )); + Assert.assertEquals(74, td1.estimateBytesInMemory()); + } + + private MapBasedInputRow toMapRow(long time, Object... dimAndVal) + { + Map data = Maps.newHashMap(); + for (int i = 0; i < dimAndVal.length; i += 2) { + data.put((String) dimAndVal[i], dimAndVal[i + 1]); + } + return new MapBasedInputRow(time, Lists.newArrayList(data.keySet()), data); + } +} diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index c9cdf6358aed..42899b33ecb3 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -42,6 +42,7 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig { private static final int defaultMaxRowsInMemory = 75000; + private static final long defaultMaxBytesInMemory = getDefaultMaxBytesInMemory(); private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); private static final Period defaultWindowPeriod = new Period("PT10M"); private static final VersioningPolicy defaultVersioningPolicy = new IntervalStartVersioningPolicy(); @@ -58,11 +59,17 @@ private static File createNewBasePersistDirectory() return Files.createTempDir(); } + private static long getDefaultMaxBytesInMemory() + { + return (Runtime.getRuntime().maxMemory()) / 3; + } + // Might make sense for this to be a builder public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File basePersistDirectory) { return new RealtimeTuningConfig( defaultMaxRowsInMemory, + defaultMaxBytesInMemory, defaultIntermediatePersistPeriod, defaultWindowPeriod, basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory, @@ -82,6 +89,7 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File } private final int maxRowsInMemory; + private final long maxBytesInMemory; private final Period intermediatePersistPeriod; private final Period windowPeriod; private final File basePersistDirectory; @@ -101,6 +109,7 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File @JsonCreator public RealtimeTuningConfig( @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -120,6 +129,7 @@ public RealtimeTuningConfig( ) { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory == null ? defaultMaxBytesInMemory : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaultIntermediatePersistPeriod : intermediatePersistPeriod; @@ -154,6 +164,12 @@ public int getMaxRowsInMemory() return maxRowsInMemory; } + @Override + public long getMaxBytesInMemory() + { + return maxBytesInMemory; + } + @Override @JsonProperty public Period getIntermediatePersistPeriod() @@ -259,6 +275,7 @@ public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) { return new RealtimeTuningConfig( maxRowsInMemory, + maxBytesInMemory, intermediatePersistPeriod, windowPeriod, basePersistDirectory, @@ -281,6 +298,7 @@ public RealtimeTuningConfig withBasePersistDirectory(File dir) { return new RealtimeTuningConfig( maxRowsInMemory, + maxBytesInMemory, intermediatePersistPeriod, windowPeriod, dir, diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java index 286470cae761..05758b1645a7 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java @@ -32,6 +32,8 @@ public interface AppenderatorConfig int getMaxRowsInMemory(); + long getMaxBytesInMemory(); + int getMaxPendingPersists(); Period getIntermediatePersistPeriod(); diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index be32f2960fff..751f10a46b61 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -283,6 +283,18 @@ int getRowsInMemory() return rowsCurrentlyInMemory.get(); } + @VisibleForTesting + long getRowSizeInMemory(SegmentIdentifier identifier) + { + final Sink sink = sinks.get(identifier); + + if (sink == null) { + throw new ISE("No such sink: %s", identifier); + } else { + return sink.getBytesInMemory(); + } + } + private Sink getOrCreateSink(final SegmentIdentifier identifier) { Sink retVal = sinks.get(identifier); @@ -294,6 +306,7 @@ private Sink getOrCreateSink(final SegmentIdentifier identifier) identifier.getShardSpec(), identifier.getVersion(), tuningConfig.getMaxRowsInMemory(), + tuningConfig.getMaxBytesInMemory(), tuningConfig.isReportParseExceptions() ); @@ -559,7 +572,7 @@ private ListenableFuture pushBarrier() /** * Merge segment, push to deep storage. Should only be used on segments that have been fully persisted. Must only * be run in the single-threaded pushExecutor. - * + *

* Note that this calls DataSegmentPusher.push() with replaceExisting == true which is appropriate for the indexing * tasks it is currently being used for (local indexing and Kafka indexing). If this is going to be used by an * indexing task type that requires replaceExisting == false, this setting will need to be pushed to the caller. @@ -975,6 +988,7 @@ public int compare(File o1, File o2) identifier.getShardSpec(), identifier.getVersion(), tuningConfig.getMaxRowsInMemory(), + tuningConfig.getMaxBytesInMemory(), tuningConfig.isReportParseExceptions(), hydrants ); diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 2f6a4a1dd995..dac2be05a11c 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -251,6 +251,7 @@ private Sink getSink(long timestamp) config.getShardSpec(), versioningPolicy.getVersion(sinkInterval), config.getMaxRowsInMemory(), + config.getMaxBytesInMemory(), config.isReportParseExceptions() ); addSink(retVal); @@ -732,6 +733,7 @@ public int compare(File o1, File o2) config.getShardSpec(), versioningPolicy.getVersion(sinkInterval), config.getMaxRowsInMemory(), + config.getMaxBytesInMemory(), config.isReportParseExceptions(), hydrants ); diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index 634d8f514bd6..c59408c02586 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -61,6 +61,7 @@ public class Sink implements Iterable private final ShardSpec shardSpec; private final String version; private final int maxRowsInMemory; + private final long maxBytesInMemory; private final boolean reportParseExceptions; private final CopyOnWriteArrayList hydrants = new CopyOnWriteArrayList(); private final LinkedHashSet dimOrder = Sets.newLinkedHashSet(); @@ -74,6 +75,7 @@ public Sink( ShardSpec shardSpec, String version, int maxRowsInMemory, + long maxBytesInMemory, boolean reportParseExceptions ) { @@ -82,6 +84,7 @@ public Sink( this.interval = interval; this.version = version; this.maxRowsInMemory = maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory; this.reportParseExceptions = reportParseExceptions; makeNewCurrIndex(interval.getStartMillis(), schema); @@ -93,6 +96,7 @@ public Sink( ShardSpec shardSpec, String version, int maxRowsInMemory, + long maxBytesInMemory, boolean reportParseExceptions, List hydrants ) @@ -102,6 +106,7 @@ public Sink( this.interval = interval; this.version = version; this.maxRowsInMemory = maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory; this.reportParseExceptions = reportParseExceptions; int maxCount = -1; @@ -172,6 +177,8 @@ public boolean isEmpty() } } + //add methods for byte mem checks + public boolean isWritable() { return writable; @@ -249,6 +256,19 @@ public int getNumRowsInMemory() } } + + public long getBytesInMemory() + { + synchronized (hydrantLock) { + IncrementalIndex index = currHydrant.getIndex(); + if (index == null) { + return 0; + } + + return currHydrant.getIndex().sizeInBytes(); + } + } + private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) { final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder() @@ -262,7 +282,7 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) final IncrementalIndex newIndex = new IncrementalIndex.Builder() .setIndexSchema(indexSchema) .setReportParseExceptions(reportParseExceptions) - .setMaxRowCount(maxRowsInMemory) + .setMaxRowCount(maxRowsInMemory).setMaxBytesInMemory(maxBytesInMemory) .buildOnheap(); final FireHydrant old; diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index bd4b42102553..6b78d4005079 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -199,6 +199,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws ParseExcept ); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( 1, + null, new Period("P1Y"), null, null, @@ -221,6 +222,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws ParseExcept tuningConfig.getShardSpec(), DateTimes.nowUtc().toString(), tuningConfig.getMaxRowsInMemory(), + tuningConfig.getMaxBytesInMemory(), tuningConfig.isReportParseExceptions() )); @@ -241,6 +243,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws ParseExcept tuningConfig.getShardSpec(), DateTimes.nowUtc().toString(), tuningConfig.getMaxRowsInMemory(), + tuningConfig.getMaxBytesInMemory(), tuningConfig.isReportParseExceptions() )); @@ -258,6 +261,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws ParseExcept tuningConfig_0 = new RealtimeTuningConfig( 1, + null, new Period("P1Y"), null, null, @@ -277,6 +281,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws ParseExcept tuningConfig_1 = new RealtimeTuningConfig( 1, + null, new Period("P1Y"), null, null, diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java index d7b0aa4a6477..58e67b84648d 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java @@ -71,6 +71,7 @@ EasyMock. anyObject(), null, null, null, + null, new IntervalStartVersioningPolicy(), new NoopRejectionPolicyFactory(), null, diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java index c07ac066adb9..9aa4f85ccff5 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java @@ -144,6 +144,81 @@ public SegmentIdentifier apply(DataSegment input) } } + @Test + public void testMaxBytesInMemory() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester(100, 1024, true)) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + final Supplier committerSupplier = () -> { + final Object metadata = ImmutableMap.of(eventCount, eventCount.get()); + + return new Committer() + { + @Override + public Object getMetadata() + { + return metadata; + } + + @Override + public void run() + { + //Do nothing + } + }; + }; + + appenderator.startJob(); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); + //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 24 (aggregator metrics) + 10 (dimsKeySize) = 106 + Assert.assertEquals(106, ((AppenderatorImpl) appenderator).getRowSizeInMemory(IDENTIFIERS.get(0))); + appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); + Assert.assertEquals(106, ((AppenderatorImpl) appenderator).getRowSizeInMemory(IDENTIFIERS.get(1))); + appenderator.close(); + Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + } + } + + @Test + public void testIgnoreMaxBytesInMemory() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester(100, -1, true)) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + final Supplier committerSupplier = () -> { + final Object metadata = ImmutableMap.of(eventCount, eventCount.get()); + + return new Committer() + { + @Override + public Object getMetadata() + { + return metadata; + } + + @Override + public void run() + { + //Do nothing + } + }; + }; + + Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + appenderator.startJob(); + Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); + Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowSizeInMemory(IDENTIFIERS.get(0))); + Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); + appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); + Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowSizeInMemory(IDENTIFIERS.get(1))); + Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); + appenderator.close(); + Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + } + } + @Test public void testMaxRowsInMemory() throws Exception { @@ -242,6 +317,7 @@ public void run() Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); } } + @Test public void testRestoreFromDisk() throws Exception { @@ -288,7 +364,12 @@ public void run() appenderator.add(IDENTIFIERS.get(0), IR("2000", "bob", 5), committerSupplier); appenderator.close(); - try (final AppenderatorTester tester2 = new AppenderatorTester(2, tuningConfig.getBasePersistDirectory(), true)) { + try (final AppenderatorTester tester2 = new AppenderatorTester( + 2, + -1, + tuningConfig.getBasePersistDirectory(), + true + )) { final Appenderator appenderator2 = tester2.getAppenderator(); Assert.assertEquals(ImmutableMap.of("eventCount", 4), appenderator2.startJob()); Assert.assertEquals(ImmutableList.of(IDENTIFIERS.get(0)), appenderator2.getSegments()); diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index 20543010782f..86eaec928fbd 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -89,7 +89,7 @@ public AppenderatorTester( final int maxRowsInMemory ) { - this(maxRowsInMemory, null, false); + this(maxRowsInMemory, -1, null, false); } public AppenderatorTester( @@ -97,11 +97,21 @@ public AppenderatorTester( final boolean enablePushFailure ) { - this(maxRowsInMemory, null, enablePushFailure); + this(maxRowsInMemory, -1, null, enablePushFailure); } public AppenderatorTester( final int maxRowsInMemory, + final long maxSizeInBytes, + final boolean enablePushFailure + ) + { + this(maxRowsInMemory, maxSizeInBytes, null, enablePushFailure); + } + + public AppenderatorTester( + final int maxRowsInMemory, + long maxSizeInBytes, final File basePersistDirectory, final boolean enablePushFailure ) @@ -131,9 +141,10 @@ public AppenderatorTester( null, objectMapper ); - + maxSizeInBytes = maxSizeInBytes == 0l ? getDefaultMaxBytesInMemory() : maxSizeInBytes; tuningConfig = new RealtimeTuningConfig( maxRowsInMemory, + maxSizeInBytes, null, null, basePersistDirectory, @@ -267,6 +278,11 @@ public void unannounceSegments(Iterable segments) ); } + private long getDefaultMaxBytesInMemory() + { + return (Runtime.getRuntime().totalMemory()) / 3; + } + public DataSchema getSchema() { return schema; diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index 08157eaf7f61..b837bf5ee453 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -136,6 +136,7 @@ public int columnCacheSizeBytes() 75000, null, null, + null, temporaryFolder.newFolder(), null, null, diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 32900d8900fc..f75cccfb1382 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -199,6 +199,7 @@ public void setUp() throws Exception null, null, null, + null, new IntervalStartVersioningPolicy(), rejectionPolicy, null, @@ -269,6 +270,7 @@ private void testPersist(final Object commitMetadata) throws Exception tuningConfig.getShardSpec(), DateTimes.of("2014-12-01T12:34:56.789").toString(), tuningConfig.getMaxRowsInMemory(), + tuningConfig.getMaxBytesInMemory(), tuningConfig.isReportParseExceptions() ); plumber.getSinks().put(0L, sink); @@ -313,6 +315,7 @@ public void testPersistFails() throws Exception tuningConfig.getShardSpec(), DateTimes.of("2014-12-01T12:34:56.789").toString(), tuningConfig.getMaxRowsInMemory(), + tuningConfig.getMaxBytesInMemory(), tuningConfig.isReportParseExceptions() ); plumber.getSinks().put(0L, sink); @@ -367,6 +370,7 @@ private void testPersistHydrantGapsHelper(final Object commitMetadata) throws Ex tuningConfig.getShardSpec(), DateTimes.of("2014-12-01T12:34:56.789").toString(), tuningConfig.getMaxRowsInMemory(), + tuningConfig.getMaxBytesInMemory(), tuningConfig.isReportParseExceptions() ); plumber2.getSinks().put(0L, sink); diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index dce3e7a89780..f9bf8f03e9ba 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -61,6 +61,7 @@ public void testSwap() throws Exception final String version = DateTimes.nowUtc().toString(); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( 100, + null, new Period("P1Y"), null, null, @@ -83,6 +84,7 @@ public void testSwap() throws Exception tuningConfig.getShardSpec(), version, tuningConfig.getMaxRowsInMemory(), + tuningConfig.getMaxBytesInMemory(), tuningConfig.isReportParseExceptions() ); diff --git a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java index 49518d6483dd..9417e9412904 100644 --- a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java @@ -169,6 +169,7 @@ public Plumber findPlumber( new RealtimeTuningConfig( 1, + null, new Period("PT10M"), null, null, From b31d6342d1a25b465d31fb53d7f0fec8b0b3faff Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Thu, 5 Apr 2018 15:23:56 -0700 Subject: [PATCH 02/31] Fix check style and remove a comment --- .../src/main/java/io/druid/segment/realtime/plumber/Sink.java | 2 -- .../druid/segment/realtime/appenderator/AppenderatorTester.java | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index c59408c02586..b655d9f7afe5 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -177,8 +177,6 @@ public boolean isEmpty() } } - //add methods for byte mem checks - public boolean isWritable() { return writable; diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index 86eaec928fbd..3395158981be 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -141,7 +141,7 @@ public AppenderatorTester( null, objectMapper ); - maxSizeInBytes = maxSizeInBytes == 0l ? getDefaultMaxBytesInMemory() : maxSizeInBytes; + maxSizeInBytes = maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes; tuningConfig = new RealtimeTuningConfig( maxRowsInMemory, maxSizeInBytes, From ac401c5b4dd99fa12d9f201d1a7875bb858c8ca2 Mon Sep 17 00:00:00 2001 From: Jonathan Wei Date: Thu, 5 Apr 2018 14:16:06 -0700 Subject: [PATCH 03/31] Add overlord unsecured paths to coordinator when using combined service (#5579) * Add overlord unsecured paths to coordinator when using combined service * PR comment --- services/src/main/java/io/druid/cli/CliOverlord.java | 3 +-- .../java/io/druid/cli/CoordinatorJettyServerInitializer.java | 4 ++++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index a6b7fbfab9bd..51ca382590d4 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; @@ -118,7 +117,7 @@ public class CliOverlord extends ServerRunnable { private static Logger log = new Logger(CliOverlord.class); - private static List UNSECURED_PATHS = Lists.newArrayList( + protected static List UNSECURED_PATHS = ImmutableList.of( "/", "/console.html", "/old-console/*", diff --git a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java index 09a5a7b519e2..7a0562f3dfd4 100644 --- a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java @@ -118,6 +118,10 @@ public void initialize(Server server, Injector injector) // perform no-op authorization for these resources AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS); + if (beOverlord) { + AuthenticationUtils.addNoopAuthorizationFilters(root, CliOverlord.UNSECURED_PATHS); + } + authenticators = authenticatorMapper.getAuthenticatorChain(); AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); From 9e786e89b5fb58758247f486f9d5b5170a4a9cca Mon Sep 17 00:00:00 2001 From: Jonathan Wei Date: Thu, 5 Apr 2018 21:38:57 -0700 Subject: [PATCH 04/31] More error reporting and stats for ingestion tasks (#5418) * Add more indexing task status and error reporting * PR comments, add support in AppenderatorDriverRealtimeIndexTask * Use TaskReport instead of metrics/context * Fix tests * Use TaskReport uploads * Refactor fire department metrics retrieval * Refactor input row serde in hadoop task * Refactor hadoop task loader names * Truncate error message in TaskStatus, add errorMsg to task report * PR comments --- .../data/input/impl/MapInputRowParser.java | 2 +- .../java/io/druid/indexer/IngestionState.java | 28 + .../io/druid/indexer/TaskMetricsGetter.java | 29 + .../io/druid/indexer/TaskMetricsUtils.java | 47 ++ .../java/io/druid/indexer/TaskStatusPlus.java | 71 +-- .../java/io/druid/utils/CircularBuffer.java | 92 +++ .../io/druid/indexer/TaskStatusPlusTest.java | 3 +- .../FilteredAggregatorBenchmark.java | 2 +- .../IncrementalIndexRowTypeBenchmark.java | 6 +- .../indexing/IndexIngestionBenchmark.java | 2 +- .../src/main/java/io/druid/indexer/Jobby.java | 28 + .../input/orc/OrcIndexGeneratorJobTest.java | 4 +- .../druid/indexing/kafka/KafkaIndexTask.java | 221 ++++++-- .../indexing/kafka/KafkaTuningConfig.java | 63 ++- .../kafka/supervisor/KafkaSupervisorSpec.java | 3 + .../KafkaSupervisorTuningConfig.java | 13 +- .../indexing/kafka/KafkaIndexTaskTest.java | 275 +++++++-- .../indexing/kafka/KafkaTuningConfigTest.java | 3 + .../kafka/supervisor/KafkaSupervisorTest.java | 3 + .../indexer/DetermineHashedPartitionsJob.java | 47 +- .../druid/indexer/DeterminePartitionsJob.java | 48 +- .../HadoopDruidDetermineConfigurationJob.java | 25 +- .../indexer/HadoopDruidIndexerConfig.java | 16 +- .../druid/indexer/HadoopDruidIndexerJob.java | 26 +- .../indexer/HadoopDruidIndexerMapper.java | 76 ++- .../io/druid/indexer/HadoopTuningConfig.java | 42 +- .../io/druid/indexer/IndexGeneratorJob.java | 78 ++- .../java/io/druid/indexer/InputRowSerde.java | 98 +++- .../main/java/io/druid/indexer/JobHelper.java | 39 +- .../src/main/java/io/druid/indexer/Utils.java | 26 + .../indexer/BatchDeltaIngestionTest.java | 4 +- .../DetermineHashedPartitionsJobTest.java | 2 + .../indexer/DeterminePartitionsJobTest.java | 2 + .../indexer/HadoopDruidIndexerConfigTest.java | 4 + .../indexer/HadoopDruidIndexerMapperTest.java | 66 +++ .../druid/indexer/HadoopTuningConfigTest.java | 2 + .../indexer/IndexGeneratorCombinerTest.java | 8 +- .../druid/indexer/IndexGeneratorJobTest.java | 4 +- .../io/druid/indexer/InputRowSerdeTest.java | 41 +- .../java/io/druid/indexer/JobHelperTest.java | 2 + .../indexer/path/GranularityPathSpecTest.java | 2 + .../updater/HadoopConverterJobTest.java | 4 +- .../IngestionStatsAndErrorsTaskReport.java | 102 ++++ ...IngestionStatsAndErrorsTaskReportData.java | 119 ++++ .../io/druid/indexing/common/TaskReport.java | 1 + .../indexing/common/TaskReportFileWriter.java | 5 +- .../io/druid/indexing/common/TaskStatus.java | 74 ++- .../RealtimeAppenderatorTuningConfig.java | 43 +- .../common/index/YeOldePlumberSchool.java | 2 +- .../AppenderatorDriverRealtimeIndexTask.java | 187 ++++++- .../indexing/common/task/CompactionTask.java | 12 +- .../indexing/common/task/HadoopIndexTask.java | 515 ++++++++++++++--- .../indexing/common/task/HadoopTask.java | 28 + .../druid/indexing/common/task/IndexTask.java | 523 +++++++++++++++--- .../indexing/common/task/IndexTaskUtils.java | 78 +++ .../overlord/ThreadPoolTaskRunner.java | 4 +- .../overlord/http/OverlordResource.java | 7 +- ...penderatorDriverRealtimeIndexTaskTest.java | 317 +++++++++-- .../common/task/CompactionTaskTest.java | 29 +- .../indexing/common/task/IndexTaskTest.java | 492 +++++++++++++++- .../common/task/NoopTestTaskFileWriter.java | 4 +- .../common/task/RealtimeIndexTaskTest.java | 16 +- .../common/task/TaskReportSerdeTest.java | 80 +++ .../indexing/common/task/TaskSerdeTest.java | 13 + .../indexing/overlord/TaskLifecycleTest.java | 16 + .../util/common/parsers/ObjectFlatteners.java | 2 +- .../util/common/parsers/ParseException.java | 12 + .../segment/incremental/IncrementalIndex.java | 127 ++++- .../IncrementalIndexAddResult.java | 52 ++ .../incremental/OffheapIncrementalIndex.java | 4 +- .../incremental/OnheapIncrementalIndex.java | 27 +- .../incremental/IncrementalIndexTest.java | 34 +- .../OnheapIncrementalIndexBenchmark.java | 5 +- .../incremental/TimeAndDimsCompTest.java | 12 +- .../druid/segment/indexing/TuningConfig.java | 3 + .../realtime/FireDepartmentMetrics.java | 14 +- ...ireDepartmentMetricsTaskMetricsGetter.java | 64 +++ .../realtime/appenderator/Appenderator.java | 18 +- .../AppenderatorDriverAddResult.java | 28 +- .../appenderator/AppenderatorImpl.java | 7 +- .../appenderator/BaseAppenderatorDriver.java | 3 +- .../realtime/plumber/RealtimePlumber.java | 8 +- .../druid/segment/realtime/plumber/Sink.java | 5 +- .../coordination/ChangeRequestHistory.java | 44 +- .../segment/realtime/RealtimeManagerTest.java | 2 +- .../StreamAppenderatorDriverFailTest.java | 2 +- .../ChangeRequestHistoryTest.java | 3 +- 87 files changed, 4085 insertions(+), 615 deletions(-) create mode 100644 api/src/main/java/io/druid/indexer/IngestionState.java create mode 100644 api/src/main/java/io/druid/indexer/TaskMetricsGetter.java create mode 100644 api/src/main/java/io/druid/indexer/TaskMetricsUtils.java create mode 100644 api/src/main/java/io/druid/utils/CircularBuffer.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/IngestionStatsAndErrorsTaskReport.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/IngestionStatsAndErrorsTaskReportData.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskUtils.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/task/TaskReportSerdeTest.java create mode 100644 processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAddResult.java create mode 100644 server/src/main/java/io/druid/segment/realtime/FireDepartmentMetricsTaskMetricsGetter.java diff --git a/api/src/main/java/io/druid/data/input/impl/MapInputRowParser.java b/api/src/main/java/io/druid/data/input/impl/MapInputRowParser.java index 3fa2305a7007..49d40fd88c47 100644 --- a/api/src/main/java/io/druid/data/input/impl/MapInputRowParser.java +++ b/api/src/main/java/io/druid/data/input/impl/MapInputRowParser.java @@ -72,7 +72,7 @@ public List parseBatch(Map theMap) } } catch (Exception e) { - throw new ParseException(e, "Unparseable timestamp found!"); + throw new ParseException(e, "Unparseable timestamp found! Event: %s", theMap); } return ImmutableList.of(new MapBasedInputRow(timestamp.getMillis(), dimensions, theMap)); diff --git a/api/src/main/java/io/druid/indexer/IngestionState.java b/api/src/main/java/io/druid/indexer/IngestionState.java new file mode 100644 index 000000000000..9a52e3ee1c87 --- /dev/null +++ b/api/src/main/java/io/druid/indexer/IngestionState.java @@ -0,0 +1,28 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexer; + +public enum IngestionState +{ + NOT_STARTED, + DETERMINE_PARTITIONS, + BUILD_SEGMENTS, + COMPLETED +} diff --git a/api/src/main/java/io/druid/indexer/TaskMetricsGetter.java b/api/src/main/java/io/druid/indexer/TaskMetricsGetter.java new file mode 100644 index 000000000000..c29e890010cb --- /dev/null +++ b/api/src/main/java/io/druid/indexer/TaskMetricsGetter.java @@ -0,0 +1,29 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexer; + +import java.util.List; +import java.util.Map; + +public interface TaskMetricsGetter +{ + List getKeys(); + Map getTotalMetrics(); +} diff --git a/api/src/main/java/io/druid/indexer/TaskMetricsUtils.java b/api/src/main/java/io/druid/indexer/TaskMetricsUtils.java new file mode 100644 index 000000000000..5845b245aa07 --- /dev/null +++ b/api/src/main/java/io/druid/indexer/TaskMetricsUtils.java @@ -0,0 +1,47 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexer; + +import com.google.common.collect.Maps; + +import java.util.Map; + +public class TaskMetricsUtils +{ + public static final String ROWS_PROCESSED = "rowsProcessed"; + public static final String ROWS_PROCESSED_WITH_ERRORS = "rowsProcessedWithErrors"; + public static final String ROWS_UNPARSEABLE = "rowsUnparseable"; + public static final String ROWS_THROWN_AWAY = "rowsThrownAway"; + + public static Map makeIngestionRowMetrics( + long rowsProcessed, + long rowsProcessedWithErrors, + long rowsUnparseable, + long rowsThrownAway + ) + { + Map metricsMap = Maps.newHashMap(); + metricsMap.put(ROWS_PROCESSED, rowsProcessed); + metricsMap.put(ROWS_PROCESSED_WITH_ERRORS, rowsProcessedWithErrors); + metricsMap.put(ROWS_UNPARSEABLE, rowsUnparseable); + metricsMap.put(ROWS_THROWN_AWAY, rowsThrownAway); + return metricsMap; + } +} diff --git a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java index d8d93d6055d5..898532637872 100644 --- a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java +++ b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java @@ -38,6 +38,9 @@ public class TaskStatusPlus private final TaskLocation location; private final String dataSource; + @Nullable + private final String errorMsg; + @JsonCreator public TaskStatusPlus( @JsonProperty("id") String id, @@ -47,7 +50,8 @@ public TaskStatusPlus( @JsonProperty("statusCode") @Nullable TaskState state, @JsonProperty("duration") @Nullable Long duration, @JsonProperty("location") TaskLocation location, - @JsonProperty("dataSource") String dataSource + @JsonProperty("dataSource") String dataSource, + @JsonProperty("errorMsg") String errorMsg ) { if (state != null && state.isComplete()) { @@ -61,6 +65,7 @@ public TaskStatusPlus( this.duration = duration; this.location = Preconditions.checkNotNull(location, "location"); this.dataSource = dataSource; + this.errorMsg = errorMsg; } @JsonProperty @@ -108,49 +113,53 @@ public TaskLocation getLocation() return location; } + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @Nullable + @JsonProperty("errorMsg") + public String getErrorMsg() + { + return errorMsg; + } + @Override public boolean equals(Object o) { if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { return false; } - - final TaskStatusPlus that = (TaskStatusPlus) o; - if (!id.equals(that.id)) { - return false; - } - if (!type.equals(that.type)) { - return false; - } - if (!createdTime.equals(that.createdTime)) { - return false; - } - if (!queueInsertionTime.equals(that.queueInsertionTime)) { - return false; - } - if (!Objects.equals(state, that.state)) { - return false; - } - if (!Objects.equals(duration, that.duration)) { - return false; - } - return location.equals(that.location); + TaskStatusPlus that = (TaskStatusPlus) o; + return Objects.equals(getId(), that.getId()) && + Objects.equals(getType(), that.getType()) && + Objects.equals(getCreatedTime(), that.getCreatedTime()) && + Objects.equals(getQueueInsertionTime(), that.getQueueInsertionTime()) && + getState() == that.getState() && + Objects.equals(getDuration(), that.getDuration()) && + Objects.equals(getLocation(), that.getLocation()) && + Objects.equals(getDataSource(), that.getDataSource()) && + Objects.equals(getErrorMsg(), that.getErrorMsg()); } @Override public int hashCode() { - return Objects.hash(id, type, createdTime, queueInsertionTime, state, duration, location); - } - - @JsonProperty - public String getDataSource() - { - return dataSource; + return Objects.hash( + getId(), + getType(), + getCreatedTime(), + getQueueInsertionTime(), + getState(), + getDuration(), + getLocation(), + getDataSource(), + getErrorMsg() + ); } - } diff --git a/api/src/main/java/io/druid/utils/CircularBuffer.java b/api/src/main/java/io/druid/utils/CircularBuffer.java new file mode 100644 index 000000000000..e5f8158e0efe --- /dev/null +++ b/api/src/main/java/io/druid/utils/CircularBuffer.java @@ -0,0 +1,92 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.utils; + +import com.google.common.base.Preconditions; + +/** + * A circular buffer that supports random bidirectional access. + * + * @param Type of object to be stored in the buffer + */ +public class CircularBuffer +{ + public E[] getBuffer() + { + return buffer; + } + + private final E[] buffer; + + private int start = 0; + private int size = 0; + + public CircularBuffer(int capacity) + { + Preconditions.checkArgument(capacity > 0, "Capacity must be greater than 0."); + buffer = (E[]) new Object[capacity]; + } + + public void add(E item) + { + buffer[start++] = item; + + if (start >= buffer.length) { + start = 0; + } + + if (size < buffer.length) { + size++; + } + } + + /** + * Access object at a given index, starting from the latest entry added and moving backwards. + */ + public E getLatest(int index) + { + Preconditions.checkArgument(index >= 0 && index < size, "invalid index"); + + int bufferIndex = start - index - 1; + if (bufferIndex < 0) { + bufferIndex = buffer.length + bufferIndex; + } + return buffer[bufferIndex]; + } + + /** + * Access object at a given index, starting from the earliest entry added and moving forward. + */ + public E get(int index) + { + Preconditions.checkArgument(index >= 0 && index < size, "invalid index"); + + int bufferIndex = (start - size + index) % buffer.length; + if (bufferIndex < 0) { + bufferIndex += buffer.length; + } + return buffer[bufferIndex]; + } + + public int size() + { + return size; + } +} diff --git a/api/src/test/java/io/druid/indexer/TaskStatusPlusTest.java b/api/src/test/java/io/druid/indexer/TaskStatusPlusTest.java index 200a6b4c701b..eeefe130e5be 100644 --- a/api/src/test/java/io/druid/indexer/TaskStatusPlusTest.java +++ b/api/src/test/java/io/druid/indexer/TaskStatusPlusTest.java @@ -53,7 +53,8 @@ public void testSerde() throws IOException TaskState.RUNNING, 1000L, TaskLocation.create("testHost", 1010, -1), - "ds_test" + "ds_test", + null ); final String json = mapper.writeValueAsString(status); Assert.assertEquals(status, mapper.readValue(json, TaskStatusPlus.class)); diff --git a/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java index 9848e3e62f1b..6c90ba0407e1 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java @@ -255,7 +255,7 @@ public void ingest(Blackhole blackhole) throws Exception { incIndexFilteredAgg = makeIncIndex(filteredMetrics); for (InputRow row : inputRows) { - int rv = incIndexFilteredAgg.add(row); + int rv = incIndexFilteredAgg.add(row).getRowCount(); blackhole.consume(rv); } } diff --git a/benchmarks/src/main/java/io/druid/benchmark/IncrementalIndexRowTypeBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/IncrementalIndexRowTypeBenchmark.java index 6af9a93f79e3..bd622f6017f9 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/IncrementalIndexRowTypeBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/IncrementalIndexRowTypeBenchmark.java @@ -160,7 +160,7 @@ public void normalLongs(Blackhole blackhole) throws Exception { for (int i = 0; i < maxRows; i++) { InputRow row = longRows.get(i); - int rv = incIndex.add(row); + int rv = incIndex.add(row).getRowCount(); blackhole.consume(rv); } } @@ -173,7 +173,7 @@ public void normalFloats(Blackhole blackhole) throws Exception { for (int i = 0; i < maxRows; i++) { InputRow row = floatRows.get(i); - int rv = incFloatIndex.add(row); + int rv = incFloatIndex.add(row).getRowCount(); blackhole.consume(rv); } } @@ -186,7 +186,7 @@ public void normalStrings(Blackhole blackhole) throws Exception { for (int i = 0; i < maxRows; i++) { InputRow row = stringRows.get(i); - int rv = incStrIndex.add(row); + int rv = incStrIndex.add(row).getRowCount(); blackhole.consume(rv); } } diff --git a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexIngestionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexIngestionBenchmark.java index 4fc7cf2b8d72..521ae869ab8d 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexIngestionBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexIngestionBenchmark.java @@ -119,7 +119,7 @@ public void addRows(Blackhole blackhole) throws Exception { for (int i = 0; i < rowsPerSegment; i++) { InputRow row = rows.get(i); - int rv = incIndex.add(row); + int rv = incIndex.add(row).getRowCount(); blackhole.consume(rv); } } diff --git a/common/src/main/java/io/druid/indexer/Jobby.java b/common/src/main/java/io/druid/indexer/Jobby.java index 4423cad03684..b0d26affdf40 100644 --- a/common/src/main/java/io/druid/indexer/Jobby.java +++ b/common/src/main/java/io/druid/indexer/Jobby.java @@ -19,9 +19,37 @@ package io.druid.indexer; +import io.druid.java.util.common.StringUtils; + +import javax.annotation.Nullable; +import java.util.Map; + /** */ public interface Jobby { boolean run(); + + /** + * @return A map containing statistics for a Jobby, optionally null if the Jobby is unable to provide stats. + */ + @Nullable + default Map getStats() + { + throw new UnsupportedOperationException( + StringUtils.format("This Jobby does not implement getJobStats(), Jobby class: [%s]", getClass()) + ); + } + + /** + * @return A string representing the error that caused a Jobby to fail. Can be null if the Jobby did not fail, + * or is unable to provide an error message. + */ + @Nullable + default String getErrorMessage() + { + throw new UnsupportedOperationException( + StringUtils.format("This Jobby does not implement getErrorMessage(), Jobby class: [%s]", getClass()) + ); + } } diff --git a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java index f073ce0b5691..bf5383b5fbf6 100644 --- a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java @@ -234,6 +234,8 @@ public void setUp() throws Exception null, false, false, + null, + null, null ) ) @@ -252,7 +254,7 @@ public void testIndexGeneratorJob() throws IOException private void verifyJob(IndexGeneratorJob job) throws IOException { - JobHelper.runJobs(ImmutableList.of(job), config); + Assert.assertTrue(JobHelper.runJobs(ImmutableList.of(job), config)); int segmentNum = 0; for (DateTime currTime = interval.getStart(); currTime.isBefore(interval.getEnd()); currTime = currTime.plusDays(1)) { diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 82558f0c796a..ab69c43b212d 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -49,8 +49,13 @@ import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.discovery.LookupNodeService; +import io.druid.indexer.IngestionState; +import io.druid.indexer.TaskMetricsGetter; import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReport; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import io.druid.indexing.common.TaskReport; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.CheckPointDataSourceMetadataAction; @@ -58,6 +63,7 @@ import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.task.AbstractTask; +import io.druid.indexing.common.task.IndexTaskUtils; import io.druid.indexing.common.task.RealtimeIndexTask; import io.druid.indexing.common.task.TaskResource; import io.druid.indexing.common.task.Tasks; @@ -80,6 +86,7 @@ import io.druid.segment.indexing.RealtimeIOConfig; import io.druid.segment.realtime.FireDepartment; import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.FireDepartmentMetricsTaskMetricsGetter; import io.druid.segment.realtime.RealtimeMetricsMonitor; import io.druid.segment.realtime.appenderator.Appenderator; import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; @@ -92,13 +99,9 @@ import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.server.security.Access; import io.druid.server.security.Action; -import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.AuthorizerMapper; -import io.druid.server.security.ForbiddenException; -import io.druid.server.security.Resource; -import io.druid.server.security.ResourceAction; -import io.druid.server.security.ResourceType; import io.druid.timeline.DataSegment; +import io.druid.utils.CircularBuffer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -246,6 +249,11 @@ public enum Status private volatile CopyOnWriteArrayList sequences; private ListeningExecutorService publishExecService; private final boolean useLegacy; + private CircularBuffer savedParseExceptions; + private IngestionState ingestionState; + + private TaskMetricsGetter metricsGetter; + private String errorMsg; @JsonCreator public KafkaIndexTask( @@ -276,6 +284,7 @@ public KafkaIndexTask( this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionOffsetMap()); this.topic = ioConfig.getStartPartitions().getTopic(); this.sequences = new CopyOnWriteArrayList<>(); + this.ingestionState = IngestionState.NOT_STARTED; if (context != null && context.get(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED) != null && ((boolean) context.get(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED))) { @@ -283,6 +292,9 @@ public KafkaIndexTask( } else { useLegacy = true; } + if (tuningConfig.getMaxSavedParseExceptions() > 0) { + savedParseExceptions = new CircularBuffer(tuningConfig.getMaxSavedParseExceptions()); + } resetNextCheckpointTime(); } @@ -414,11 +426,27 @@ private void createAndStartPublishExecutor() @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { - // for backwards compatibility, should be remove from versions greater than 0.12.x - if (useLegacy) { - return runLegacy(toolbox); + try { + // for backwards compatibility, should be remove from versions greater than 0.12.x + if (useLegacy) { + return runInternalLegacy(toolbox); + } else { + return runInternal(toolbox); + } } + catch (Exception e) { + log.error(e, "Encountered exception while running task."); + errorMsg = Throwables.getStackTraceAsString(e); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + return TaskStatus.failure( + getId(), + errorMsg + ); + } + } + private TaskStatus runInternal(final TaskToolbox toolbox) throws Exception + { log.info("Starting up!"); startTime = DateTimes.nowUtc(); @@ -484,6 +512,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception null ); fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + metricsGetter = new FireDepartmentMetricsTaskMetricsGetter(fireDepartmentMetrics); toolbox.getMonitorScheduler().addMonitor( new RealtimeMetricsMonitor( ImmutableList.of(fireDepartmentForMetrics), @@ -595,6 +624,8 @@ public void run() Set assignment = assignPartitionsAndSeekToNext(consumer, topic); + ingestionState = IngestionState.BUILD_SEGMENTS; + // Main loop. // Could eventually support leader/follower mode (for keeping replicas more in sync) boolean stillReading = !assignment.isEmpty(); @@ -730,7 +761,11 @@ public void run() throw new ISE("Could not allocate segment for row with timestamp[%s]", row.getTimestamp()); } - fireDepartmentMetrics.incrementProcessed(); + if (addResult.getParseException() != null) { + handleParseException(addResult.getParseException(), record); + } else { + fireDepartmentMetrics.incrementProcessed(); + } } else { fireDepartmentMetrics.incrementThrownAway(); } @@ -757,18 +792,7 @@ public void onFailure(Throwable t) } } catch (ParseException e) { - if (tuningConfig.isReportParseExceptions()) { - throw e; - } else { - log.debug( - e, - "Dropping unparseable row from partition[%d] offset[%,d].", - record.partition(), - record.offset() - ); - - fireDepartmentMetrics.incrementUnparseable(); - } + handleParseException(e, record); } nextOffsets.put(record.partition(), record.offset() + 1); @@ -806,6 +830,7 @@ public void onFailure(Throwable t) } } } + ingestionState = IngestionState.COMPLETED; } catch (Exception e) { log.error(e, "Encountered exception in run() before persisting."); @@ -904,11 +929,11 @@ public void onFailure(Throwable t) toolbox.getDataSegmentServerAnnouncer().unannounce(); } - toolbox.getTaskReportFileWriter().write(null); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); return success(); } - private TaskStatus runLegacy(final TaskToolbox toolbox) throws Exception + private TaskStatus runInternalLegacy(final TaskToolbox toolbox) throws Exception { log.info("Starting up!"); startTime = DateTimes.nowUtc(); @@ -931,6 +956,7 @@ private TaskStatus runLegacy(final TaskToolbox toolbox) throws Exception null ); fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + metricsGetter = new FireDepartmentMetricsTaskMetricsGetter(fireDepartmentMetrics); toolbox.getMonitorScheduler().addMonitor( new RealtimeMetricsMonitor( ImmutableList.of(fireDepartmentForMetrics), @@ -950,6 +976,8 @@ private TaskStatus runLegacy(final TaskToolbox toolbox) throws Exception ) ); + ingestionState = IngestionState.BUILD_SEGMENTS; + try ( final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox); final StreamAppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics); @@ -1127,11 +1155,17 @@ public void run() // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. throw new ISE("Could not allocate segment for row with timestamp[%s]", row.getTimestamp()); } - fireDepartmentMetrics.incrementProcessed(); + + if (addResult.getParseException() != null) { + handleParseException(addResult.getParseException(), record); + } else { + fireDepartmentMetrics.incrementProcessed(); + } } else { fireDepartmentMetrics.incrementThrownAway(); } } + if (isPersistRequired) { driver.persist(committerSupplier.get()); } @@ -1141,18 +1175,7 @@ public void run() )); } catch (ParseException e) { - if (tuningConfig.isReportParseExceptions()) { - throw e; - } else { - log.debug( - e, - "Dropping unparseable row from partition[%d] offset[%,d].", - record.partition(), - record.offset() - ); - - fireDepartmentMetrics.incrementUnparseable(); - } + handleParseException(e, record); } nextOffsets.put(record.partition(), record.offset() + 1); @@ -1166,6 +1189,7 @@ public void run() } } } + ingestionState = IngestionState.COMPLETED; } catch (Exception e) { log.error(e, "Encountered exception in runLegacy() before persisting."); @@ -1273,8 +1297,76 @@ public String apply(DataSegment input) toolbox.getDataSegmentServerAnnouncer().unannounce(); } - toolbox.getTaskReportFileWriter().write(null); - return success(); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + return TaskStatus.success( + getId(), + null + ); + } + + private void handleParseException(ParseException pe, ConsumerRecord record) + { + if (pe.isFromPartiallyValidRow()) { + fireDepartmentMetrics.incrementProcessedWithErrors(); + } else { + fireDepartmentMetrics.incrementUnparseable(); + } + + if (tuningConfig.isLogParseExceptions()) { + log.error( + pe, + "Encountered parse exception on row from partition[%d] offset[%d]", + record.partition(), + record.offset() + ); + } + + if (savedParseExceptions != null) { + savedParseExceptions.add(pe); + } + + if (fireDepartmentMetrics.unparseable() + fireDepartmentMetrics.processedWithErrors() + > tuningConfig.getMaxParseExceptions()) { + log.error("Max parse exceptions exceeded, terminating task..."); + throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); + } + } + + private Map getTaskCompletionReports() + { + return TaskReport.buildTaskReports( + new IngestionStatsAndErrorsTaskReport( + getId(), + new IngestionStatsAndErrorsTaskReportData( + ingestionState, + getTaskCompletionUnparseableEvents(), + getTaskCompletionRowStats(), + errorMsg + ) + ) + ); + } + + private Map getTaskCompletionUnparseableEvents() + { + Map unparseableEventsMap = Maps.newHashMap(); + List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions); + if (buildSegmentsParseExceptionMessages != null) { + unparseableEventsMap.put("buildSegments", buildSegmentsParseExceptionMessages); + } + return unparseableEventsMap; + } + + private Map getTaskCompletionRowStats() + { + Map metrics = Maps.newHashMap(); + if (metricsGetter != null) { + metrics.put( + "buildSegments", + metricsGetter.getTotalMetrics() + ); + } + return metrics; } private void checkAndMaybeThrowException() @@ -1342,17 +1434,7 @@ public boolean canRestore() */ private Access authorizationCheck(final HttpServletRequest req, Action action) { - ResourceAction resourceAction = new ResourceAction( - new Resource(dataSchema.getDataSource(), ResourceType.DATASOURCE), - action - ); - - Access access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); - if (!access.isAllowed()) { - throw new ForbiddenException(access.toString()); - } - - return access; + return IndexTaskUtils.datasourceAuthorizationCheck(req, action, getDataSource(), authorizerMapper); } @VisibleForTesting @@ -1493,6 +1575,40 @@ public Response setEndOffsetsHTTP( return setEndOffsets(offsets, resume, finish); } + @GET + @Path("/rowStats") + @Produces(MediaType.APPLICATION_JSON) + public Response getRowStats( + @Context final HttpServletRequest req + ) + { + authorizationCheck(req, Action.READ); + Map returnMap = Maps.newHashMap(); + Map totalsMap = Maps.newHashMap(); + + if (metricsGetter != null) { + totalsMap.put( + "buildSegments", + metricsGetter.getTotalMetrics() + ); + } + + returnMap.put("totals", totalsMap); + return Response.ok(returnMap).build(); + } + + @GET + @Path("/unparseableEvents") + @Produces(MediaType.APPLICATION_JSON) + public Response getUnparseableEvents( + @Context final HttpServletRequest req + ) + { + authorizationCheck(req, Action.READ); + List events = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions); + return Response.ok(events).build(); + } + public Response setEndOffsets( Map offsets, final boolean resume, @@ -2039,12 +2155,7 @@ private boolean withinMinMaxRecordTime(final InputRow row) "Encountered row with timestamp that cannot be represented as a long: [%s]", row ); - log.debug(errorMsg); - if (tuningConfig.isReportParseExceptions()) { - throw new ParseException(errorMsg); - } else { - return false; - } + throw new ParseException(errorMsg); } if (log.isDebugEnabled()) { diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index 2b4ad4017e14..b953da4fcb4b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -52,6 +52,10 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; private final Period intermediateHandoffPeriod; + private final boolean logParseExceptions; + private final int maxParseExceptions; + private final int maxSavedParseExceptions; + @JsonCreator public KafkaTuningConfig( @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @@ -63,11 +67,14 @@ public KafkaTuningConfig( @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. @JsonProperty("buildV9Directly") @Nullable Boolean buildV9Directly, - @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, + @Deprecated @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") @Nullable Long handoffConditionTimeout, @JsonProperty("resetOffsetAutomatically") @Nullable Boolean resetOffsetAutomatically, @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod + @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions ) { // Cannot be a static because default basePersistDirectory is unique per-instance @@ -95,6 +102,17 @@ public KafkaTuningConfig( this.intermediateHandoffPeriod = intermediateHandoffPeriod == null ? new Period().withDays(Integer.MAX_VALUE) : intermediateHandoffPeriod; + + if (this.reportParseExceptions) { + this.maxParseExceptions = 0; + this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions); + } else { + this.maxParseExceptions = maxParseExceptions == null ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS : maxParseExceptions; + this.maxSavedParseExceptions = maxSavedParseExceptions == null + ? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS + : maxSavedParseExceptions; + } + this.logParseExceptions = logParseExceptions == null ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS : logParseExceptions; } public static KafkaTuningConfig copyOf(KafkaTuningConfig config) @@ -112,7 +130,10 @@ public static KafkaTuningConfig copyOf(KafkaTuningConfig config) config.handoffConditionTimeout, config.resetOffsetAutomatically, config.segmentWriteOutMediumFactory, - config.intermediateHandoffPeriod + config.intermediateHandoffPeriod, + config.logParseExceptions, + config.maxParseExceptions, + config.maxSavedParseExceptions ); } @@ -208,6 +229,24 @@ public Period getIntermediateHandoffPeriod() return intermediateHandoffPeriod; } + @JsonProperty + public boolean isLogParseExceptions() + { + return logParseExceptions; + } + + @JsonProperty + public int getMaxParseExceptions() + { + return maxParseExceptions; + } + + @JsonProperty + public int getMaxSavedParseExceptions() + { + return maxSavedParseExceptions; + } + public KafkaTuningConfig withBasePersistDirectory(File dir) { return new KafkaTuningConfig( @@ -223,7 +262,10 @@ public KafkaTuningConfig withBasePersistDirectory(File dir) handoffConditionTimeout, resetOffsetAutomatically, segmentWriteOutMediumFactory, - intermediateHandoffPeriod + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); } @@ -248,7 +290,10 @@ public boolean equals(Object o) Objects.equals(basePersistDirectory, that.basePersistDirectory) && Objects.equals(indexSpec, that.indexSpec) && Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory) && - Objects.equals(intermediateHandoffPeriod, that.intermediateHandoffPeriod); + Objects.equals(intermediateHandoffPeriod, that.intermediateHandoffPeriod) && + logParseExceptions == that.logParseExceptions && + maxParseExceptions == that.maxParseExceptions && + maxSavedParseExceptions == that.maxSavedParseExceptions; } @Override @@ -266,7 +311,10 @@ public int hashCode() handoffConditionTimeout, resetOffsetAutomatically, segmentWriteOutMediumFactory, - intermediateHandoffPeriod + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); } @@ -286,6 +334,9 @@ public String toString() ", resetOffsetAutomatically=" + resetOffsetAutomatically + ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + ", intermediateHandoffPeriod=" + intermediateHandoffPeriod + + ", logParseExceptions=" + logParseExceptions + + ", maxParseExceptions=" + maxParseExceptions + + ", maxSavedParseExceptions=" + maxSavedParseExceptions + '}'; } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index d3e89a925a65..53678810cdc3 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -91,6 +91,9 @@ public KafkaSupervisorSpec( null, null, null, + null, + null, + null, null ); this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 7ebf71415a78..71a2a801baed 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -58,7 +58,10 @@ public KafkaSupervisorTuningConfig( @JsonProperty("httpTimeout") Period httpTimeout, @JsonProperty("shutdownTimeout") Period shutdownTimeout, @JsonProperty("offsetFetchPeriod") Period offsetFetchPeriod, - @JsonProperty("intermediateHandoffPeriod") Period intermediateHandoffPeriod + @JsonProperty("intermediateHandoffPeriod") Period intermediateHandoffPeriod, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions ) { super( @@ -74,7 +77,10 @@ public KafkaSupervisorTuningConfig( handoffConditionTimeout, resetOffsetAutomatically, segmentWriteOutMediumFactory, - intermediateHandoffPeriod + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); this.workerThreads = workerThreads; @@ -143,6 +149,9 @@ public String toString() ", shutdownTimeout=" + shutdownTimeout + ", offsetFetchPeriod=" + offsetFetchPeriod + ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + + ", logParseExceptions=" + isLogParseExceptions() + + ", maxParseExceptions=" + getMaxParseExceptions() + + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + '}'; } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index edb4d406cab0..cf6cb1317ad7 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -36,6 +36,14 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import io.druid.data.input.impl.FloatDimensionSchema; +import io.druid.data.input.impl.LongDimensionSchema; +import io.druid.data.input.impl.StringDimensionSchema; +import io.druid.indexer.TaskMetricsUtils; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import io.druid.indexing.common.TaskReport; +import io.druid.indexing.common.TaskReportFileWriter; +import io.druid.indexing.common.task.IndexTaskTest; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.MapCache; import io.druid.data.input.impl.DimensionsSpec; @@ -57,7 +65,6 @@ import io.druid.indexing.common.actions.TaskActionToolbox; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskStorageConfig; -import io.druid.indexing.common.task.NoopTestTaskFileWriter; import io.druid.indexing.common.task.Task; import io.druid.indexing.kafka.supervisor.KafkaSupervisor; import io.druid.indexing.kafka.test.TestBroker; @@ -101,6 +108,7 @@ import io.druid.query.SegmentDescriptor; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.filter.SelectorDimFilter; import io.druid.query.timeseries.TimeseriesQuery; @@ -152,6 +160,7 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -183,6 +192,9 @@ public class KafkaIndexTaskTest private long handoffConditionTimeout = 0; private boolean reportParseExceptions = false; + private boolean logParseExceptions = true; + private Integer maxParseExceptions = null; + private Integer maxSavedParseExceptions = null; private boolean resetOffsetAutomatically = false; private boolean doHandoff = true; private Integer maxRowsPerSegment = null; @@ -197,6 +209,7 @@ public class KafkaIndexTaskTest private List> records; private final boolean isIncrementalHandoffSupported; private final Set checkpointRequestsHash = Sets.newHashSet(); + private File reportsFile; // This should be removed in versions greater that 0.12.x // isIncrementalHandoffSupported should always be set to true in those later versions @@ -218,7 +231,13 @@ public KafkaIndexTaskTest(boolean isIncrementalHandoffSupported) new JSONParseSpec( new TimestampSpec("timestamp", "iso", null), new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")), + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ), null, null ), @@ -229,7 +248,10 @@ public KafkaIndexTaskTest(boolean isIncrementalHandoffSupported) ), Map.class ), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + }, new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), null, objectMapper @@ -238,17 +260,21 @@ public KafkaIndexTaskTest(boolean isIncrementalHandoffSupported) private static List> generateRecords(String topic) { return ImmutableList.of( - new ProducerRecord(topic, 0, null, JB("2008", "a", "y", 1.0f)), - new ProducerRecord(topic, 0, null, JB("2009", "b", "y", 1.0f)), - new ProducerRecord(topic, 0, null, JB("2010", "c", "y", 1.0f)), - new ProducerRecord(topic, 0, null, JB("2011", "d", "y", 1.0f)), - new ProducerRecord(topic, 0, null, JB("2011", "e", "y", 1.0f)), - new ProducerRecord(topic, 0, null, JB("246140482-04-24T15:36:27.903Z", "x", "z", 1.0f)), + new ProducerRecord(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")), + new ProducerRecord(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")), + new ProducerRecord(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")), + new ProducerRecord(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")), + new ProducerRecord(topic, 0, null, JB("2011", "e", "y", "10", "20.0", "1.0")), + new ProducerRecord(topic, 0, null, JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")), new ProducerRecord(topic, 0, null, StringUtils.toUtf8("unparseable")), + new ProducerRecord(topic, 0, null, StringUtils.toUtf8("unparseable2")), new ProducerRecord(topic, 0, null, null), - new ProducerRecord(topic, 0, null, JB("2013", "f", "y", 1.0f)), - new ProducerRecord(topic, 1, null, JB("2012", "g", "y", 1.0f)), - new ProducerRecord(topic, 1, null, JB("2011", "h", "y", 1.0f)) + new ProducerRecord(topic, 0, null, JB("2013", "f", "y", "10", "20.0", "1.0")), + new ProducerRecord(topic, 0, null, JB("2049", "f", "y", "notanumber", "20.0", "1.0")), + new ProducerRecord(topic, 0, null, JB("2049", "f", "y", "10", "notanumber", "1.0")), + new ProducerRecord(topic, 0, null, JB("2049", "f", "y", "10", "20.0", "notanumber")), + new ProducerRecord(topic, 1, null, JB("2012", "g", "y", "10", "20.0", "1.0")), + new ProducerRecord(topic, 1, null, JB("2011", "h", "y", "10", "20.0", "1.0")) ); } @@ -297,9 +323,13 @@ public void setupTest() throws IOException { handoffConditionTimeout = 0; reportParseExceptions = false; + logParseExceptions = true; + maxParseExceptions = null; + maxSavedParseExceptions = null; doHandoff = true; topic = getTopicName(); records = generateRecords(topic); + reportsFile = File.createTempFile("KafkaIndexTaskTestReports-" + System.currentTimeMillis(), "json"); makeToolboxFactory(); } @@ -313,7 +343,7 @@ public void tearDownTest() runningTasks.clear(); } - + reportsFile.delete(); destroyToolboxFactory(); } @@ -459,7 +489,7 @@ public void testIncrementalHandOff() throws Exception // of events fetched across two partitions from Kafka final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 0L)); final KafkaPartitions checkpoint2 = new KafkaPartitions(topic, ImmutableMap.of(0, 4L, 1, 2L)); - final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 9L, 1, 2L)); + final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L)); final KafkaIndexTask task = createTask( null, new KafkaIOConfig( @@ -496,8 +526,8 @@ public void testIncrementalHandOff() throws Exception // Check metrics Assert.assertEquals(8, task.getFireDepartmentMetrics().processed()); - Assert.assertEquals(1, task.getFireDepartmentMetrics().unparseable()); - Assert.assertEquals(2, task.getFireDepartmentMetrics().thrownAway()); + Assert.assertEquals(3, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(1, task.getFireDepartmentMetrics().thrownAway()); // Check published metadata SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); @@ -509,7 +539,7 @@ public void testIncrementalHandOff() throws Exception SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 9L, 1, 2L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -924,6 +954,10 @@ public void testReportParseExceptions() throws Exception { reportParseExceptions = true; + // these will be ignored because reportParseExceptions is true + maxParseExceptions = 1000; + maxSavedParseExceptions = 2; + // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { for (ProducerRecord record : records) { @@ -953,12 +987,165 @@ public void testReportParseExceptions() throws Exception // Check metrics Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); - Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(1, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); + + // Check published metadata + Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + } + + @Test(timeout = 60_000L) + public void testMultipleParseExceptionsSuccess() throws Exception + { + reportParseExceptions = false; + maxParseExceptions = 6; + maxSavedParseExceptions = 6; + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + } + + final KafkaIndexTask task = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 13L)), + kafkaServer.consumerProperties(), + true, + false, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + TaskStatus status = future.get(); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); + Assert.assertEquals(null, status.getErrorMsg()); + + // Check metrics + Assert.assertEquals(4, task.getFireDepartmentMetrics().processed()); + Assert.assertEquals(3, task.getFireDepartmentMetrics().processedWithErrors()); + Assert.assertEquals(3, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(1, task.getFireDepartmentMetrics().thrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + SegmentDescriptor desc3 = SD(task, "2013/P1D", 0); + SegmentDescriptor desc4 = SD(task, "2049/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 13L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedMetrics = ImmutableMap.of( + "buildSegments", + ImmutableMap.of( + TaskMetricsUtils.ROWS_PROCESSED, 4, + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, 3, + TaskMetricsUtils.ROWS_UNPARSEABLE, 3, + TaskMetricsUtils.ROWS_THROWN_AWAY, 1 + ) + ); + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + + Map unparseableEvents = ImmutableMap.of( + "buildSegments", + Arrays.asList( + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=20.0, met1=notanumber}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [Unable to parse value[notanumber] for field[met1],]", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=notanumber, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to float,]", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=notanumber, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to long,]", + "Unable to parse row [unparseable2]", + "Unable to parse row [unparseable]", + "Encountered row with timestamp that cannot be represented as a long: [MapBasedInputRow{timestamp=246140482-04-24T15:36:27.903Z, event={timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}]" + ) + ); + + Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); + } + + @Test(timeout = 60_000L) + public void testMultipleParseExceptionsFailure() throws Exception + { + reportParseExceptions = false; + maxParseExceptions = 2; + maxSavedParseExceptions = 2; + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + } + + final KafkaIndexTask task = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 10L)), + kafkaServer.consumerProperties(), + true, + false, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + TaskStatus status = future.get(); + + // Wait for task to exit + Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); + IndexTaskTest.checkTaskStatusErrorMsgForParseExceptionsExceeded(status); + + // Check metrics + Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().processedWithErrors()); + Assert.assertEquals(3, task.getFireDepartmentMetrics().unparseable()); Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); // Check published metadata Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedMetrics = ImmutableMap.of( + "buildSegments", + ImmutableMap.of( + TaskMetricsUtils.ROWS_PROCESSED, 3, + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, 0, + TaskMetricsUtils.ROWS_UNPARSEABLE, 3, + TaskMetricsUtils.ROWS_THROWN_AWAY, 0 + ) + ); + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + + Map unparseableEvents = ImmutableMap.of( + "buildSegments", + Arrays.asList( + "Unable to parse row [unparseable2]", + "Unable to parse row [unparseable]" + ) + ); + + Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); } @Test(timeout = 60_000L) @@ -1051,7 +1238,7 @@ public void testRunConflicting() throws Exception new KafkaIOConfig( "sequence1", new KafkaPartitions(topic, ImmutableMap.of(0, 3L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 9L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 10L)), kafkaServer.consumerProperties(), true, false, @@ -1081,8 +1268,8 @@ public void testRunConflicting() throws Exception Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); - Assert.assertEquals(1, task2.getFireDepartmentMetrics().unparseable()); - Assert.assertEquals(2, task2.getFireDepartmentMetrics().thrownAway()); + Assert.assertEquals(3, task2.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(1, task2.getFireDepartmentMetrics().thrownAway()); // Check published segments & metadata, should all be from the first task SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); @@ -1120,7 +1307,7 @@ public void testRunConflictingWithoutTransactions() throws Exception new KafkaIOConfig( "sequence1", new KafkaPartitions(topic, ImmutableMap.of(0, 3L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 9L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 10L)), kafkaServer.consumerProperties(), false, false, @@ -1156,8 +1343,8 @@ public void testRunConflictingWithoutTransactions() throws Exception Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); - Assert.assertEquals(1, task2.getFireDepartmentMetrics().unparseable()); - Assert.assertEquals(2, task2.getFireDepartmentMetrics().thrownAway()); + Assert.assertEquals(3, task2.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(1, task2.getFireDepartmentMetrics().thrownAway()); // Check published segments & metadata SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1); @@ -1548,8 +1735,8 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception // Check metrics Assert.assertEquals(4, task.getFireDepartmentMetrics().processed()); - Assert.assertEquals(1, task.getFireDepartmentMetrics().unparseable()); - Assert.assertEquals(1, task.getFireDepartmentMetrics().thrownAway()); + Assert.assertEquals(2, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); // Check published metadata SegmentDescriptor desc1 = SD(task, "2009/P1D", 0); @@ -1788,7 +1975,10 @@ private KafkaIndexTask createTask( handoffConditionTimeout, resetOffsetAutomatically, null, - intermediateHandoffPeriod + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); final Map context = isIncrementalHandoffSupported ? ImmutableMap.of(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true) @@ -1827,7 +2017,10 @@ private KafkaIndexTask createTask( handoffConditionTimeout, resetOffsetAutomatically, null, - null + null, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); if (isIncrementalHandoffSupported) { context.put(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true); @@ -2034,7 +2227,7 @@ public List getLocations() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), - new NoopTestTaskFileWriter() + new TaskReportFileWriter(reportsFile) ); } @@ -2132,11 +2325,18 @@ public long countEvents(final Task task) return results.isEmpty() ? 0 : results.get(0).getValue().getLongMetric("rows"); } - private static byte[] JB(String timestamp, String dim1, String dim2, double met1) + private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) { try { return new ObjectMapper().writeValueAsBytes( - ImmutableMap.of("timestamp", timestamp, "dim1", dim1, "dim2", dim2, "met1", met1) + ImmutableMap.builder() + .put("timestamp", timestamp) + .put("dim1", dim1) + .put("dim2", dim2) + .put("dimLong", dimLong) + .put("dimFloat", dimFloat) + .put("met1", met1) + .build() ); } catch (Exception e) { @@ -2149,4 +2349,17 @@ private SegmentDescriptor SD(final Task task, final String intervalString, final final Interval interval = Intervals.of(intervalString); return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum); } + + private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException + { + Map taskReports = objectMapper.readValue( + reportsFile, + new TypeReference>() + { + } + ); + return IngestionStatsAndErrorsTaskReportData.getPayloadFromTaskReports( + taskReports + ); + } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java index 435b08dd05d9..38c6537318c8 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java @@ -113,6 +113,9 @@ public void testCopyOf() 5L, null, null, + null, + null, + null, null ); KafkaTuningConfig copy = KafkaTuningConfig.copyOf(original); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 34d3958a13ae..21c51d5a899a 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -201,6 +201,9 @@ public void setupTest() TEST_HTTP_TIMEOUT, TEST_SHUTDOWN_TIMEOUT, null, + null, + null, + null, null ); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java index 44e75805d77d..7955aae220b7 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java @@ -46,6 +46,7 @@ import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Partitioner; import org.apache.hadoop.mapreduce.Reducer; @@ -55,6 +56,7 @@ import org.joda.time.DateTimeComparator; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -69,6 +71,8 @@ public class DetermineHashedPartitionsJob implements Jobby { private static final Logger log = new Logger(DetermineHashedPartitionsJob.class); private final HadoopDruidIndexerConfig config; + private String failureCause; + private Job groupByJob; public DetermineHashedPartitionsJob( HadoopDruidIndexerConfig config @@ -86,7 +90,7 @@ public boolean run() * in the final segment. */ final long startTime = System.currentTimeMillis(); - final Job groupByJob = Job.getInstance( + groupByJob = Job.getInstance( new Configuration(), StringUtils.format("%s-determine_partitions_hashed-%s", config.getDataSource(), config.getIntervals()) ); @@ -121,6 +125,7 @@ public boolean run() if (!groupByJob.waitForCompletion(true)) { log.error("Job failed: %s", groupByJob.getJobID()); + failureCause = Utils.getFailureMessage(groupByJob, config.JSON_MAPPER); return false; } @@ -197,6 +202,7 @@ public boolean run() log.info("Path[%s] didn't exist!?", partitionInfoPath); } } + config.setShardSpecs(shardSpecs); log.info( "DetermineHashedPartitionsJob took %d millis", @@ -210,6 +216,42 @@ public boolean run() } } + @Override + public Map getStats() + { + if (groupByJob == null) { + return null; + } + + try { + Counters jobCounters = groupByJob.getCounters(); + + Map metrics = TaskMetricsUtils.makeIngestionRowMetrics( + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_COUNTER).getValue(), + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_WITH_ERRORS_COUNTER).getValue(), + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_UNPARSEABLE_COUNTER).getValue(), + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_THROWN_AWAY_COUNTER).getValue() + ); + + return metrics; + } + catch (IllegalStateException ise) { + log.debug("Couldn't get counters due to job state"); + return null; + } + catch (Exception e) { + log.debug(e, "Encountered exception in getStats()."); + return null; + } + } + + @Nullable + @Override + public String getErrorMessage() + { + return failureCause; + } + public static class DetermineCardinalityMapper extends HadoopDruidIndexerMapper { private static HashFunction hashFunction = Hashing.murmur3_128(); @@ -269,9 +311,12 @@ protected void innerMap( } interval = maybeInterval.get(); } + hyperLogLogs .get(interval) .add(hashFunction.hashBytes(HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsBytes(groupKey)).asBytes()); + + context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_COUNTER).increment(1); } @Override diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java index 46f8aa6fed83..8052469daa23 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java @@ -54,6 +54,7 @@ import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.InvalidJobConfException; +import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.Mapper; @@ -70,6 +71,7 @@ import org.joda.time.Interval; import org.joda.time.chrono.ISOChronology; +import javax.annotation.Nullable; import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -100,6 +102,10 @@ public class DeterminePartitionsJob implements Jobby private final HadoopDruidIndexerConfig config; + private Job groupByJob; + + private String failureCause; + public DeterminePartitionsJob( HadoopDruidIndexerConfig config ) @@ -124,7 +130,7 @@ public boolean run() } if (!config.getPartitionsSpec().isAssumeGrouped()) { - final Job groupByJob = Job.getInstance( + groupByJob = Job.getInstance( new Configuration(), StringUtils.format("%s-determine_partitions_groupby-%s", config.getDataSource(), config.getIntervals()) ); @@ -155,6 +161,7 @@ public boolean run() if (!groupByJob.waitForCompletion(true)) { log.error("Job failed: %s", groupByJob.getJobID()); + failureCause = Utils.getFailureMessage(groupByJob, config.JSON_MAPPER); return false; } } else { @@ -212,6 +219,7 @@ public boolean run() if (!dimSelectionJob.waitForCompletion(true)) { log.error("Job failed: %s", dimSelectionJob.getJobID().toString()); + failureCause = Utils.getFailureMessage(dimSelectionJob, config.JSON_MAPPER); return false; } @@ -255,6 +263,42 @@ public boolean run() } } + @Override + public Map getStats() + { + if (groupByJob == null) { + return null; + } + + try { + Counters jobCounters = groupByJob.getCounters(); + + Map metrics = TaskMetricsUtils.makeIngestionRowMetrics( + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_COUNTER).getValue(), + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_WITH_ERRORS_COUNTER).getValue(), + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_UNPARSEABLE_COUNTER).getValue(), + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_THROWN_AWAY_COUNTER).getValue() + ); + + return metrics; + } + catch (IllegalStateException ise) { + log.debug("Couldn't get counters due to job state"); + return null; + } + catch (Exception e) { + log.debug(e, "Encountered exception in getStats()."); + return null; + } + } + + @Nullable + @Override + public String getErrorMessage() + { + return failureCause; + } + public static class DeterminePartitionsGroupByMapper extends HadoopDruidIndexerMapper { private Granularity rollupGranularity = null; @@ -282,6 +326,8 @@ protected void innerMap( new BytesWritable(HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsBytes(groupKey)), NullWritable.get() ); + + context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_COUNTER).increment(1); } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java index 0229b073beeb..3adcf31c00e1 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java @@ -38,6 +38,7 @@ public class HadoopDruidDetermineConfigurationJob implements Jobby { private static final Logger log = new Logger(HadoopDruidDetermineConfigurationJob.class); private final HadoopDruidIndexerConfig config; + private Jobby job; @Inject public HadoopDruidDetermineConfigurationJob( @@ -50,12 +51,11 @@ public HadoopDruidDetermineConfigurationJob( @Override public boolean run() { - List jobs = Lists.newArrayList(); - JobHelper.ensurePaths(config); if (config.isDeterminingPartitions()) { - jobs.add(config.getPartitionsSpec().getPartitionJob(config)); + job = config.getPartitionsSpec().getPartitionJob(config); + return JobHelper.runSingleJob(job, config); } else { int shardsPerInterval = config.getPartitionsSpec().getNumShards(); Map> shardSpecs = Maps.newTreeMap(); @@ -86,10 +86,27 @@ public boolean run() } } config.setShardSpecs(shardSpecs); + return true; } + } - return JobHelper.runJobs(jobs, config); + @Override + public Map getStats() + { + if (job == null) { + return null; + } + return job.getStats(); } + @Override + public String getErrorMessage() + { + if (job == null) { + return null; + } + + return job.getErrorMessage(); + } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index 569bebdcffa5..f07bcb702343 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -126,7 +126,11 @@ public void configure(Binder binder) public enum IndexJobCounters { - INVALID_ROW_COUNTER + INVALID_ROW_COUNTER, + ROWS_PROCESSED_COUNTER, + ROWS_PROCESSED_WITH_ERRORS_COUNTER, + ROWS_UNPARSEABLE_COUNTER, + ROWS_THROWN_AWAY_COUNTER } public static HadoopDruidIndexerConfig fromSpec(HadoopIngestionSpec spec) @@ -370,6 +374,16 @@ public int getShardSpecCount(Bucket bucket) return schema.getTuningConfig().getShardSpecs().get(bucket.time.getMillis()).size(); } + public boolean isLogParseExceptions() + { + return schema.getTuningConfig().isLogParseExceptions(); + } + + public int getMaxParseExceptions() + { + return schema.getTuningConfig().getMaxParseExceptions(); + } + /** * Job instance should have Configuration set (by calling {@link #addJobProperties(Job)} * or via injected system properties) before this method is called. The {@link PathSpec} may diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java index d46b73cd4c7c..e4096122c025 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerJob.java @@ -26,7 +26,9 @@ import io.druid.java.util.common.logger.Logger; import io.druid.timeline.DataSegment; +import javax.annotation.Nullable; import java.util.List; +import java.util.Map; /** */ @@ -92,8 +94,28 @@ public boolean run() ); - JobHelper.runJobs(jobs, config); - return true; + return JobHelper.runJobs(jobs, config); + } + + @Override + public Map getStats() + { + if (indexJob == null) { + return null; + } + + return indexJob.getStats(); + } + + @Nullable + @Override + public String getErrorMessage() + { + if (indexJob == null) { + return null; + } + + return indexJob.getErrorMessage(); } public List getPublishedSegments() diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java index b5707c4fa598..f905a24c9c23 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java @@ -24,12 +24,15 @@ import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.StringInputRowParser; import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.Intervals; import io.druid.java.util.common.RE; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.collect.Utils; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.ParseException; import io.druid.segment.indexing.granularity.GranularitySpec; import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.Mapper; import java.io.IOException; @@ -63,37 +66,70 @@ public HadoopDruidIndexerConfig getConfig() protected void map(Object key, Object value, Context context) throws IOException, InterruptedException { try { - final List inputRows; - try { - inputRows = parseInputRow(value, parser); - } - catch (ParseException e) { - if (reportParseExceptions) { - throw e; - } - log.debug(e, "Ignoring invalid row [%s] due to parsing error", value); - context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.INVALID_ROW_COUNTER).increment(1); - return; // we're ignoring this invalid row - } + final List inputRows = parseInputRow(value, parser); for (InputRow inputRow : inputRows) { - if (inputRow == null) { - // Throw away null rows from the parser. - log.debug("Throwing away row [%s]", value); - continue; + try { + if (inputRow == null) { + // Throw away null rows from the parser. + log.debug("Throwing away row [%s]", value); + context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_THROWN_AWAY_COUNTER).increment(1); + continue; + } + + if (!Intervals.ETERNITY.contains(inputRow.getTimestamp())) { + final String errorMsg = StringUtils.format( + "Encountered row with timestamp that cannot be represented as a long: [%s]", + inputRow + ); + throw new ParseException(errorMsg); + } + + if (!granularitySpec.bucketIntervals().isPresent() + || granularitySpec.bucketInterval(DateTimes.utc(inputRow.getTimestampFromEpoch())) + .isPresent()) { + innerMap(inputRow, context, reportParseExceptions); + } else { + context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_THROWN_AWAY_COUNTER).increment(1); + } } - if (!granularitySpec.bucketIntervals().isPresent() - || granularitySpec.bucketInterval(DateTimes.utc(inputRow.getTimestampFromEpoch())) - .isPresent()) { - innerMap(inputRow, context, reportParseExceptions); + catch (ParseException pe) { + handleParseException(pe, context); } } } + catch (ParseException pe) { + handleParseException(pe, context); + } catch (RuntimeException e) { throw new RE(e, "Failure on row[%s]", value); } } + private void handleParseException(ParseException pe, Context context) + { + context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.INVALID_ROW_COUNTER).increment(1); + Counter unparseableCounter = context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_UNPARSEABLE_COUNTER); + Counter processedWithErrorsCounter = context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_WITH_ERRORS_COUNTER); + + if (pe.isFromPartiallyValidRow()) { + processedWithErrorsCounter.increment(1); + } else { + unparseableCounter.increment(1); + } + + if (config.isLogParseExceptions()) { + log.error(pe, "Encountered parse exception: "); + } + + long rowsUnparseable = unparseableCounter.getValue(); + long rowsProcessedWithError = processedWithErrorsCounter.getValue(); + if (rowsUnparseable + rowsProcessedWithError > config.getMaxParseExceptions()) { + log.error("Max parse exceptions exceeded, terminating task..."); + throw new RuntimeException("Max parse exceptions exceeded, terminating task...", pe); + } + } + private static List parseInputRow(Object value, InputRowParser parser) { if (parser instanceof StringInputRowParser && value instanceof Text) { diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java index aeb72c033f87..a997e40d2994 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java @@ -31,6 +31,7 @@ import io.druid.segment.IndexSpec; import io.druid.segment.indexing.TuningConfig; +import javax.annotation.Nullable; import java.util.List; import java.util.Map; @@ -67,6 +68,8 @@ public static HadoopTuningConfig makeDefaultTuningConfig() DEFAULT_NUM_BACKGROUND_PERSIST_THREADS, false, false, + null, + null, null ); } @@ -88,6 +91,8 @@ public static HadoopTuningConfig makeDefaultTuningConfig() private final boolean forceExtendableShardSpecs; private final boolean useExplicitVersion; private final List allowedHadoopPrefix; + private final boolean logParseExceptions; + private final int maxParseExceptions; @JsonCreator public HadoopTuningConfig( @@ -100,7 +105,7 @@ public HadoopTuningConfig( final @JsonProperty("leaveIntermediate") boolean leaveIntermediate, final @JsonProperty("cleanupOnFailure") Boolean cleanupOnFailure, final @JsonProperty("overwriteFiles") boolean overwriteFiles, - final @JsonProperty("ignoreInvalidRows") boolean ignoreInvalidRows, + final @Deprecated @JsonProperty("ignoreInvalidRows") boolean ignoreInvalidRows, final @JsonProperty("jobProperties") Map jobProperties, final @JsonProperty("combineText") boolean combineText, final @JsonProperty("useCombiner") Boolean useCombiner, @@ -111,7 +116,9 @@ public HadoopTuningConfig( final @JsonProperty("numBackgroundPersistThreads") Integer numBackgroundPersistThreads, final @JsonProperty("forceExtendableShardSpecs") boolean forceExtendableShardSpecs, final @JsonProperty("useExplicitVersion") boolean useExplicitVersion, - final @JsonProperty("allowedHadoopPrefix") List allowedHadoopPrefix + final @JsonProperty("allowedHadoopPrefix") List allowedHadoopPrefix, + final @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + final @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions ) { this.workingPath = workingPath; @@ -138,6 +145,13 @@ public HadoopTuningConfig( Preconditions.checkArgument(this.numBackgroundPersistThreads >= 0, "Not support persistBackgroundCount < 0"); this.useExplicitVersion = useExplicitVersion; this.allowedHadoopPrefix = allowedHadoopPrefix == null ? ImmutableList.of() : allowedHadoopPrefix; + + if (!this.ignoreInvalidRows) { + this.maxParseExceptions = 0; + } else { + this.maxParseExceptions = maxParseExceptions == null ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS : maxParseExceptions; + } + this.logParseExceptions = logParseExceptions == null ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS : logParseExceptions; } @JsonProperty @@ -253,6 +267,18 @@ public List getUserAllowedHadoopPrefix() return allowedHadoopPrefix; } + @JsonProperty + public boolean isLogParseExceptions() + { + return logParseExceptions; + } + + @JsonProperty + public int getMaxParseExceptions() + { + return maxParseExceptions; + } + public HadoopTuningConfig withWorkingPath(String path) { return new HadoopTuningConfig( @@ -274,7 +300,9 @@ public HadoopTuningConfig withWorkingPath(String path) numBackgroundPersistThreads, forceExtendableShardSpecs, useExplicitVersion, - allowedHadoopPrefix + allowedHadoopPrefix, + logParseExceptions, + maxParseExceptions ); } @@ -299,7 +327,9 @@ public HadoopTuningConfig withVersion(String ver) numBackgroundPersistThreads, forceExtendableShardSpecs, useExplicitVersion, - allowedHadoopPrefix + allowedHadoopPrefix, + logParseExceptions, + maxParseExceptions ); } @@ -324,7 +354,9 @@ public HadoopTuningConfig withShardSpecs(Map> specs numBackgroundPersistThreads, forceExtendableShardSpecs, useExplicitVersion, - allowedHadoopPrefix + allowedHadoopPrefix, + logParseExceptions, + maxParseExceptions ); } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index cd2389f52ea9..b5708b94354c 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -43,6 +43,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.logger.Logger; +import io.druid.java.util.common.parsers.ParseException; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.BaseProgressIndicator; import io.druid.segment.ProgressIndicator; @@ -137,6 +138,7 @@ public static List getPublishedSegments(HadoopDruidIndexerConfig co private final HadoopDruidIndexerConfig config; private IndexGeneratorStats jobStats; + private Job job; public IndexGeneratorJob( HadoopDruidIndexerConfig config @@ -155,7 +157,7 @@ protected void setReducerClass(final Job job) public boolean run() { try { - Job job = Job.getInstance( + job = Job.getInstance( new Configuration(), StringUtils.format("%s-index-generator-%s", config.getDataSource(), config.getIntervals()) ); @@ -225,6 +227,45 @@ public boolean run() } } + @Override + public Map getStats() + { + if (job == null) { + return null; + } + + try { + Counters jobCounters = job.getCounters(); + + Map metrics = TaskMetricsUtils.makeIngestionRowMetrics( + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_COUNTER).getValue(), + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_WITH_ERRORS_COUNTER).getValue(), + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_UNPARSEABLE_COUNTER).getValue(), + jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_THROWN_AWAY_COUNTER).getValue() + ); + + return metrics; + } + catch (IllegalStateException ise) { + log.debug("Couldn't get counters due to job state"); + return null; + } + catch (Exception e) { + log.debug(e, "Encountered exception in getStats()."); + return null; + } + } + + @Override + public String getErrorMessage() + { + if (job == null) { + return null; + } + + return Utils.getFailureMessage(job, config.JSON_MAPPER); + } + private static IncrementalIndex makeIncrementalIndex( Bucket theBucket, AggregatorFactory[] aggs, @@ -316,10 +357,18 @@ protected void innerMap( // type SegmentInputRow serves as a marker that these InputRow instances have already been combined // and they contain the columns as they show up in the segment after ingestion, not what you would see in raw // data - byte[] serializedInputRow = inputRow instanceof SegmentInputRow ? - InputRowSerde.toBytes(typeHelperMap, inputRow, aggsForSerializingSegmentInputRow, reportParseExceptions) - : - InputRowSerde.toBytes(typeHelperMap, inputRow, aggregators, reportParseExceptions); + InputRowSerde.SerializeResult serializeResult = inputRow instanceof SegmentInputRow ? + InputRowSerde.toBytes( + typeHelperMap, + inputRow, + aggsForSerializingSegmentInputRow + ) + : + InputRowSerde.toBytes( + typeHelperMap, + inputRow, + aggregators + ); context.write( new SortableBytes( @@ -330,8 +379,19 @@ protected void innerMap( .put(hashedDimensions) .array() ).toBytesWritable(), - new BytesWritable(serializedInputRow) + new BytesWritable(serializeResult.getSerializedRow()) + ); + + ParseException pe = IncrementalIndex.getCombinedParseException( + inputRow, + serializeResult.getParseExceptionMessages(), + null ); + if (pe != null) { + throw pe; + } else { + context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_COUNTER).increment(1); + } } } @@ -406,11 +466,11 @@ private void flushIndexToContextAndClose(BytesWritable key, IncrementalIndex ind InputRow inputRow = getInputRowFromRow(row, dimensions); // reportParseExceptions is true as any unparseable data is already handled by the mapper. - byte[] serializedRow = InputRowSerde.toBytes(typeHelperMap, inputRow, combiningAggs, true); + InputRowSerde.SerializeResult serializeResult = InputRowSerde.toBytes(typeHelperMap, inputRow, combiningAggs); context.write( key, - new BytesWritable(serializedRow) + new BytesWritable(serializeResult.getSerializedRow()) ); } index.close(); @@ -629,7 +689,7 @@ public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) context.progress(); final InputRow inputRow = index.formatRow(InputRowSerde.fromBytes(typeHelperMap, bw.getBytes(), aggregators)); - int numRows = index.add(inputRow); + int numRows = index.add(inputRow).getRowCount(); ++lineCount; diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java b/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java index cd1dd531604a..4f0d9d4c81a3 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java @@ -47,6 +47,7 @@ import java.io.DataInput; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -65,7 +66,7 @@ public interface IndexSerdeTypeHelper { ValueType getType(); - void serialize(ByteArrayDataOutput out, Object value, boolean reportParseExceptions); + void serialize(ByteArrayDataOutput out, Object value); T deserialize(ByteArrayDataInput in); } @@ -96,6 +97,31 @@ public static Map getTypeHelperMap(DimensionsSpec return typeHelperMap; } + public static class SerializeResult + { + private final byte[] serializedRow; + private final List parseExceptionMessages; + + public SerializeResult( + final byte[] serializedRow, + final List parseExceptionMessages + ) + { + this.serializedRow = serializedRow; + this.parseExceptionMessages = parseExceptionMessages; + } + + public byte[] getSerializedRow() + { + return serializedRow; + } + + public List getParseExceptionMessages() + { + return parseExceptionMessages; + } + } + public static class StringIndexSerdeTypeHelper implements IndexSerdeTypeHelper> { @Override @@ -105,7 +131,7 @@ public ValueType getType() } @Override - public void serialize(ByteArrayDataOutput out, Object value, boolean reportParseExceptions) + public void serialize(ByteArrayDataOutput out, Object value) { List values = Rows.objectToStrings(value); try { @@ -137,15 +163,27 @@ public ValueType getType() } @Override - public void serialize(ByteArrayDataOutput out, Object value, boolean reportParseExceptions) + public void serialize(ByteArrayDataOutput out, Object value) { - Long ret = DimensionHandlerUtils.convertObjectToLong(value, reportParseExceptions); + ParseException exceptionToThrow = null; + Long ret = null; + try { + ret = DimensionHandlerUtils.convertObjectToLong(value, true); + } + catch (ParseException pe) { + exceptionToThrow = pe; + } + if (ret == null) { // remove null -> zero conversion when https://github.com/druid-io/druid/pull/5278 series of patches is merged // we'll also need to change the serialized encoding so that it can represent numeric nulls ret = DimensionHandlerUtils.ZERO_LONG; } out.writeLong(ret); + + if (exceptionToThrow != null) { + throw exceptionToThrow; + } } @Override @@ -164,15 +202,27 @@ public ValueType getType() } @Override - public void serialize(ByteArrayDataOutput out, Object value, boolean reportParseExceptions) + public void serialize(ByteArrayDataOutput out, Object value) { - Float ret = DimensionHandlerUtils.convertObjectToFloat(value, reportParseExceptions); + ParseException exceptionToThrow = null; + Float ret = null; + try { + ret = DimensionHandlerUtils.convertObjectToFloat(value, true); + } + catch (ParseException pe) { + exceptionToThrow = pe; + } + if (ret == null) { // remove null -> zero conversion when https://github.com/druid-io/druid/pull/5278 series of patches is merged // we'll also need to change the serialized encoding so that it can represent numeric nulls ret = DimensionHandlerUtils.ZERO_FLOAT; } out.writeFloat(ret); + + if (exceptionToThrow != null) { + throw exceptionToThrow; + } } @Override @@ -191,15 +241,27 @@ public ValueType getType() } @Override - public void serialize(ByteArrayDataOutput out, Object value, boolean reportParseExceptions) + public void serialize(ByteArrayDataOutput out, Object value) { - Double ret = DimensionHandlerUtils.convertObjectToDouble(value, reportParseExceptions); + ParseException exceptionToThrow = null; + Double ret = null; + try { + ret = DimensionHandlerUtils.convertObjectToDouble(value, true); + } + catch (ParseException pe) { + exceptionToThrow = pe; + } + if (ret == null) { // remove null -> zero conversion when https://github.com/druid-io/druid/pull/5278 series of patches is merged // we'll also need to change the serialized encoding so that it can represent numeric nulls ret = DimensionHandlerUtils.ZERO_DOUBLE; } out.writeDouble(ret); + + if (exceptionToThrow != null) { + throw exceptionToThrow; + } } @Override @@ -209,14 +271,14 @@ public Double deserialize(ByteArrayDataInput in) } } - public static final byte[] toBytes( + public static final SerializeResult toBytes( final Map typeHelperMap, final InputRow row, - AggregatorFactory[] aggs, - boolean reportParseExceptions + AggregatorFactory[] aggs ) { try { + List parseExceptionMessages = new ArrayList<>(); ByteArrayDataOutput out = ByteStreams.newDataOutput(); //write timestamp @@ -233,7 +295,13 @@ public static final byte[] toBytes( typeHelper = STRING_HELPER; } writeString(dim, out); - typeHelper.serialize(out, row.getRaw(dim), reportParseExceptions); + + try { + typeHelper.serialize(out, row.getRaw(dim)); + } + catch (ParseException pe) { + parseExceptionMessages.add(pe.getMessage()); + } } } @@ -264,10 +332,8 @@ public InputRow get() } catch (ParseException e) { // "aggregate" can throw ParseExceptions if a selector expects something but gets something else. - if (reportParseExceptions) { - throw new ParseException(e, "Encountered parse error for aggregator[%s]", k); - } log.debug(e, "Encountered parse error, skipping aggregator[%s].", k); + parseExceptionMessages.add(e.getMessage()); } String t = aggFactory.getTypeName(); @@ -287,7 +353,7 @@ public InputRow get() } } - return out.toByteArray(); + return new SerializeResult(out.toByteArray(), parseExceptionMessages); } catch (IOException ex) { throw new RuntimeException(ex); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java index db44e01e422a..221c8a033939 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java @@ -345,19 +345,40 @@ public static void ensurePaths(HadoopDruidIndexerConfig config) } } + public static boolean runSingleJob(Jobby job, HadoopDruidIndexerConfig config) + { + boolean succeeded = job.run(); + + if (!config.getSchema().getTuningConfig().isLeaveIntermediate()) { + if (succeeded || config.getSchema().getTuningConfig().isCleanupOnFailure()) { + Path workingPath = config.makeIntermediatePath(); + log.info("Deleting path[%s]", workingPath); + try { + Configuration conf = injectSystemProperties(new Configuration()); + config.addJobProperties(conf); + workingPath.getFileSystem(conf).delete(workingPath, true); + } + catch (IOException e) { + log.error(e, "Failed to cleanup path[%s]", workingPath); + } + } + } + + return succeeded; + } + public static boolean runJobs(List jobs, HadoopDruidIndexerConfig config) { - String failedMessage = null; + boolean succeeded = true; for (Jobby job : jobs) { - if (failedMessage == null) { - if (!job.run()) { - failedMessage = StringUtils.format("Job[%s] failed!", job.getClass()); - } + if (!job.run()) { + succeeded = false; + break; } } if (!config.getSchema().getTuningConfig().isLeaveIntermediate()) { - if (failedMessage == null || config.getSchema().getTuningConfig().isCleanupOnFailure()) { + if (succeeded || config.getSchema().getTuningConfig().isCleanupOnFailure()) { Path workingPath = config.makeIntermediatePath(); log.info("Deleting path[%s]", workingPath); try { @@ -371,11 +392,7 @@ public static boolean runJobs(List jobs, HadoopDruidIndexerConfig config) } } - if (failedMessage != null) { - throw new ISE(failedMessage); - } - - return true; + return succeeded; } public static DataSegment serializeOutIndex( diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/Utils.java b/indexing-hadoop/src/main/java/io/druid/indexer/Utils.java index 3f3523e74049..1a899df18ee3 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/Utils.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/Utils.java @@ -20,15 +20,19 @@ package io.druid.indexer; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Maps; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.jackson.JacksonUtils; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.logger.Logger; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.TaskCompletionEvent; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.ReflectionUtils; @@ -41,6 +45,7 @@ */ public class Utils { + private static final Logger log = new Logger(Utils.class); private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); public static OutputStream makePathAndOutputStream(JobContext job, Path outputPath, boolean deleteExisting) @@ -123,4 +128,25 @@ public static void storeStats( stats ); } + + public static String getFailureMessage(Job failedJob, ObjectMapper jsonMapper) + { + try { + Map taskDiagsMap = Maps.newHashMap(); + TaskCompletionEvent[] completionEvents = failedJob.getTaskCompletionEvents(0, 100); + for (TaskCompletionEvent tce : completionEvents) { + String[] taskDiags = failedJob.getTaskDiagnostics(tce.getTaskAttemptId()); + String combinedTaskDiags = ""; + for (String taskDiag : taskDiags) { + combinedTaskDiags += taskDiag; + } + taskDiagsMap.put(tce.getTaskAttemptId().toString(), combinedTaskDiags); + } + return jsonMapper.writeValueAsString(taskDiagsMap); + } + catch (IOException | InterruptedException ie) { + log.error(ie, "couldn't get failure cause for job [%s]", failedJob.getJobName()); + return null; + } + } } diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java index 59d1f3103553..b7bb444c1cc7 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java @@ -371,7 +371,7 @@ private void testIngestion( ) throws Exception { IndexGeneratorJob job = new IndexGeneratorJob(config); - JobHelper.runJobs(ImmutableList.of(job), config); + Assert.assertTrue(JobHelper.runJobs(ImmutableList.of(job), config)); File segmentFolder = new File( StringUtils.format( @@ -492,6 +492,8 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig( null, false, false, + null, + null, null ) ) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java index 4ef1d02bd804..8d656f20d4b8 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java @@ -210,6 +210,8 @@ public DetermineHashedPartitionsJobTest( null, false, false, + null, + null, null ) ); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java index 0496cb3d7f5d..908425198783 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java @@ -271,6 +271,8 @@ public DeterminePartitionsJobTest( null, false, false, + null, + null, null ) ) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java index 2b134c2aa364..2ed052eafec9 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -96,6 +96,8 @@ public void testHashedBucketSelection() null, false, false, + null, + null, null ) ); @@ -170,6 +172,8 @@ public void testNoneShardSpecBucketSelection() null, false, false, + null, + null, null ) ); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java index dd7975d9b0a2..c4e5db5fe01f 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java @@ -42,11 +42,15 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.Mapper; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -144,6 +148,8 @@ public void testHadoopyStringParserWithTransformSpec() throws Exception ); final Mapper.Context mapContext = EasyMock.mock(Mapper.Context.class); EasyMock.expect(mapContext.getConfiguration()).andReturn(hadoopConfig).once(); + EasyMock.expect(mapContext.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_THROWN_AWAY_COUNTER)) + .andReturn(getTestCounter()); EasyMock.replay(mapContext); mapper.setup(mapContext); final List> rows = ImmutableList.of( @@ -189,6 +195,66 @@ private static Map rowToMap(final InputRow row) return builder.build(); } + private static Counter getTestCounter() + { + return new Counter() + { + @Override + public void setDisplayName(String displayName) + { + + } + + @Override + public String getName() + { + return null; + } + + @Override + public String getDisplayName() + { + return null; + } + + @Override + public long getValue() + { + return 0; + } + + @Override + public void setValue(long value) + { + + } + + @Override + public void increment(long incr) + { + + } + + @Override + public Counter getUnderlyingCounter() + { + return null; + } + + @Override + public void write(DataOutput out) throws IOException + { + + } + + @Override + public void readFields(DataInput in) throws IOException + { + + } + }; + } + public static class MyMapper extends HadoopDruidIndexerMapper { private final List rows = new ArrayList<>(); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java index 753379ba709b..9e4a26a22b43 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java @@ -58,6 +58,8 @@ public void testSerde() throws Exception null, true, true, + null, + null, null ); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java index 9eb75e27b779..3bfb1fb39832 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java @@ -175,8 +175,8 @@ public void testMultipleRowsMerged() throws Exception ) ); List rows = Lists.newArrayList( - new BytesWritable(InputRowSerde.toBytes(typeHelperMap, row1, aggregators, true)), - new BytesWritable(InputRowSerde.toBytes(typeHelperMap, row2, aggregators, true)) + new BytesWritable(InputRowSerde.toBytes(typeHelperMap, row1, aggregators).getSerializedRow()), + new BytesWritable(InputRowSerde.toBytes(typeHelperMap, row2, aggregators).getSerializedRow()) ); Reducer.Context context = EasyMock.createNiceMock(Reducer.Context.class); @@ -253,8 +253,8 @@ public void testMultipleRowsNotMerged() throws Exception Map typeHelperMap = InputRowSerde.getTypeHelperMap(dimensionsSpec); List rows = Lists.newArrayList( - new BytesWritable(InputRowSerde.toBytes(typeHelperMap, row1, aggregators, true)), - new BytesWritable(InputRowSerde.toBytes(typeHelperMap, row2, aggregators, true)) + new BytesWritable(InputRowSerde.toBytes(typeHelperMap, row1, aggregators).getSerializedRow()), + new BytesWritable(InputRowSerde.toBytes(typeHelperMap, row2, aggregators).getSerializedRow()) ); Reducer.Context context = EasyMock.createNiceMock(Reducer.Context.class); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java index 5fc2d1c5f256..1b422b6c223e 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java @@ -523,6 +523,8 @@ public void setUp() throws Exception null, forceExtendableShardSpecs, false, + null, + null, null ) ) @@ -580,7 +582,7 @@ public void testIndexGeneratorJob() throws IOException private void verifyJob(IndexGeneratorJob job) throws IOException { - JobHelper.runJobs(ImmutableList.of(job), config); + Assert.assertTrue(JobHelper.runJobs(ImmutableList.of(job), config)); int segmentNum = 0; for (DateTime currTime = interval.getStart(); currTime.isBefore(interval.getEnd()); currTime = currTime.plusDays(1)) { diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/InputRowSerdeTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/InputRowSerdeTest.java index 71609e42dd32..0b72d31a71aa 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/InputRowSerdeTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/InputRowSerdeTest.java @@ -30,7 +30,6 @@ import io.druid.data.input.impl.StringDimensionSchema; import io.druid.hll.HyperLogLogCollector; import io.druid.jackson.AggregatorsModule; -import io.druid.java.util.common.parsers.ParseException; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregator; @@ -124,7 +123,8 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory) null ); - byte[] data = InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories, false); // Ignore Unparseable aggregator + byte[] data = InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories) + .getSerializedRow(); // Ignore Unparseable aggregator InputRow out = InputRowSerde.fromBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), data, aggregatorFactories); Assert.assertEquals(timestamp, out.getTimestampFromEpoch()); @@ -173,14 +173,21 @@ public void testThrowParseExceptions() null ); - expectedException.expect(ParseException.class); - expectedException.expectMessage("Encountered parse error for aggregator[unparseable]"); - InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories, true); + InputRowSerde.SerializeResult result = InputRowSerde.toBytes( + InputRowSerde.getTypeHelperMap(dimensionsSpec), + in, + aggregatorFactories + ); + Assert.assertEquals( + Arrays.asList("Unable to parse value[m3v] for field[m3]"), + result.getParseExceptionMessages() + ); } @Test public void testDimensionParseExceptions() { + InputRowSerde.SerializeResult result; InputRow in = new MapBasedInputRow( timestamp, dims, @@ -190,8 +197,6 @@ public void testDimensionParseExceptions() new LongSumAggregatorFactory("m2out", "m2") }; - expectedException.expect(ParseException.class); - expectedException.expectMessage("could not convert value [d1v] to long"); DimensionsSpec dimensionsSpec = new DimensionsSpec( Arrays.asList( new LongDimensionSchema("d1") @@ -199,10 +204,12 @@ public void testDimensionParseExceptions() null, null ); - InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories, true); + result = InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories); + Assert.assertEquals( + Arrays.asList("could not convert value [d1v] to long"), + result.getParseExceptionMessages() + ); - expectedException.expect(ParseException.class); - expectedException.expectMessage("could not convert value [d1v] to float"); dimensionsSpec = new DimensionsSpec( Arrays.asList( new FloatDimensionSchema("d1") @@ -210,10 +217,12 @@ public void testDimensionParseExceptions() null, null ); - InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories, true); + result = InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories); + Assert.assertEquals( + Arrays.asList("could not convert value [d1v] to float"), + result.getParseExceptionMessages() + ); - expectedException.expect(ParseException.class); - expectedException.expectMessage("could not convert value [d1v] to double"); dimensionsSpec = new DimensionsSpec( Arrays.asList( new DoubleDimensionSchema("d1") @@ -221,6 +230,10 @@ public void testDimensionParseExceptions() null, null ); - InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories, true); + result = InputRowSerde.toBytes(InputRowSerde.getTypeHelperMap(dimensionsSpec), in, aggregatorFactories); + Assert.assertEquals( + Arrays.asList("could not convert value [d1v] to double"), + result.getParseExceptionMessages() + ); } } diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java index e8b5888e324a..c768e2c8e10d 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java @@ -126,6 +126,8 @@ public void setup() throws Exception null, false, false, + null, + null, null ) ) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java index 3aec576f4f56..b4caeed21f43 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java @@ -73,6 +73,8 @@ public class GranularityPathSpecTest null, false, false, + null, + null, null ); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java index 255e0a9dc4eb..c8d763544340 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java @@ -212,6 +212,8 @@ public InputStream openStream() throws IOException null, false, false, + null, + null, null ) ) @@ -251,7 +253,7 @@ public boolean run() new SQLMetadataStorageUpdaterJobHandler(connector) ) ); - JobHelper.runJobs(jobs, hadoopDruidIndexerConfig); + Assert.assertTrue(JobHelper.runJobs(jobs, hadoopDruidIndexerConfig)); } private List getDataSegments( diff --git a/indexing-service/src/main/java/io/druid/indexing/common/IngestionStatsAndErrorsTaskReport.java b/indexing-service/src/main/java/io/druid/indexing/common/IngestionStatsAndErrorsTaskReport.java new file mode 100644 index 000000000000..3c636f2678f4 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/IngestionStatsAndErrorsTaskReport.java @@ -0,0 +1,102 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +@JsonTypeName("ingestionStatsAndErrors") +public class IngestionStatsAndErrorsTaskReport implements TaskReport +{ + public static final String REPORT_KEY = "ingestionStatsAndErrors"; + + @JsonProperty + private String taskId; + + @JsonProperty + private IngestionStatsAndErrorsTaskReportData payload; + + public IngestionStatsAndErrorsTaskReport( + @JsonProperty("taskId") String taskId, + @JsonProperty("payload") IngestionStatsAndErrorsTaskReportData payload + ) + { + this.taskId = taskId; + this.payload = payload; + } + + @Override + public String getTaskId() + { + return taskId; + } + + @Override + public String getReportKey() + { + return REPORT_KEY; + } + + @Override + public Object getPayload() + { + return payload; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IngestionStatsAndErrorsTaskReport that = (IngestionStatsAndErrorsTaskReport) o; + return Objects.equals(getTaskId(), that.getTaskId()) && + Objects.equals(getPayload(), that.getPayload()); + } + + @Override + public int hashCode() + { + return Objects.hash(getTaskId(), getPayload()); + } + + @Override + public String toString() + { + return "IngestionStatsAndErrorsTaskReport{" + + "taskId='" + taskId + '\'' + + ", payload=" + payload + + '}'; + } + + // TaskReports are put into a Map and serialized. + // Jackson doesn't normally serialize the TaskReports with a "type" field in that situation, + // so explictly serialize the "type" field (otherwise, deserialization fails). + @JsonProperty("type") + private String getType() + { + return "ingestionStatsAndErrors"; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/IngestionStatsAndErrorsTaskReportData.java b/indexing-service/src/main/java/io/druid/indexing/common/IngestionStatsAndErrorsTaskReportData.java new file mode 100644 index 000000000000..24114e5f11eb --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/IngestionStatsAndErrorsTaskReportData.java @@ -0,0 +1,119 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.indexer.IngestionState; + +import java.util.Map; +import java.util.Objects; + +public class IngestionStatsAndErrorsTaskReportData +{ + @JsonProperty + private IngestionState ingestionState; + + @JsonProperty + private Map unparseableEvents; + + @JsonProperty + private Map rowStats; + + @JsonProperty + private String errorMsg; + + public IngestionStatsAndErrorsTaskReportData( + @JsonProperty("ingestionState") IngestionState ingestionState, + @JsonProperty("unparseableEvents") Map unparseableEvents, + @JsonProperty("rowStats") Map rowStats, + @JsonProperty("errorMsg") String errorMsg + ) + { + this.ingestionState = ingestionState; + this.unparseableEvents = unparseableEvents; + this.rowStats = rowStats; + this.errorMsg = errorMsg; + } + + @JsonProperty + public IngestionState getIngestionState() + { + return ingestionState; + } + + @JsonProperty + public Map getUnparseableEvents() + { + return unparseableEvents; + } + + @JsonProperty + public Map getRowStats() + { + return rowStats; + } + + @JsonProperty + public String getErrorMsg() + { + return errorMsg; + } + + public static IngestionStatsAndErrorsTaskReportData getPayloadFromTaskReports( + Map taskReports + ) + { + return (IngestionStatsAndErrorsTaskReportData) taskReports.get(IngestionStatsAndErrorsTaskReport.REPORT_KEY) + .getPayload(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IngestionStatsAndErrorsTaskReportData that = (IngestionStatsAndErrorsTaskReportData) o; + return getIngestionState() == that.getIngestionState() && + Objects.equals(getUnparseableEvents(), that.getUnparseableEvents()) && + Objects.equals(getRowStats(), that.getRowStats()) && + Objects.equals(getErrorMsg(), that.getErrorMsg()); + } + + @Override + public int hashCode() + { + return Objects.hash(getIngestionState(), getUnparseableEvents(), getRowStats(), getErrorMsg()); + } + + @Override + public String toString() + { + return "IngestionStatsAndErrorsTaskReportData{" + + "ingestionState=" + ingestionState + + ", unparseableEvents=" + unparseableEvents + + ", rowStats=" + rowStats + + ", errorMsg='" + errorMsg + '\'' + + '}'; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskReport.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskReport.java index eff6520741ba..335b75bc3c66 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskReport.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskReport.java @@ -31,6 +31,7 @@ */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { + @JsonSubTypes.Type(name = "ingestionStatsAndErrors", value = IngestionStatsAndErrorsTaskReport.class) }) public interface TaskReport { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskReportFileWriter.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskReportFileWriter.java index eb5e9d9db40b..392fdc527aea 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskReportFileWriter.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskReportFileWriter.java @@ -24,6 +24,7 @@ import org.apache.commons.io.FileUtils; import java.io.File; +import java.util.Map; public class TaskReportFileWriter { @@ -37,14 +38,14 @@ public TaskReportFileWriter(File reportFile) this.reportsFile = reportFile; } - public void write(TaskReport report) + public void write(Map reports) { try { final File reportsFileParent = reportsFile.getParentFile(); if (reportsFileParent != null) { FileUtils.forceMkdir(reportsFileParent); } - objectMapper.writeValue(reportsFile, report); + objectMapper.writeValue(reportsFile, reports); } catch (Exception e) { log.error(e, "Encountered exception in write()."); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java index b0249440c7a1..61e64917ca82 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java @@ -34,40 +34,66 @@ */ public class TaskStatus { + public static final int MAX_ERROR_MSG_LENGTH = 100; + public static TaskStatus running(String taskId) { - return new TaskStatus(taskId, TaskState.RUNNING, -1); + return new TaskStatus(taskId, TaskState.RUNNING, -1, null); } public static TaskStatus success(String taskId) { - return new TaskStatus(taskId, TaskState.SUCCESS, -1); + return new TaskStatus(taskId, TaskState.SUCCESS, -1, null); + } + + public static TaskStatus success(String taskId, String errorMsg) + { + return new TaskStatus(taskId, TaskState.SUCCESS, -1, errorMsg); } public static TaskStatus failure(String taskId) { - return new TaskStatus(taskId, TaskState.FAILED, -1); + return new TaskStatus(taskId, TaskState.FAILED, -1, null); + } + + public static TaskStatus failure(String taskId, String errorMsg) + { + return new TaskStatus(taskId, TaskState.FAILED, -1, errorMsg); } public static TaskStatus fromCode(String taskId, TaskState code) { - return new TaskStatus(taskId, code, -1); + return new TaskStatus(taskId, code, -1, null); + } + + // The error message can be large, so truncate it to avoid storing large objects in zookeeper/metadata storage. + // The full error message will be available via a TaskReport. + private static String truncateErrorMsg(String errorMsg) + { + if (errorMsg != null && errorMsg.length() > MAX_ERROR_MSG_LENGTH) { + return errorMsg.substring(0, MAX_ERROR_MSG_LENGTH) + "..."; + } else { + return errorMsg; + } } private final String id; private final TaskState status; private final long duration; + private final String errorMsg; @JsonCreator protected TaskStatus( @JsonProperty("id") String id, @JsonProperty("status") TaskState status, - @JsonProperty("duration") long duration + @JsonProperty("duration") long duration, + @JsonProperty("errorMsg") String errorMsg ) { this.id = id; this.status = status; this.duration = duration; + this.errorMsg = truncateErrorMsg(errorMsg); // Check class invariants. Preconditions.checkNotNull(id, "id"); @@ -92,6 +118,12 @@ public long getDuration() return duration; } + @JsonProperty("errorMsg") + public String getErrorMsg() + { + return errorMsg; + } + /** * Signals that a task is not yet complete, and is still runnable on a worker. Exactly one of isRunnable, * isSuccess, or isFailure will be true at any one time. @@ -141,7 +173,18 @@ public boolean isFailure() public TaskStatus withDuration(long _duration) { - return new TaskStatus(id, status, _duration); + return new TaskStatus(id, status, _duration, errorMsg); + } + + @Override + public String toString() + { + return Objects.toStringHelper(this) + .add("id", id) + .add("status", status) + .add("duration", duration) + .add("errorMsg", errorMsg) + .toString(); } @Override @@ -154,24 +197,15 @@ public boolean equals(Object o) return false; } TaskStatus that = (TaskStatus) o; - return duration == that.duration && - java.util.Objects.equals(id, that.id) && - status == that.status; + return getDuration() == that.getDuration() && + java.util.Objects.equals(getId(), that.getId()) && + status == that.status && + java.util.Objects.equals(getErrorMsg(), that.getErrorMsg()); } @Override public int hashCode() { - return java.util.Objects.hash(id, status, duration); - } - - @Override - public String toString() - { - return Objects.toStringHelper(this) - .add("id", id) - .add("status", status) - .add("duration", duration) - .toString(); + return java.util.Objects.hash(getId(), status, getDuration(), getErrorMsg()); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index 41550f345bed..ca83fb51e59a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -73,6 +73,10 @@ private static long getDefaultMaxBytesInMemory() @Nullable private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + private final boolean logParseExceptions; + private final int maxParseExceptions; + private final int maxSavedParseExceptions; + @JsonCreator public RealtimeAppenderatorTuningConfig( @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, @@ -86,7 +90,10 @@ public RealtimeAppenderatorTuningConfig( @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, @JsonProperty("publishAndHandoffTimeout") Long publishAndHandoffTimeout, @JsonProperty("alertTimeout") Long alertTimeout, - @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions ) { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; @@ -110,6 +117,17 @@ public RealtimeAppenderatorTuningConfig( this.alertTimeout = alertTimeout == null ? defaultAlertTimeout : alertTimeout; Preconditions.checkArgument(this.alertTimeout >= 0, "alertTimeout must be >= 0"); this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + + if (this.reportParseExceptions) { + this.maxParseExceptions = 0; + this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions); + } else { + this.maxParseExceptions = maxParseExceptions == null ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS : maxParseExceptions; + this.maxSavedParseExceptions = maxSavedParseExceptions == null + ? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS + : maxSavedParseExceptions; + } + this.logParseExceptions = logParseExceptions == null ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS : logParseExceptions; } @Override @@ -192,6 +210,24 @@ public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() return segmentWriteOutMediumFactory; } + @JsonProperty + public boolean isLogParseExceptions() + { + return logParseExceptions; + } + + @JsonProperty + public int getMaxParseExceptions() + { + return maxParseExceptions; + } + + @JsonProperty + public int getMaxSavedParseExceptions() + { + return maxSavedParseExceptions; + } + public RealtimeAppenderatorTuningConfig withBasePersistDirectory(File dir) { return new RealtimeAppenderatorTuningConfig( @@ -206,7 +242,10 @@ public RealtimeAppenderatorTuningConfig withBasePersistDirectory(File dir) reportParseExceptions, publishAndHandoffTimeout, alertTimeout, - segmentWriteOutMediumFactory + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java index 3bc5aefb33dc..536500bd191e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java @@ -131,7 +131,7 @@ public int add(InputRow row, Supplier committerSupplier) throws Index return -1; } - final int numRows = sink.add(row, false); + final int numRows = sink.add(row, false).getRowCount(); if (!sink.canAppendRow()) { persist(committerSupplier.get()); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index cf408e14aaa0..9e2751ffe7ff 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -19,13 +19,16 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Optional; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import io.druid.data.input.Committer; @@ -35,8 +38,13 @@ import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.discovery.LookupNodeService; +import io.druid.indexer.IngestionState; +import io.druid.indexer.TaskMetricsGetter; import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReport; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import io.druid.indexing.common.TaskReport; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; @@ -58,6 +66,7 @@ import io.druid.segment.indexing.RealtimeIOConfig; import io.druid.segment.realtime.FireDepartment; import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.FireDepartmentMetricsTaskMetricsGetter; import io.druid.segment.realtime.RealtimeMetricsMonitor; import io.druid.segment.realtime.appenderator.Appenderator; import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; @@ -65,14 +74,27 @@ import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; import io.druid.segment.realtime.appenderator.StreamAppenderatorDriver; import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; +import io.druid.segment.realtime.firehose.ChatHandler; +import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.segment.realtime.firehose.ClippedFirehoseFactory; import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; import io.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; import io.druid.segment.realtime.plumber.Committers; +import io.druid.server.security.Action; +import io.druid.server.security.AuthorizerMapper; +import io.druid.utils.CircularBuffer; import org.apache.commons.io.FileUtils; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; import java.io.File; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Queue; import java.util.Random; @@ -84,7 +106,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -public class AppenderatorDriverRealtimeIndexTask extends AbstractTask +public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements ChatHandler { private static final String CTX_KEY_LOOKUP_TIER = "lookupTier"; @@ -121,6 +143,9 @@ private static String makeTaskId(RealtimeAppenderatorIngestionSpec spec) @JsonIgnore private volatile FireDepartmentMetrics metrics = null; + @JsonIgnore + private TaskMetricsGetter metricsGetter; + @JsonIgnore private volatile boolean gracefullyStopped = false; @@ -130,12 +155,29 @@ private static String makeTaskId(RealtimeAppenderatorIngestionSpec spec) @JsonIgnore private volatile Thread runThread = null; + @JsonIgnore + private CircularBuffer savedParseExceptions; + + @JsonIgnore + private final Optional chatHandlerProvider; + + @JsonIgnore + private final AuthorizerMapper authorizerMapper; + + @JsonIgnore + private IngestionState ingestionState; + + @JsonIgnore + private String errorMsg; + @JsonCreator public AppenderatorDriverRealtimeIndexTask( @JsonProperty("id") String id, @JsonProperty("resource") TaskResource taskResource, @JsonProperty("spec") RealtimeAppenderatorIngestionSpec spec, - @JsonProperty("context") Map context + @JsonProperty("context") Map context, + @JacksonInject ChatHandlerProvider chatHandlerProvider, + @JacksonInject AuthorizerMapper authorizerMapper ) { super( @@ -147,6 +189,14 @@ public AppenderatorDriverRealtimeIndexTask( ); this.spec = spec; this.pendingHandoffs = new ConcurrentLinkedQueue<>(); + this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); + this.authorizerMapper = authorizerMapper; + + if (spec.getTuningConfig().getMaxSavedParseExceptions() > 0) { + savedParseExceptions = new CircularBuffer<>(spec.getTuningConfig().getMaxSavedParseExceptions()); + } + + this.ingestionState = IngestionState.NOT_STARTED; } @Override @@ -207,6 +257,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null ); this.metrics = fireDepartmentForMetrics.getMetrics(); + metricsGetter = new FireDepartmentMetricsTaskMetricsGetter(metrics); Supplier committerSupplier = null; final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); @@ -217,6 +268,13 @@ dataSchema, new RealtimeIOConfig(null, null, null), null StreamAppenderatorDriver driver = newDriver(dataSchema, appenderator, toolbox, metrics); try { + if (chatHandlerProvider.isPresent()) { + log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); + chatHandlerProvider.get().register(getId(), this, false); + } else { + log.warn("No chat handler detected"); + } + toolbox.getDataSegmentServerAnnouncer().announce(); toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); @@ -248,6 +306,8 @@ dataSchema, new RealtimeIOConfig(null, null, null), null } } + ingestionState = IngestionState.BUILD_SEGMENTS; + // Time to read data! while (!gracefullyStopped && firehoseDrainableByClosing && firehose.hasMore()) { try { @@ -273,19 +333,20 @@ dataSchema, new RealtimeIOConfig(null, null, null), null throw new ISE("Could not allocate segment for row with timestamp[%s]", inputRow.getTimestamp()); } - metrics.incrementProcessed(); + if (addResult.getParseException() != null) { + handleParseException(addResult.getParseException()); + } else { + metrics.incrementProcessed(); + } } } catch (ParseException e) { - if (tuningConfig.isReportParseExceptions()) { - throw e; - } else { - log.debug(e, "Discarded row due to exception, considering unparseable."); - metrics.incrementUnparseable(); - } + handleParseException(e); } } + ingestionState = IngestionState.COMPLETED; + if (!gracefullyStopped) { synchronized (this) { if (gracefullyStopped) { @@ -312,9 +373,18 @@ dataSchema, new RealtimeIOConfig(null, null, null), null catch (Throwable e) { log.makeAlert(e, "Exception aborted realtime processing[%s]", dataSchema.getDataSource()) .emit(); - throw e; + errorMsg = Throwables.getStackTraceAsString(e); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + return TaskStatus.failure( + getId(), + errorMsg + ); } finally { + if (chatHandlerProvider.isPresent()) { + chatHandlerProvider.get().unregister(getId()); + } + CloseQuietly.close(firehose); CloseQuietly.close(appenderator); CloseQuietly.close(driver); @@ -326,7 +396,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null } log.info("Job done!"); - toolbox.getTaskReportFileWriter().write(null); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); return TaskStatus.success(getId()); } @@ -387,6 +457,41 @@ public RealtimeAppenderatorIngestionSpec getSpec() return spec; } + + @GET + @Path("/rowStats") + @Produces(MediaType.APPLICATION_JSON) + public Response getRowStats( + @Context final HttpServletRequest req + ) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + Map returnMap = Maps.newHashMap(); + Map totalsMap = Maps.newHashMap(); + + if (metricsGetter != null) { + totalsMap.put( + "buildSegments", + metricsGetter.getTotalMetrics() + ); + } + + returnMap.put("totals", totalsMap); + return Response.ok(returnMap).build(); + } + + @GET + @Path("/unparseableEvents") + @Produces(MediaType.APPLICATION_JSON) + public Response getUnparseableEvents( + @Context final HttpServletRequest req + ) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + List events = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions); + return Response.ok(events).build(); + } + /** * Is a firehose from this factory drainable by closing it? If so, we should drain on stopGracefully rather than * abruptly stopping. @@ -404,6 +509,66 @@ && isFirehoseDrainableByClosing(((TimedShutoffFirehoseFactory) firehoseFactory). && isFirehoseDrainableByClosing(((ClippedFirehoseFactory) firehoseFactory).getDelegate())); } + private Map getTaskCompletionReports() + { + return TaskReport.buildTaskReports( + new IngestionStatsAndErrorsTaskReport( + getId(), + new IngestionStatsAndErrorsTaskReportData( + ingestionState, + getTaskCompletionUnparseableEvents(), + getTaskCompletionRowStats(), + errorMsg + ) + ) + ); + } + + private Map getTaskCompletionUnparseableEvents() + { + Map unparseableEventsMap = Maps.newHashMap(); + List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions); + if (buildSegmentsParseExceptionMessages != null) { + unparseableEventsMap.put("buildSegments", buildSegmentsParseExceptionMessages); + } + return unparseableEventsMap; + } + + private Map getTaskCompletionRowStats() + { + Map metricsMap = Maps.newHashMap(); + if (metricsGetter != null) { + metricsMap.put( + "buildSegments", + metricsGetter.getTotalMetrics() + ); + } + return metricsMap; + } + + private void handleParseException(ParseException pe) + { + if (pe.isFromPartiallyValidRow()) { + metrics.incrementProcessedWithErrors(); + } else { + metrics.incrementUnparseable(); + } + + if (spec.getTuningConfig().isLogParseExceptions()) { + log.error(pe, "Encountered parse exception: "); + } + + if (savedParseExceptions != null) { + savedParseExceptions.add(pe); + } + + if (metrics.unparseable() + metrics.processedWithErrors() + > spec.getTuningConfig().getMaxParseExceptions()) { + log.error("Max parse exceptions exceeded, terminating task..."); + throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); + } + } + private void setupTimeoutAlert() { if (spec.getTuningConfig().getAlertTimeout() > 0) { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java index 411715a814fd..a751958f200d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -65,6 +65,7 @@ import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.loading.SegmentLoadingException; +import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; @@ -102,6 +103,9 @@ public class CompactionTask extends AbstractTask @JsonIgnore private IndexTask indexTaskSpec; + @JsonIgnore + private final AuthorizerMapper authorizerMapper; + @JsonCreator public CompactionTask( @JsonProperty("id") final String id, @@ -112,7 +116,8 @@ public CompactionTask( @Nullable @JsonProperty("dimensions") final DimensionsSpec dimensionsSpec, @Nullable @JsonProperty("tuningConfig") final IndexTuningConfig tuningConfig, @Nullable @JsonProperty("context") final Map context, - @JacksonInject ObjectMapper jsonMapper + @JacksonInject ObjectMapper jsonMapper, + @JacksonInject AuthorizerMapper authorizerMapper ) { super(getOrMakeId(id, TYPE, dataSource), null, taskResource, dataSource, context); @@ -125,6 +130,7 @@ public CompactionTask( this.tuningConfig = tuningConfig; this.jsonMapper = jsonMapper; this.segmentProvider = segments == null ? new SegmentProvider(dataSource, interval) : new SegmentProvider(segments); + this.authorizerMapper = authorizerMapper; } @JsonProperty @@ -195,7 +201,9 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception getTaskResource(), getDataSource(), ingestionSpec, - getContext() + getContext(), + authorizerMapper, + null ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index f8e80e569a36..984a9fd6a523 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -23,20 +23,27 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; import io.druid.indexer.HadoopDruidDetermineConfigurationJob; import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.HadoopDruidIndexerJob; import io.druid.indexer.HadoopIngestionSpec; -import io.druid.indexer.Jobby; +import io.druid.indexer.IngestionState; import io.druid.indexer.MetadataStorageUpdaterJobHandler; +import io.druid.indexer.TaskMetricsGetter; +import io.druid.indexer.TaskMetricsUtils; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReport; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskLockType; +import io.druid.indexing.common.TaskReport; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.LockAcquireAction; @@ -47,14 +54,29 @@ import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.realtime.firehose.ChatHandler; +import io.druid.segment.realtime.firehose.ChatHandlerProvider; +import io.druid.server.security.Action; +import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; import org.joda.time.Interval; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.SortedSet; -public class HadoopIndexTask extends HadoopTask +public class HadoopIndexTask extends HadoopTask implements ChatHandler { private static final Logger log = new Logger(HadoopIndexTask.class); @@ -72,6 +94,30 @@ private static String getTheDataSource(HadoopIngestionSpec spec) @JsonIgnore private final ObjectMapper jsonMapper; + @JsonIgnore + private final AuthorizerMapper authorizerMapper; + + @JsonIgnore + private final Optional chatHandlerProvider; + + @JsonIgnore + private InnerProcessingStatsGetter determinePartitionsStatsGetter; + + @JsonIgnore + private InnerProcessingStatsGetter buildSegmentsStatsGetter; + + @JsonIgnore + private IngestionState ingestionState; + + @JsonIgnore + private HadoopDetermineConfigInnerProcessingStatus determineConfigStatus = null; + + @JsonIgnore + private HadoopIndexGeneratorInnerProcessingStatus buildSegmentsStatus = null; + + @JsonIgnore + private String errorMsg; + /** * @param spec is used by the HadoopDruidIndexerJob to set up the appropriate parameters * for creating Druid index segments. It may be modified. @@ -90,7 +136,9 @@ public HadoopIndexTask( @JsonProperty("hadoopDependencyCoordinates") List hadoopDependencyCoordinates, @JsonProperty("classpathPrefix") String classpathPrefix, @JacksonInject ObjectMapper jsonMapper, - @JsonProperty("context") Map context + @JsonProperty("context") Map context, + @JacksonInject AuthorizerMapper authorizerMapper, + @JacksonInject ChatHandlerProvider chatHandlerProvider ) { super( @@ -101,8 +149,8 @@ public HadoopIndexTask( : hadoopDependencyCoordinates, context ); - - + this.authorizerMapper = authorizerMapper; + this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); this.spec = spec; // Some HadoopIngestionSpec stuff doesn't make sense in the context of the indexing service @@ -118,6 +166,7 @@ public HadoopIndexTask( this.classpathPrefix = classpathPrefix; this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "null ObjectMappper"); + this.ingestionState = IngestionState.NOT_STARTED; } @Override @@ -168,9 +217,46 @@ public String getClasspathPrefix() return classpathPrefix; } - @SuppressWarnings("unchecked") @Override public TaskStatus run(TaskToolbox toolbox) throws Exception + { + try { + if (chatHandlerProvider.isPresent()) { + log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); + chatHandlerProvider.get().register(getId(), this, false); + } else { + log.warn("No chat handler detected"); + } + + return runInternal(toolbox); + } + catch (Exception e) { + Throwable effectiveException; + if (e instanceof RuntimeException && e.getCause() instanceof InvocationTargetException) { + InvocationTargetException ite = (InvocationTargetException) e.getCause(); + effectiveException = ite.getCause(); + log.error(effectiveException, "Got invocation target exception in run(), cause: "); + } else { + effectiveException = e; + log.error(e, "Encountered exception in run():"); + } + + errorMsg = Throwables.getStackTraceAsString(effectiveException); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + return TaskStatus.failure( + getId(), + errorMsg + ); + } + finally { + if (chatHandlerProvider.isPresent()) { + chatHandlerProvider.get().unregister(getId()); + } + } + } + + @SuppressWarnings("unchecked") + private TaskStatus runInternal(TaskToolbox toolbox) throws Exception { final ClassLoader loader = buildClassLoader(toolbox); boolean determineIntervals = !spec.getDataSchema().getGranularitySpec().bucketIntervals().isPresent(); @@ -181,20 +267,56 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception new OverlordActionBasedUsedSegmentLister(toolbox) ); - final String config = invokeForeignLoader( - "io.druid.indexing.common.task.HadoopIndexTask$HadoopDetermineConfigInnerProcessing", - new String[]{ - toolbox.getObjectMapper().writeValueAsString(spec), - toolbox.getConfig().getHadoopWorkingPath(), - toolbox.getSegmentPusher().getPathForHadoop() - }, + Object determinePartitionsInnerProcessingRunner = getForeignClassloaderObject( + "io.druid.indexing.common.task.HadoopIndexTask$HadoopDetermineConfigInnerProcessingRunner", loader ); + determinePartitionsStatsGetter = new InnerProcessingStatsGetter(determinePartitionsInnerProcessingRunner); + + String[] determinePartitionsInput = new String[]{ + toolbox.getObjectMapper().writeValueAsString(spec), + toolbox.getConfig().getHadoopWorkingPath(), + toolbox.getSegmentPusher().getPathForHadoop() + }; + + HadoopIngestionSpec indexerSchema = null; + final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); + Class determinePartitionsRunnerClass = determinePartitionsInnerProcessingRunner.getClass(); + Method determinePartitionsInnerProcessingRunTask = determinePartitionsRunnerClass.getMethod( + "runTask", + determinePartitionsInput.getClass() + ); + try { + Thread.currentThread().setContextClassLoader(loader); + + ingestionState = IngestionState.DETERMINE_PARTITIONS; + + final String determineConfigStatusString = (String) determinePartitionsInnerProcessingRunTask.invoke( + determinePartitionsInnerProcessingRunner, + new Object[]{determinePartitionsInput} + ); + - final HadoopIngestionSpec indexerSchema = toolbox - .getObjectMapper() - .readValue(config, HadoopIngestionSpec.class); + determineConfigStatus = toolbox + .getObjectMapper() + .readValue(determineConfigStatusString, HadoopDetermineConfigInnerProcessingStatus.class); + indexerSchema = determineConfigStatus.getSchema(); + if (indexerSchema == null) { + errorMsg = determineConfigStatus.getErrorMsg(); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + return TaskStatus.failure( + getId(), + errorMsg + ); + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + finally { + Thread.currentThread().setContextClassLoader(oldLoader); + } // We should have a lock from before we started running only if interval was specified String version; @@ -236,40 +358,187 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception log.info("Setting version to: %s", version); - final String segments = invokeForeignLoader( - "io.druid.indexing.common.task.HadoopIndexTask$HadoopIndexGeneratorInnerProcessing", - new String[]{ - toolbox.getObjectMapper().writeValueAsString(indexerSchema), - version - }, + Object innerProcessingRunner = getForeignClassloaderObject( + "io.druid.indexing.common.task.HadoopIndexTask$HadoopIndexGeneratorInnerProcessingRunner", loader ); + buildSegmentsStatsGetter = new InnerProcessingStatsGetter(innerProcessingRunner); + + String[] buildSegmentsInput = new String[]{ + toolbox.getObjectMapper().writeValueAsString(indexerSchema), + version + }; + + Class buildSegmentsRunnerClass = innerProcessingRunner.getClass(); + Method innerProcessingRunTask = buildSegmentsRunnerClass.getMethod("runTask", buildSegmentsInput.getClass()); - if (segments != null) { - List publishedSegments = toolbox.getObjectMapper().readValue( - segments, - new TypeReference>() - { - } + try { + Thread.currentThread().setContextClassLoader(loader); + + ingestionState = IngestionState.BUILD_SEGMENTS; + final String jobStatusString = (String) innerProcessingRunTask.invoke( + innerProcessingRunner, + new Object[]{buildSegmentsInput} ); - toolbox.publishSegments(publishedSegments); - toolbox.getTaskReportFileWriter().write(null); - return TaskStatus.success(getId()); - } else { - toolbox.getTaskReportFileWriter().write(null); - return TaskStatus.failure(getId()); + buildSegmentsStatus = toolbox.getObjectMapper().readValue( + jobStatusString, + HadoopIndexGeneratorInnerProcessingStatus.class + ); + + if (buildSegmentsStatus.getDataSegments() != null) { + ingestionState = IngestionState.COMPLETED; + toolbox.publishSegments(buildSegmentsStatus.getDataSegments()); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + return TaskStatus.success( + getId(), + null + ); + } else { + errorMsg = buildSegmentsStatus.getErrorMsg(); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + return TaskStatus.failure( + getId(), + errorMsg + ); + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + finally { + Thread.currentThread().setContextClassLoader(oldLoader); + } + } + + @GET + @Path("/rowStats") + @Produces(MediaType.APPLICATION_JSON) + public Response getRowStats( + @Context final HttpServletRequest req, + @QueryParam("windows") List windows + ) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + Map returnMap = Maps.newHashMap(); + Map totalsMap = Maps.newHashMap(); + + if (determinePartitionsStatsGetter != null) { + totalsMap.put("determinePartitions", determinePartitionsStatsGetter.getTotalMetrics()); + } + + if (buildSegmentsStatsGetter != null) { + totalsMap.put("buildSegments", buildSegmentsStatsGetter.getTotalMetrics()); + } + + returnMap.put("totals", totalsMap); + return Response.ok(returnMap).build(); + } + + private Map getTaskCompletionReports() + { + return TaskReport.buildTaskReports( + new IngestionStatsAndErrorsTaskReport( + getId(), + new IngestionStatsAndErrorsTaskReportData( + ingestionState, + null, + getTaskCompletionRowStats(), + errorMsg + ) + ) + ); + } + + private Map getTaskCompletionRowStats() + { + Map metrics = Maps.newHashMap(); + if (determineConfigStatus != null) { + metrics.put( + "determinePartitions", + determineConfigStatus.getMetrics() + ); + } + if (buildSegmentsStatus != null) { + metrics.put( + "buildSegments", + buildSegmentsStatus.getMetrics() + ); + } + return metrics; + } + + public static class InnerProcessingStatsGetter implements TaskMetricsGetter + { + public static final List KEYS = Arrays.asList( + TaskMetricsUtils.ROWS_PROCESSED, + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, + TaskMetricsUtils.ROWS_THROWN_AWAY, + TaskMetricsUtils.ROWS_UNPARSEABLE + ); + + private final Method getStatsMethod; + private final Object innerProcessingRunner; + + public InnerProcessingStatsGetter( + Object innerProcessingRunner + ) + { + try { + Class aClazz = innerProcessingRunner.getClass(); + this.getStatsMethod = aClazz.getMethod("getStats"); + this.innerProcessingRunner = innerProcessingRunner; + } + catch (NoSuchMethodException nsme) { + throw new RuntimeException(nsme); + } + } + + @Override + public List getKeys() + { + return KEYS; + } + + @Override + public Map getTotalMetrics() + { + try { + Map statsMap = (Map) getStatsMethod.invoke(innerProcessingRunner); + if (statsMap == null) { + return null; + } + long curProcessed = (Long) statsMap.get(TaskMetricsUtils.ROWS_PROCESSED); + long curProcessedWithErrors = (Long) statsMap.get(TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS); + long curThrownAway = (Long) statsMap.get(TaskMetricsUtils.ROWS_THROWN_AWAY); + long curUnparseable = (Long) statsMap.get(TaskMetricsUtils.ROWS_UNPARSEABLE); + + return ImmutableMap.of( + TaskMetricsUtils.ROWS_PROCESSED, curProcessed, + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, curProcessedWithErrors, + TaskMetricsUtils.ROWS_THROWN_AWAY, curThrownAway, + TaskMetricsUtils.ROWS_UNPARSEABLE, curUnparseable + ); + } + catch (Exception e) { + log.error(e, "Got exception from getTotalMetrics(): "); + return null; + } } } + /** Called indirectly in {@link HadoopIndexTask#run(TaskToolbox)}. */ @SuppressWarnings("unused") - public static class HadoopIndexGeneratorInnerProcessing + public static class HadoopDetermineConfigInnerProcessingRunner { - public static String runTask(String[] args) throws Exception + private HadoopDruidDetermineConfigurationJob job; + + public String runTask(String[] args) throws Exception { final String schema = args[0]; - String version = args[1]; + final String workingPath = args[1]; + final String segmentOutputPath = args[2]; final HadoopIngestionSpec theSchema = HadoopDruidIndexerConfig.JSON_MAPPER .readValue( @@ -278,38 +547,43 @@ public static String runTask(String[] args) throws Exception ); final HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSpec( theSchema - .withTuningConfig(theSchema.getTuningConfig().withVersion(version)) + .withIOConfig(theSchema.getIOConfig().withSegmentOutputPath(segmentOutputPath)) + .withTuningConfig(theSchema.getTuningConfig().withWorkingPath(workingPath)) ); - // MetadataStorageUpdaterJobHandler is only needed when running standalone without indexing service - // In that case the whatever runs the Hadoop Index Task must ensure MetadataStorageUpdaterJobHandler - // can be injected based on the configuration given in config.getSchema().getIOConfig().getMetadataUpdateSpec() - final MetadataStorageUpdaterJobHandler maybeHandler; - if (config.isUpdaterJobSpecSet()) { - maybeHandler = injector.getInstance(MetadataStorageUpdaterJobHandler.class); + job = new HadoopDruidDetermineConfigurationJob(config); + + log.info("Starting a hadoop determine configuration job..."); + if (job.run()) { + return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString( + new HadoopDetermineConfigInnerProcessingStatus(config.getSchema(), job.getStats(), null) + ); } else { - maybeHandler = null; + return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString( + new HadoopDetermineConfigInnerProcessingStatus(null, job.getStats(), job.getErrorMessage()) + ); } - HadoopDruidIndexerJob job = new HadoopDruidIndexerJob(config, maybeHandler); + } - log.info("Starting a hadoop index generator job..."); - if (job.run()) { - return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(job.getPublishedSegments()); + public Map getStats() + { + if (job == null) { + return null; } - return null; + return job.getStats(); } } - /** Called indirectly in {@link HadoopIndexTask#run(TaskToolbox)}. */ @SuppressWarnings("unused") - public static class HadoopDetermineConfigInnerProcessing + public static class HadoopIndexGeneratorInnerProcessingRunner { - public static String runTask(String[] args) throws Exception + private HadoopDruidIndexerJob job; + + public String runTask(String[] args) throws Exception { final String schema = args[0]; - final String workingPath = args[1]; - final String segmentOutputPath = args[2]; + String version = args[1]; final HadoopIngestionSpec theSchema = HadoopDruidIndexerConfig.JSON_MAPPER .readValue( @@ -318,18 +592,133 @@ public static String runTask(String[] args) throws Exception ); final HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSpec( theSchema - .withIOConfig(theSchema.getIOConfig().withSegmentOutputPath(segmentOutputPath)) - .withTuningConfig(theSchema.getTuningConfig().withWorkingPath(workingPath)) + .withTuningConfig(theSchema.getTuningConfig().withVersion(version)) ); - Jobby job = new HadoopDruidDetermineConfigurationJob(config); + // MetadataStorageUpdaterJobHandler is only needed when running standalone without indexing service + // In that case the whatever runs the Hadoop Index Task must ensure MetadataStorageUpdaterJobHandler + // can be injected based on the configuration given in config.getSchema().getIOConfig().getMetadataUpdateSpec() + final MetadataStorageUpdaterJobHandler maybeHandler; + if (config.isUpdaterJobSpecSet()) { + maybeHandler = injector.getInstance(MetadataStorageUpdaterJobHandler.class); + } else { + maybeHandler = null; + } + job = new HadoopDruidIndexerJob(config, maybeHandler); + + log.info("Starting a hadoop index generator job..."); + try { + if (job.run()) { + return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString( + new HadoopIndexGeneratorInnerProcessingStatus( + job.getPublishedSegments(), + job.getStats(), + null + ) + ); + } else { + return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString( + new HadoopIndexGeneratorInnerProcessingStatus( + null, + job.getStats(), + job.getErrorMessage() + ) + ); + } + } + catch (Exception e) { + log.error(e, "Encountered exception in HadoopIndexGeneratorInnerProcessing."); + return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString( + new HadoopIndexGeneratorInnerProcessingStatus( + null, + job.getStats(), + e.getMessage() + ) + ); + } + } - log.info("Starting a hadoop determine configuration job..."); - if (job.run()) { - return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(config.getSchema()); + public Map getStats() + { + if (job == null) { + return null; } - return null; + return job.getStats(); + } + } + + public static class HadoopIndexGeneratorInnerProcessingStatus + { + private final List dataSegments; + private final Map metrics; + private final String errorMsg; + + @JsonCreator + public HadoopIndexGeneratorInnerProcessingStatus( + @JsonProperty("dataSegments") List dataSegments, + @JsonProperty("metrics") Map metrics, + @JsonProperty("errorMsg") String errorMsg + ) + { + this.dataSegments = dataSegments; + this.metrics = metrics; + this.errorMsg = errorMsg; + } + + @JsonProperty + public List getDataSegments() + { + return dataSegments; + } + + @JsonProperty + public Map getMetrics() + { + return metrics; + } + + @JsonProperty + public String getErrorMsg() + { + return errorMsg; + } + } + + public static class HadoopDetermineConfigInnerProcessingStatus + { + private final HadoopIngestionSpec schema; + private final Map metrics; + private final String errorMsg; + + @JsonCreator + public HadoopDetermineConfigInnerProcessingStatus( + @JsonProperty("schema") HadoopIngestionSpec schema, + @JsonProperty("metrics") Map metrics, + @JsonProperty("errorMsg") String errorMsg + ) + { + this.schema = schema; + this.metrics = metrics; + this.errorMsg = errorMsg; + } + + @JsonProperty + public HadoopIngestionSpec getSchema() + { + return schema; + } + + @JsonProperty + public Map getMetrics() + { + return metrics; + } + + @JsonProperty + public String getErrorMsg() + { + return errorMsg; } } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopTask.java index 8963559e3123..60be2b8639a1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopTask.java @@ -223,4 +223,32 @@ public static OutputType invokeForeignLoader( Thread.currentThread().setContextClassLoader(oldLoader); } } + + /** + * This method tries to isolate class loading during a Function call + * + * @param clazzName The Class which has an instance method called `runTask` + * @param loader The loader to use as the context class loader during invocation + * + * @return The result of the method invocation + */ + public static Object getForeignClassloaderObject( + final String clazzName, + final ClassLoader loader + ) + { + log.debug("Launching [%s] on class loader [%s]", clazzName, loader); + final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(loader); + final Class clazz = loader.loadClass(clazzName); + return clazz.newInstance(); + } + catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) { + throw Throwables.propagate(e); + } + finally { + Thread.currentThread().setContextClassLoader(oldLoader); + } + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index a9f78fb3ee4d..0b9f4c056994 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -19,6 +19,7 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; @@ -31,6 +32,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import com.google.common.util.concurrent.ListenableFuture; @@ -39,15 +41,21 @@ import io.druid.data.input.InputRow; import io.druid.data.input.Rows; import io.druid.hll.HyperLogLogCollector; +import io.druid.indexer.IngestionState; +import io.druid.indexer.TaskMetricsGetter; import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReport; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskReport; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Intervals; import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularity; @@ -64,29 +72,43 @@ import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.realtime.FireDepartment; import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.FireDepartmentMetricsTaskMetricsGetter; import io.druid.segment.realtime.RealtimeMetricsMonitor; import io.druid.segment.realtime.appenderator.Appenderator; import io.druid.segment.realtime.appenderator.AppenderatorConfig; -import io.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import io.druid.segment.realtime.appenderator.Appenderators; +import io.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import io.druid.segment.realtime.appenderator.BatchAppenderatorDriver; import io.druid.segment.realtime.appenderator.SegmentAllocator; import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; +import io.druid.segment.realtime.firehose.ChatHandler; +import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.server.security.Action; +import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.ShardSpec; +import io.druid.utils.CircularBuffer; import org.codehaus.plexus.util.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; import javax.annotation.Nullable; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; import java.io.File; import java.io.IOException; import java.util.HashMap; @@ -106,7 +128,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -public class IndexTask extends AbstractTask +public class IndexTask extends AbstractTask implements ChatHandler { private static final Logger log = new Logger(IndexTask.class); private static final HashFunction hashFunction = Hashing.murmur3_128(); @@ -131,12 +153,44 @@ private static String makeGroupId(boolean isAppendToExisting, String dataSource) @JsonIgnore private final IndexIngestionSpec ingestionSchema; + @JsonIgnore + private IngestionState ingestionState; + + @JsonIgnore + private final AuthorizerMapper authorizerMapper; + + @JsonIgnore + private final Optional chatHandlerProvider; + + @JsonIgnore + private FireDepartmentMetrics buildSegmentsFireDepartmentMetrics; + + @JsonIgnore + private TaskMetricsGetter buildSegmentsMetricsGetter; + + @JsonIgnore + private CircularBuffer buildSegmentsSavedParseExceptions; + + @JsonIgnore + private FireDepartmentMetrics determinePartitionsFireDepartmentMetrics; + + @JsonIgnore + private TaskMetricsGetter determinePartitionsMetricsGetter; + + @JsonIgnore + private CircularBuffer determinePartitionsSavedParseExceptions; + + @JsonIgnore + private String errorMsg; + @JsonCreator public IndexTask( @JsonProperty("id") final String id, @JsonProperty("resource") final TaskResource taskResource, @JsonProperty("spec") final IndexIngestionSpec ingestionSchema, - @JsonProperty("context") final Map context + @JsonProperty("context") final Map context, + @JacksonInject AuthorizerMapper authorizerMapper, + @JacksonInject ChatHandlerProvider chatHandlerProvider ) { this( @@ -145,7 +199,9 @@ public IndexTask( taskResource, ingestionSchema.dataSchema.getDataSource(), ingestionSchema, - context + context, + authorizerMapper, + chatHandlerProvider ); } @@ -155,7 +211,9 @@ public IndexTask( TaskResource resource, String dataSource, IndexIngestionSpec ingestionSchema, - Map context + Map context, + AuthorizerMapper authorizerMapper, + ChatHandlerProvider chatHandlerProvider ) { super( @@ -165,8 +223,19 @@ public IndexTask( dataSource, context ); - this.ingestionSchema = ingestionSchema; + this.authorizerMapper = authorizerMapper; + this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); + if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() > 0) { + determinePartitionsSavedParseExceptions = new CircularBuffer( + ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() + ); + + buildSegmentsSavedParseExceptions = new CircularBuffer( + ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() + ); + } + this.ingestionState = IngestionState.NOT_STARTED; } @Override @@ -209,6 +278,108 @@ static boolean isReady(TaskActionClient actionClient, SortedSet interv return true; } + @GET + @Path("/unparseableEvents") + @Produces(MediaType.APPLICATION_JSON) + public Response getUnparseableEvents( + @Context final HttpServletRequest req, + @QueryParam("full") String full + ) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + Map> events = Maps.newHashMap(); + + boolean needsDeterminePartitions = false; + boolean needsBuildSegments = false; + + if (full != null) { + needsDeterminePartitions = true; + needsBuildSegments = true; + } else { + switch (ingestionState) { + case DETERMINE_PARTITIONS: + needsDeterminePartitions = true; + break; + case BUILD_SEGMENTS: + case COMPLETED: + needsBuildSegments = true; + break; + default: + break; + } + } + + if (needsDeterminePartitions) { + events.put( + "determinePartitions", + IndexTaskUtils.getMessagesFromSavedParseExceptions(determinePartitionsSavedParseExceptions) + ); + } + + if (needsBuildSegments) { + events.put( + "buildSegments", + IndexTaskUtils.getMessagesFromSavedParseExceptions(buildSegmentsSavedParseExceptions) + ); + } + + return Response.ok(events).build(); + } + + @GET + @Path("/rowStats") + @Produces(MediaType.APPLICATION_JSON) + public Response getRowStats( + @Context final HttpServletRequest req, + @QueryParam("full") String full + ) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + Map returnMap = Maps.newHashMap(); + Map totalsMap = Maps.newHashMap(); + + boolean needsDeterminePartitions = false; + boolean needsBuildSegments = false; + + if (full != null) { + needsDeterminePartitions = true; + needsBuildSegments = true; + } else { + switch (ingestionState) { + case DETERMINE_PARTITIONS: + needsDeterminePartitions = true; + break; + case BUILD_SEGMENTS: + case COMPLETED: + needsBuildSegments = true; + break; + default: + break; + } + } + + if (needsDeterminePartitions) { + if (determinePartitionsMetricsGetter != null) { + totalsMap.put( + "determinePartitions", + determinePartitionsMetricsGetter.getTotalMetrics() + ); + } + } + + if (needsBuildSegments) { + if (buildSegmentsMetricsGetter != null) { + totalsMap.put( + "buildSegments", + buildSegmentsMetricsGetter.getTotalMetrics() + ); + } + } + + returnMap.put("totals", totalsMap); + return Response.ok(returnMap).build(); + } + @JsonProperty("spec") public IndexIngestionSpec getIngestionSchema() { @@ -218,56 +389,127 @@ public IndexIngestionSpec getIngestionSchema() @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { - final boolean determineIntervals = !ingestionSchema.getDataSchema() - .getGranularitySpec() - .bucketIntervals() - .isPresent(); - - final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); - - if (firehoseFactory instanceof IngestSegmentFirehoseFactory) { - // pass toolbox to Firehose - ((IngestSegmentFirehoseFactory) firehoseFactory).setTaskToolbox(toolbox); - } - - final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); - // Firehose temporary directory is automatically removed when this IndexTask completes. - FileUtils.forceMkdir(firehoseTempDir); - - final ShardSpecs shardSpecs = determineShardSpecs(toolbox, firehoseFactory, firehoseTempDir); - - final DataSchema dataSchema; - final Map versions; - if (determineIntervals) { - final SortedSet intervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); - intervals.addAll(shardSpecs.getIntervals()); - final Map locks = Tasks.tryAcquireExclusiveLocks(toolbox.getTaskActionClient(), intervals); - versions = locks.entrySet().stream() - .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getVersion())); - - dataSchema = ingestionSchema.getDataSchema().withGranularitySpec( - ingestionSchema.getDataSchema() - .getGranularitySpec() - .withIntervals( - JodaUtils.condenseIntervals( - shardSpecs.getIntervals() - ) - ) + try { + if (chatHandlerProvider.isPresent()) { + log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); + chatHandlerProvider.get().register(getId(), this, false); + } else { + log.warn("No chat handler detected"); + } + + final boolean determineIntervals = !ingestionSchema.getDataSchema() + .getGranularitySpec() + .bucketIntervals() + .isPresent(); + + final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); + + if (firehoseFactory instanceof IngestSegmentFirehoseFactory) { + // pass toolbox to Firehose + ((IngestSegmentFirehoseFactory) firehoseFactory).setTaskToolbox(toolbox); + } + + final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); + // Firehose temporary directory is automatically removed when this IndexTask completes. + FileUtils.forceMkdir(firehoseTempDir); + + ingestionState = IngestionState.DETERMINE_PARTITIONS; + final ShardSpecs shardSpecs = determineShardSpecs(toolbox, firehoseFactory, firehoseTempDir); + final DataSchema dataSchema; + final Map versions; + if (determineIntervals) { + final SortedSet intervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); + intervals.addAll(shardSpecs.getIntervals()); + final Map locks = Tasks.tryAcquireExclusiveLocks( + toolbox.getTaskActionClient(), + intervals + ); + versions = locks.entrySet().stream() + .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getVersion())); + + dataSchema = ingestionSchema.getDataSchema().withGranularitySpec( + ingestionSchema.getDataSchema() + .getGranularitySpec() + .withIntervals( + JodaUtils.condenseIntervals( + shardSpecs.getIntervals() + ) + ) + ); + } else { + versions = getTaskLocks(toolbox.getTaskActionClient()) + .stream() + .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); + dataSchema = ingestionSchema.getDataSchema(); + } + + ingestionState = IngestionState.BUILD_SEGMENTS; + return generateAndPublishSegments(toolbox, dataSchema, shardSpecs, versions, firehoseFactory, firehoseTempDir); + } + catch (Exception e) { + log.error(e, "Encountered exception in %s.", ingestionState); + errorMsg = Throwables.getStackTraceAsString(e); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + return TaskStatus.failure( + getId(), + errorMsg ); - } else { - versions = getTaskLocks(toolbox.getTaskActionClient()) - .stream() - .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); - dataSchema = ingestionSchema.getDataSchema(); } - if (generateAndPublishSegments(toolbox, dataSchema, shardSpecs, versions, firehoseFactory, firehoseTempDir)) { - toolbox.getTaskReportFileWriter().write(null); - return TaskStatus.success(getId()); - } else { - toolbox.getTaskReportFileWriter().write(null); - return TaskStatus.failure(getId()); + finally { + if (chatHandlerProvider.isPresent()) { + chatHandlerProvider.get().unregister(getId()); + } + } + } + + private Map getTaskCompletionReports() + { + return TaskReport.buildTaskReports( + new IngestionStatsAndErrorsTaskReport( + getId(), + new IngestionStatsAndErrorsTaskReportData( + ingestionState, + getTaskCompletionUnparseableEvents(), + getTaskCompletionRowStats(), + errorMsg + ) + ) + ); + } + + private Map getTaskCompletionUnparseableEvents() + { + Map unparseableEventsMap = Maps.newHashMap(); + List determinePartitionsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( + determinePartitionsSavedParseExceptions); + List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( + buildSegmentsSavedParseExceptions); + + if (determinePartitionsParseExceptionMessages != null || buildSegmentsParseExceptionMessages != null) { + unparseableEventsMap.put("determinePartitions", determinePartitionsParseExceptionMessages); + unparseableEventsMap.put("buildSegments", buildSegmentsParseExceptionMessages); + } + + return unparseableEventsMap; + } + + private Map getTaskCompletionRowStats() + { + Map metrics = Maps.newHashMap(); + if (determinePartitionsMetricsGetter != null) { + metrics.put( + "determinePartitions", + determinePartitionsMetricsGetter.getTotalMetrics() + ); + } + if (buildSegmentsMetricsGetter != null) { + metrics.put( + "buildSegments", + buildSegmentsMetricsGetter.getTotalMetrics() + ); } + return metrics; } private static String findVersion(Map versions, Interval interval) @@ -387,7 +629,7 @@ private static ShardSpecs createShardSpecWithoutInputScan( return new ShardSpecs(shardSpecs); } - private static ShardSpecs createShardSpecsFromInput( + private ShardSpecs createShardSpecsFromInput( ObjectMapper jsonMapper, IndexIngestionSpec ingestionSchema, FirehoseFactory firehoseFactory, @@ -449,7 +691,7 @@ private static ShardSpecs createShardSpecsFromInput( return new ShardSpecs(intervalToShardSpecs); } - private static Map> collectIntervalsAndShardSpecs( + private Map> collectIntervalsAndShardSpecs( ObjectMapper jsonMapper, IndexIngestionSpec ingestionSchema, FirehoseFactory firehoseFactory, @@ -459,6 +701,11 @@ private static Map> collectIntervalsAnd boolean determineNumPartitions ) throws IOException { + determinePartitionsFireDepartmentMetrics = new FireDepartmentMetrics(); + determinePartitionsMetricsGetter = new FireDepartmentMetricsTaskMetricsGetter( + determinePartitionsFireDepartmentMetrics + ); + final Map> hllCollectors = new TreeMap<>( Comparators.intervalsByStartThenEnd() ); @@ -469,12 +716,14 @@ private static Map> collectIntervalsAnd try ( final Firehose firehose = firehoseFactory.connect(ingestionSchema.getDataSchema().getParser(), firehoseTempDir) ) { + while (firehose.hasMore()) { try { final InputRow inputRow = firehose.nextRow(); // The null inputRow means the caller must skip this row. if (inputRow == null) { + determinePartitionsFireDepartmentMetrics.incrementThrownAway(); continue; } @@ -482,9 +731,17 @@ private static Map> collectIntervalsAnd if (determineIntervals) { interval = granularitySpec.getSegmentGranularity().bucket(inputRow.getTimestamp()); } else { + if (!Intervals.ETERNITY.contains(inputRow.getTimestamp())) { + final String errorMsg = StringUtils.format( + "Encountered row with timestamp that cannot be represented as a long: [%s]", + inputRow + ); + throw new ParseException(errorMsg); + } + final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); if (!optInterval.isPresent()) { - thrownAway++; + determinePartitionsFireDepartmentMetrics.incrementThrownAway(); continue; } interval = optInterval.get(); @@ -508,12 +765,21 @@ private static Map> collectIntervalsAnd hllCollectors.put(interval, Optional.absent()); } } + determinePartitionsFireDepartmentMetrics.incrementProcessed(); } catch (ParseException e) { - if (ingestionSchema.getTuningConfig().isReportParseExceptions()) { - throw e; - } else { - unparseable++; + if (ingestionSchema.getTuningConfig().isLogParseExceptions()) { + log.error(e, "Encountered parse exception: "); + } + + if (determinePartitionsSavedParseExceptions != null) { + determinePartitionsSavedParseExceptions.add(e); + } + + determinePartitionsFireDepartmentMetrics.incrementUnparseable(); + if (determinePartitionsFireDepartmentMetrics.unparseable() > ingestionSchema.getTuningConfig() + .getMaxParseExceptions()) { + throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); } } } @@ -561,7 +827,7 @@ private static BiFunction getShardSpecCreateFunctio * * @return true if generated segments are successfully published, otherwise false */ - private boolean generateAndPublishSegments( + private TaskStatus generateAndPublishSegments( final TaskToolbox toolbox, final DataSchema dataSchema, final ShardSpecs shardSpecs, @@ -574,7 +840,8 @@ private boolean generateAndPublishSegments( final FireDepartment fireDepartmentForMetrics = new FireDepartment( dataSchema, new RealtimeIOConfig(null, null, null), null ); - final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + buildSegmentsFireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + buildSegmentsMetricsGetter = new FireDepartmentMetricsTaskMetricsGetter(buildSegmentsFireDepartmentMetrics); if (toolbox.getMonitorScheduler() != null) { toolbox.getMonitorScheduler().addMonitor( @@ -652,7 +919,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null }; try ( - final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema, tuningConfig); + final Appenderator appenderator = newAppenderator(buildSegmentsFireDepartmentMetrics, toolbox, dataSchema, tuningConfig); final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator); final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) ) { @@ -663,13 +930,21 @@ dataSchema, new RealtimeIOConfig(null, null, null), null final InputRow inputRow = firehose.nextRow(); if (inputRow == null) { - fireDepartmentMetrics.incrementThrownAway(); + buildSegmentsFireDepartmentMetrics.incrementThrownAway(); continue; } + if (!Intervals.ETERNITY.contains(inputRow.getTimestamp())) { + final String errorMsg = StringUtils.format( + "Encountered row with timestamp that cannot be represented as a long: [%s]", + inputRow + ); + throw new ParseException(errorMsg); + } + final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); if (!optInterval.isPresent()) { - fireDepartmentMetrics.incrementThrownAway(); + buildSegmentsFireDepartmentMetrics.incrementThrownAway(); continue; } @@ -702,15 +977,15 @@ dataSchema, new RealtimeIOConfig(null, null, null), null throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp()); } - fireDepartmentMetrics.incrementProcessed(); - } - catch (ParseException e) { - if (tuningConfig.isReportParseExceptions()) { - throw e; + if (addResult.getParseException() != null) { + handleParseException(addResult.getParseException()); } else { - fireDepartmentMetrics.incrementUnparseable(); + buildSegmentsFireDepartmentMetrics.incrementProcessed(); } } + catch (ParseException e) { + handleParseException(e); + } } final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout); @@ -721,15 +996,21 @@ dataSchema, new RealtimeIOConfig(null, null, null), null pushTimeout ); + ingestionState = IngestionState.COMPLETED; if (published == null) { log.error("Failed to publish segments, aborting!"); - return false; + errorMsg = "Failed to publish segments."; + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + return TaskStatus.failure( + getId(), + errorMsg + ); } else { log.info( "Processed[%,d] events, unparseable[%,d], thrownAway[%,d].", - fireDepartmentMetrics.processed(), - fireDepartmentMetrics.unparseable(), - fireDepartmentMetrics.thrownAway() + buildSegmentsFireDepartmentMetrics.processed(), + buildSegmentsFireDepartmentMetrics.unparseable(), + buildSegmentsFireDepartmentMetrics.thrownAway() ); log.info( "Published segments[%s]", Joiner.on(", ").join( @@ -739,7 +1020,9 @@ dataSchema, new RealtimeIOConfig(null, null, null), null ) ) ); - return true; + + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + return TaskStatus.success(getId()); } } catch (TimeoutException | ExecutionException e) { @@ -747,6 +1030,29 @@ dataSchema, new RealtimeIOConfig(null, null, null), null } } + private void handleParseException(ParseException e) + { + if (e.isFromPartiallyValidRow()) { + buildSegmentsFireDepartmentMetrics.incrementProcessedWithErrors(); + } else { + buildSegmentsFireDepartmentMetrics.incrementUnparseable(); + } + + if (ingestionSchema.tuningConfig.isLogParseExceptions()) { + log.error(e, "Encountered parse exception:"); + } + + if (buildSegmentsSavedParseExceptions != null) { + buildSegmentsSavedParseExceptions.add(e); + } + + if (buildSegmentsFireDepartmentMetrics.unparseable() + + buildSegmentsFireDepartmentMetrics.processedWithErrors() > ingestionSchema.tuningConfig.getMaxParseExceptions()) { + log.error("Max parse exceptions exceeded, terminating task..."); + throw new RuntimeException("Max parse exceptions exceeded, terminating task...", e); + } + } + private static boolean exceedMaxRowsInSegment(int numRowsInSegment, IndexTuningConfig indexTuningConfig) { // maxRowsInSegment should be null if numShards is set in indexTuningConfig @@ -949,6 +1255,10 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi private final boolean forceGuaranteedRollup; private final boolean reportParseExceptions; private final long pushTimeout; + private final boolean logParseExceptions; + private final int maxParseExceptions; + private final int maxSavedParseExceptions; + @Nullable private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; @@ -966,10 +1276,13 @@ public IndexTuningConfig( @JsonProperty("buildV9Directly") @Nullable Boolean buildV9Directly, @JsonProperty("forceExtendableShardSpecs") @Nullable Boolean forceExtendableShardSpecs, @JsonProperty("forceGuaranteedRollup") @Nullable Boolean forceGuaranteedRollup, - @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, + @Deprecated @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, @JsonProperty("publishTimeout") @Nullable Long publishTimeout, // deprecated @JsonProperty("pushTimeout") @Nullable Long pushTimeout, - @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions ) { this( @@ -985,13 +1298,16 @@ public IndexTuningConfig( reportParseExceptions, pushTimeout != null ? pushTimeout : publishTimeout, null, - segmentWriteOutMediumFactory + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); } private IndexTuningConfig() { - this(null, null, null, null, null, null, null, null, null, null, null, null, null); + this(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); } private IndexTuningConfig( @@ -1007,7 +1323,10 @@ private IndexTuningConfig( @Nullable Boolean reportParseExceptions, @Nullable Long pushTimeout, @Nullable File basePersistDirectory, - @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @Nullable Boolean logParseExceptions, + @Nullable Integer maxParseExceptions, + @Nullable Integer maxSavedParseExceptions ) { Preconditions.checkArgument( @@ -1038,6 +1357,17 @@ private IndexTuningConfig( ); this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + + if (this.reportParseExceptions) { + this.maxParseExceptions = 0; + this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions); + } else { + this.maxParseExceptions = maxParseExceptions == null ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS : maxParseExceptions; + this.maxSavedParseExceptions = maxSavedParseExceptions == null + ? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS + : maxSavedParseExceptions; + } + this.logParseExceptions = logParseExceptions == null ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS : logParseExceptions; } private static Integer initializeTargetPartitionSize(Integer numShards, Integer targetPartitionSize) @@ -1075,7 +1405,10 @@ public IndexTuningConfig withBasePersistDirectory(File dir) reportParseExceptions, pushTimeout, dir, - segmentWriteOutMediumFactory + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); } @@ -1166,6 +1499,24 @@ public long getPushTimeout() return pushTimeout; } + @JsonProperty + public boolean isLogParseExceptions() + { + return logParseExceptions; + } + + @JsonProperty + public int getMaxParseExceptions() + { + return maxParseExceptions; + } + + @JsonProperty + public int getMaxSavedParseExceptions() + { + return maxSavedParseExceptions; + } + @Override public Period getIntermediatePersistPeriod() { @@ -1201,7 +1552,10 @@ public boolean equals(Object o) Objects.equals(numShards, that.numShards) && Objects.equals(indexSpec, that.indexSpec) && Objects.equals(basePersistDirectory, that.basePersistDirectory) && - Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory); + Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory) && + logParseExceptions == that.logParseExceptions && + maxParseExceptions == that.maxParseExceptions && + maxSavedParseExceptions == that.maxSavedParseExceptions; } @Override @@ -1219,7 +1573,10 @@ public int hashCode() forceGuaranteedRollup, reportParseExceptions, pushTimeout, - segmentWriteOutMediumFactory + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskUtils.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskUtils.java new file mode 100644 index 000000000000..aee89b58a5aa --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskUtils.java @@ -0,0 +1,78 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import io.druid.server.security.Access; +import io.druid.server.security.Action; +import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.ForbiddenException; +import io.druid.server.security.Resource; +import io.druid.server.security.ResourceAction; +import io.druid.server.security.ResourceType; +import io.druid.utils.CircularBuffer; + +import javax.annotation.Nullable; +import javax.servlet.http.HttpServletRequest; +import java.util.ArrayList; +import java.util.List; + +public class IndexTaskUtils +{ + @Nullable + public static List getMessagesFromSavedParseExceptions(CircularBuffer savedParseExceptions) + { + if (savedParseExceptions == null) { + return null; + } + + List events = new ArrayList<>(); + for (int i = 0; i < savedParseExceptions.size(); i++) { + events.add(savedParseExceptions.getLatest(i).getMessage()); + } + + return events; + } + + /** + * Authorizes action to be performed on a task's datasource + * + * @return authorization result + */ + public static Access datasourceAuthorizationCheck( + final HttpServletRequest req, + Action action, + String datasource, + AuthorizerMapper authorizerMapper + ) + { + ResourceAction resourceAction = new ResourceAction( + new Resource(datasource, ResourceType.DATASOURCE), + action + ); + + Access access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); + if (!access.isAllowed()) { + throw new ForbiddenException(access.toString()); + } + + return access; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java index 06e6342356be..daad56a8797f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java @@ -465,11 +465,11 @@ public TaskStatus call() log.warn(e, "Interrupted while running task[%s]", task); } - status = TaskStatus.failure(task.getId()); + status = TaskStatus.failure(task.getId(), e.toString()); } catch (Exception e) { log.error(e, "Exception while running task[%s]", task); - status = TaskStatus.failure(task.getId()); + status = TaskStatus.failure(task.getId(), e.toString()); } catch (Throwable t) { log.error(t, "Uncaught Throwable while running task[%s]", task); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 1fa59cebd1a8..a0dc0fe22a19 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -621,7 +621,9 @@ public Response getCompleteTasks( status.getStatusCode(), status.getDuration(), TaskLocation.unknown(), - pair.rhs); + pair.rhs, + status.getErrorMsg() + ); })); return Response.ok(completeTasks).build(); @@ -804,7 +806,8 @@ public TaskStatusPlus apply(TaskRunnerWorkItem workItem) null, null, workItem.getLocation(), - workItem.getDataSource() + workItem.getDataSource(), + null ); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 4bb79ba6a316..7b6ce6437559 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -19,6 +19,7 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -34,15 +35,23 @@ import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.FloatDimensionSchema; import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.LongDimensionSchema; import io.druid.data.input.impl.MapInputRowParser; +import io.druid.data.input.impl.StringDimensionSchema; import io.druid.data.input.impl.TimeAndDimsParseSpec; import io.druid.data.input.impl.TimestampSpec; import io.druid.discovery.DataNodeService; import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.LookupNodeService; +import io.druid.indexer.IngestionState; +import io.druid.indexer.TaskMetricsUtils; import io.druid.indexer.TaskState; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import io.druid.indexing.common.SegmentLoaderFactory; +import io.druid.indexing.common.TaskReport; +import io.druid.indexing.common.TaskReportFileWriter; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TaskToolboxFactory; @@ -117,12 +126,12 @@ import io.druid.server.DruidNode; import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.server.coordination.ServerType; +import io.druid.server.security.AuthTestUtils; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.NumberedShardSpec; import org.apache.commons.io.FileUtils; import org.easymock.EasyMock; -import org.hamcrest.CoreMatchers; import org.joda.time.DateTime; import org.joda.time.Period; import org.junit.After; @@ -130,8 +139,6 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.junit.internal.matchers.ThrowableCauseMatcher; -import org.junit.internal.matchers.ThrowableMessageMatcher; import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; @@ -149,7 +156,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; @@ -161,6 +167,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest "host", new NoopEmitter() ); + private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); private static final String FAIL_DIM = "__fail__"; @@ -261,6 +268,7 @@ public Firehose connect(InputRowParser parser, File temporaryDirectory) throws P private TaskLockbox taskLockbox; private TaskToolboxFactory taskToolboxFactory; private File baseDir; + private File reportsFile; @Before public void setUp() throws IOException @@ -277,6 +285,7 @@ public void setUp() throws IOException derbyConnector.createPendingSegmentsTable(); baseDir = tempFolder.newFolder(); + reportsFile = File.createTempFile("KafkaIndexTaskTestReports-" + System.currentTimeMillis(), "json"); makeToolboxFactory(baseDir); } @@ -284,6 +293,7 @@ public void setUp() throws IOException public void tearDown() { taskExec.shutdownNow(); + reportsFile.delete(); } @Test(timeout = 60_000L) @@ -294,11 +304,11 @@ public void testDefaultResource() } - @Test(timeout = 60_000L, expected = ExecutionException.class) + @Test(timeout = 60_000L) public void testHandoffTimeout() throws Exception { expectPublishedSegments(1); - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, true, 100L); + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, true, 100L, true, 0, 1); final ListenableFuture statusFuture = runTask(task); // Wait for firehose to show up, it starts off null. @@ -318,7 +328,8 @@ public void testHandoffTimeout() throws Exception firehose.close(); // handoff would timeout, resulting in exception - statusFuture.get(); + TaskStatus status = statusFuture.get(); + Assert.assertTrue(status.getErrorMsg().contains("java.util.concurrent.TimeoutException: Timeout waiting for task.")); } @Test(timeout = 60_000L) @@ -520,7 +531,7 @@ public void testTransformSpec() throws Exception new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) ) ); - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, transformSpec, true, 0); + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, transformSpec, true, 0, true, 0, 1); final ListenableFuture statusFuture = runTask(task); // Wait for firehose to show up, it starts off null. @@ -595,10 +606,10 @@ public void testReportParseExceptionsOnBadMetric() throws Exception firehose.addRows( ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "foo"), + ImmutableMap.of("t", 2000000L, "dim1", "foo", "met1", "1"), + ImmutableMap.of("t", 3000000L, "dim1", "foo", "met1", "foo"), ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", "foo"), - ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) + ImmutableMap.of("t", 4000000L, "dim2", "bar", "met1", 2.0) ) ); @@ -606,26 +617,19 @@ public void testReportParseExceptionsOnBadMetric() throws Exception firehose.close(); // Wait for the task to finish. - expectedException.expect(ExecutionException.class); - expectedException.expectCause(CoreMatchers.instanceOf(ParseException.class)); - expectedException.expectCause( - ThrowableMessageMatcher.hasMessage( - CoreMatchers.containsString("Encountered parse error for aggregator[met1]") - ) - ); - expectedException.expect( - ThrowableCauseMatcher.hasCause( - ThrowableCauseMatcher.hasCause( - CoreMatchers.allOf( - CoreMatchers.instanceOf(ParseException.class), - ThrowableMessageMatcher.hasMessage( - CoreMatchers.containsString("Unable to parse value[foo] for field[met1]") - ) - ) - ) + TaskStatus status = statusFuture.get(); + Assert.assertTrue(status.getErrorMsg().contains("java.lang.RuntimeException: Max parse exceptions exceeded, terminating task...")); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedUnparseables = ImmutableMap.of( + "buildSegments", + Arrays.asList( + "Found unparseable columns in row: [MapBasedInputRow{timestamp=1970-01-01T00:50:00.000Z, event={t=3000000, dim1=foo, met1=foo}, dimensions=[dim1, dim2, dim1t, dimLong, dimFloat]}], exceptions: [Unable to parse value[foo] for field[met1],]" ) ); - statusFuture.get(); + + Assert.assertEquals(expectedUnparseables, reportData.getUnparseableEvents()); } @Test(timeout = 60_000L) @@ -633,7 +637,7 @@ public void testNoReportParseExceptions() throws Exception { expectPublishedSegments(1); - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, false); + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, false, 0, true, null, 1); final ListenableFuture statusFuture = runTask(task); // Wait for firehose to show up, it starts off null. @@ -671,7 +675,8 @@ public void testNoReportParseExceptions() throws Exception DataSegment publishedSegment = Iterables.getOnlyElement(publishedSegments); // Check metrics. - Assert.assertEquals(3, task.getMetrics().processed()); + Assert.assertEquals(2, task.getMetrics().processed()); + Assert.assertEquals(1, task.getMetrics().processedWithErrors()); Assert.assertEquals(0, task.getMetrics().thrownAway()); Assert.assertEquals(2, task.getMetrics().unparseable()); @@ -696,9 +701,195 @@ public void testNoReportParseExceptions() throws Exception } handOffCallbacks.clear(); + Map expectedMetrics = ImmutableMap.of( + "buildSegments", + ImmutableMap.of( + TaskMetricsUtils.ROWS_PROCESSED, 2, + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, 1, + TaskMetricsUtils.ROWS_UNPARSEABLE, 2, + TaskMetricsUtils.ROWS_THROWN_AWAY, 0 + ) + ); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + } + + @Test(timeout = 60_000L) + public void testMultipleParseExceptionsSuccess() throws Exception + { + expectPublishedSegments(1); + + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, false, 0, true, 10, 10); + final ListenableFuture statusFuture = runTask(task); + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task.getFirehose(); + + firehose.addRows( + Arrays.asList( + // Good row- will be processed. + ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "met1", "1"), + + // Null row- will be thrown away. + null, + + // Bad metric- will count as processed, but that particular metric won't update. + ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "met1", "foo"), + + // Bad long dim- will count as processed, but bad dims will get default values + ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "dimLong", "notnumber", "dimFloat", "notnumber", "met1", "foo"), + + // Bad row- will be unparseable. + ImmutableMap.of("dim1", "foo", "met1", 2.0, FAIL_DIM, "x"), + + // Good row- will be processed. + ImmutableMap.of("t", 1521251960729L, "dim2", "bar", "met1", 2.0) + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + // Wait for publish. + Collection publishedSegments = awaitSegments(); + + DataSegment publishedSegment = Iterables.getOnlyElement(publishedSegments); + + // Check metrics. + Assert.assertEquals(2, task.getMetrics().processed()); + Assert.assertEquals(2, task.getMetrics().processedWithErrors()); + Assert.assertEquals(0, task.getMetrics().thrownAway()); + Assert.assertEquals(2, task.getMetrics().unparseable()); + + // Do some queries. + Assert.assertEquals(4, sumMetric(task, null, "rows")); + Assert.assertEquals(3, sumMetric(task, null, "met1")); + + awaitHandoffs(); + + // Simulate handoff. + for (Map.Entry> entry : handOffCallbacks.entrySet()) { + final Pair executorRunnablePair = entry.getValue(); + Assert.assertEquals( + new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ), + entry.getKey() + ); + executorRunnablePair.lhs.execute(executorRunnablePair.rhs); + } + handOffCallbacks.clear(); + + Map expectedMetrics = ImmutableMap.of( + "buildSegments", + ImmutableMap.of( + TaskMetricsUtils.ROWS_PROCESSED, 2, + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, 2, + TaskMetricsUtils.ROWS_UNPARSEABLE, 2, + TaskMetricsUtils.ROWS_THROWN_AWAY, 0 + ) + ); + // Wait for the task to finish. final TaskStatus taskStatus = statusFuture.get(); Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + Map expectedUnparseables = ImmutableMap.of( + "buildSegments", + Arrays.asList( + "Unparseable timestamp found! Event: {dim1=foo, met1=2.0, __fail__=x}", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2018-03-17T01:59:20.729Z, event={t=1521251960729, dim1=foo, dimLong=notnumber, dimFloat=notnumber, met1=foo}, dimensions=[dim1, dim2, dim1t, dimLong, dimFloat]}], exceptions: [could not convert value [notnumber] to long,could not convert value [notnumber] to float,Unable to parse value[foo] for field[met1],]", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2018-03-17T01:59:20.729Z, event={t=1521251960729, dim1=foo, met1=foo}, dimensions=[dim1, dim2, dim1t, dimLong, dimFloat]}], exceptions: [Unable to parse value[foo] for field[met1],]", + "Unparseable timestamp found! Event: null" + ) + ); + Assert.assertEquals(expectedUnparseables, reportData.getUnparseableEvents()); + Assert.assertEquals(IngestionState.COMPLETED, reportData.getIngestionState()); + } + + @Test(timeout = 60_000L) + public void testMultipleParseExceptionsFailure() throws Exception + { + expectPublishedSegments(1); + + final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, false, 0, true, 3, 10); + final ListenableFuture statusFuture = runTask(task); + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task.getFirehose(); + + firehose.addRows( + Arrays.asList( + // Good row- will be processed. + ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "met1", "1"), + + // Null row- will be thrown away. + null, + + // Bad metric- will count as processed, but that particular metric won't update. + ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "met1", "foo"), + + // Bad long dim- will count as processed, but bad dims will get default values + ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "dimLong", "notnumber", "dimFloat", "notnumber", "met1", "foo"), + + // Bad row- will be unparseable. + ImmutableMap.of("dim1", "foo", "met1", 2.0, FAIL_DIM, "x"), + + // Good row- will be processed. + ImmutableMap.of("t", 1521251960729L, "dim2", "bar", "met1", 2.0) + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskState.FAILED, taskStatus.getStatusCode()); + Assert.assertTrue(taskStatus.getErrorMsg().contains("Max parse exceptions exceeded, terminating task...")); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedMetrics = ImmutableMap.of( + "buildSegments", + ImmutableMap.of( + TaskMetricsUtils.ROWS_PROCESSED, 1, + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, 2, + TaskMetricsUtils.ROWS_UNPARSEABLE, 2, + TaskMetricsUtils.ROWS_THROWN_AWAY, 0 + ) + ); + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + Map expectedUnparseables = ImmutableMap.of( + "buildSegments", + Arrays.asList( + "Unparseable timestamp found! Event: {dim1=foo, met1=2.0, __fail__=x}", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2018-03-17T01:59:20.729Z, event={t=1521251960729, dim1=foo, dimLong=notnumber, dimFloat=notnumber, met1=foo}, dimensions=[dim1, dim2, dim1t, dimLong, dimFloat]}], exceptions: [could not convert value [notnumber] to long,could not convert value [notnumber] to float,Unable to parse value[foo] for field[met1],]", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2018-03-17T01:59:20.729Z, event={t=1521251960729, dim1=foo, met1=foo}, dimensions=[dim1, dim2, dim1t, dimLong, dimFloat]}], exceptions: [Unable to parse value[foo] for field[met1],]", + "Unparseable timestamp found! Event: null" + ) + ); + Assert.assertEquals(expectedUnparseables, reportData.getUnparseableEvents()); + Assert.assertEquals(IngestionState.BUILD_SEGMENTS, reportData.getIngestionState()); } @Test(timeout = 60_000L) @@ -929,14 +1120,21 @@ public void testRestoreCorruptData() throws Exception final ListenableFuture statusFuture = runTask(task2); // Wait for the task to finish. - boolean caught = false; - try { - statusFuture.get(); - } - catch (Exception expected) { - caught = true; - } - Assert.assertTrue("expected exception", caught); + TaskStatus status = statusFuture.get(); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedMetrics = ImmutableMap.of( + "buildSegments", + ImmutableMap.of( + TaskMetricsUtils.ROWS_PROCESSED, 0, + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, 0, + TaskMetricsUtils.ROWS_UNPARSEABLE, 0, + TaskMetricsUtils.ROWS_THROWN_AWAY, 0 + ) + ); + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + Assert.assertTrue(status.getErrorMsg().contains("java.lang.IllegalArgumentException\n\tat java.nio.Buffer.position")); } } @@ -989,19 +1187,22 @@ public TaskStatus call() throws Exception private AppenderatorDriverRealtimeIndexTask makeRealtimeTask(final String taskId) { - return makeRealtimeTask(taskId, TransformSpec.NONE, true, 0); + return makeRealtimeTask(taskId, TransformSpec.NONE, true, 0, true, 0, 1); } private AppenderatorDriverRealtimeIndexTask makeRealtimeTask(final String taskId, boolean reportParseExceptions) { - return makeRealtimeTask(taskId, TransformSpec.NONE, reportParseExceptions, 0); + return makeRealtimeTask(taskId, TransformSpec.NONE, reportParseExceptions, 0, true, null, 1); } private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( final String taskId, final TransformSpec transformSpec, final boolean reportParseExceptions, - final long handoffTimeout + final long handoffTimeout, + final Boolean logParseExceptions, + final Integer maxParseExceptions, + final Integer maxSavedParseExceptions ) { ObjectMapper objectMapper = new DefaultObjectMapper(); @@ -1012,7 +1213,13 @@ private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( new TimeAndDimsParseSpec( new TimestampSpec("t", "auto", null), new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim1t")), + ImmutableList.of( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim2"), + new StringDimensionSchema("dim1t"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ), null, null ) @@ -1042,13 +1249,18 @@ private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( reportParseExceptions, handoffTimeout, null, - null + null, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); return new AppenderatorDriverRealtimeIndexTask( taskId, null, new RealtimeAppenderatorIngestionSpec(dataSchema, realtimeIOConfig, tuningConfig), - null + null, + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER ) { @Override @@ -1245,7 +1457,7 @@ public List getLocations() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0), - new NoopTestTaskFileWriter() + new TaskReportFileWriter(reportsFile) ); } @@ -1267,4 +1479,17 @@ public long sumMetric(final Task task, final DimFilter filter, final String metr task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList(); return results.isEmpty() ? 0 : results.get(0).getValue().getLongMetric(metric); } + + private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException + { + Map taskReports = objectMapper.readValue( + reportsFile, + new TypeReference>() + { + } + ); + return IngestionStatsAndErrorsTaskReportData.getPayloadFromTaskReports( + taskReports + ); + } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java index d97a0d363468..a64353255e0e 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java @@ -27,6 +27,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.inject.Binder; +import com.google.inject.Module; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.impl.DimensionSchema; import io.druid.data.input.impl.DimensionsSpec; @@ -79,6 +81,8 @@ import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.transform.TransformingInputRowParser; import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.server.security.AuthTestUtils; +import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NumberedShardSpec; import org.hamcrest.CoreMatchers; @@ -200,7 +204,21 @@ private static ObjectMapper setupInjectablesInObjectMapper(ObjectMapper objectMa guiceIntrospector, objectMapper.getDeserializationConfig().getAnnotationIntrospector() ) ); - objectMapper.setInjectableValues(new GuiceInjectableValues(GuiceInjectors.makeStartupInjector())); + GuiceInjectableValues injectableValues = new GuiceInjectableValues( + GuiceInjectors.makeStartupInjectorWithModules( + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(AuthorizerMapper.class).toInstance(AuthTestUtils.TEST_AUTHORIZER_MAPPER); + } + } + ) + ) + ); + objectMapper.setInjectableValues(injectableValues); objectMapper.registerModule( new SimpleModule().registerSubtypes(new NamedType(NumberedShardSpec.class, "NumberedShardSpec")) ); @@ -245,6 +263,9 @@ private static IndexTuningConfig createTuningConfig() false, null, 100L, + null, + null, + null, null ); } @@ -264,7 +285,8 @@ public void testSerdeWithInterval() throws IOException null, createTuningConfig(), ImmutableMap.of("testKey", "testContext"), - objectMapper + objectMapper, + AuthTestUtils.TEST_AUTHORIZER_MAPPER ); final byte[] bytes = objectMapper.writeValueAsBytes(task); final CompactionTask fromJson = objectMapper.readValue(bytes, CompactionTask.class); @@ -290,7 +312,8 @@ public void testSerdeWithSegments() throws IOException null, createTuningConfig(), ImmutableMap.of("testKey", "testContext"), - objectMapper + objectMapper, + AuthTestUtils.TEST_AUTHORIZER_MAPPER ); final byte[] bytes = objectMapper.writeValueAsBytes(task); final CompactionTask fromJson = objectMapper.readValue(bytes, CompactionTask.class); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 5c4be2942b90..28cbf664da8a 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -19,19 +19,31 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.io.Files; import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.FloatDimensionSchema; +import io.druid.data.input.impl.JSONParseSpec; +import io.druid.data.input.impl.LongDimensionSchema; import io.druid.data.input.impl.ParseSpec; import io.druid.data.input.impl.SpatialDimensionSchema; +import io.druid.data.input.impl.StringDimensionSchema; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; +import io.druid.indexer.TaskMetricsUtils; +import io.druid.indexer.TaskState; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskLockType; +import io.druid.indexing.common.TaskReport; +import io.druid.indexing.common.TaskReportFileWriter; +import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TestUtils; import io.druid.indexing.common.actions.LockAcquireAction; @@ -46,9 +58,9 @@ import io.druid.indexing.overlord.SegmentPublishResult; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; -import io.druid.java.util.common.parsers.ParseException; import io.druid.math.expr.ExprMacroTable; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -65,13 +77,16 @@ import io.druid.segment.realtime.firehose.LocalFirehoseFactory; import io.druid.segment.transform.ExpressionTransform; import io.druid.segment.transform.TransformSpec; +import io.druid.server.security.AuthTestUtils; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -82,6 +97,7 @@ import java.io.IOException; import java.net.URI; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; @@ -119,6 +135,7 @@ public class IndexTaskTest private IndexMergerV9 indexMergerV9; private IndexIO indexIO; private volatile int segmentAllocatePartitionCounter; + private File reportsFile; public IndexTaskTest() { @@ -128,6 +145,18 @@ public IndexTaskTest() indexIO = testUtils.getTestIndexIO(); } + @Before + public void setup() throws IOException + { + reportsFile = File.createTempFile("IndexTaskTestReports-" + System.currentTimeMillis(), "json"); + } + + @After + public void teardown() throws IOException + { + reportsFile.delete(); + } + @Test public void testDeterminePartitions() throws Exception { @@ -151,10 +180,12 @@ public void testDeterminePartitions() throws Exception createTuningConfig(2, null, false, true), false ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(2, segments.size()); @@ -194,12 +225,14 @@ public void testForceExtendableShardSpecs() throws Exception createTuningConfig(2, null, true, false), false ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); Assert.assertEquals(indexTask.getId(), indexTask.getGroupId()); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(2, segments.size()); @@ -243,12 +276,14 @@ public void testTransformSpec() throws Exception createTuningConfig(2, null, true, false), false ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); Assert.assertEquals(indexTask.getId(), indexTask.getGroupId()); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(1, segments.size()); @@ -284,10 +319,12 @@ public void testWithArbitraryGranularity() throws Exception createTuningConfig(10, null, false, true), false ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - List segments = runTask(indexTask); + List segments = runTask(indexTask).rhs; Assert.assertEquals(1, segments.size()); } @@ -318,10 +355,12 @@ public void testIntervalBucketing() throws Exception createTuningConfig(50, null, false, true), false ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(1, segments.size()); } @@ -348,10 +387,12 @@ public void testNumShardsProvided() throws Exception createTuningConfig(null, 1, false, true), false ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(1, segments.size()); @@ -384,12 +425,14 @@ public void testAppendToExisting() throws Exception createTuningConfig(2, null, false, false), true ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); Assert.assertEquals("index_append_test", indexTask.getGroupId()); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(2, segmentAllocatePartitionCounter); Assert.assertEquals(2, segments.size()); @@ -431,10 +474,12 @@ public void testIntervalNotSpecified() throws Exception createTuningConfig(2, null, false, true), false ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(3, segments.size()); @@ -491,10 +536,12 @@ public void testCSVFileWithHeader() throws Exception createTuningConfig(2, null, false, true), false ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(1, segments.size()); @@ -540,10 +587,12 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception createTuningConfig(2, null, false, true), false ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(1, segments.size()); @@ -584,10 +633,12 @@ public void testWithSmallMaxTotalRows() throws Exception createTuningConfig(2, 2, null, 2L, null, false, false, true), false ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(6, segments.size()); @@ -626,10 +677,12 @@ public void testPerfectRollup() throws Exception createTuningConfig(3, 2, null, 2L, null, false, true, true), false ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(3, segments.size()); @@ -667,10 +720,12 @@ public void testBestEffortRollup() throws Exception createTuningConfig(3, 2, null, 2L, null, false, false, true), false ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(5, segments.size()); @@ -742,10 +797,12 @@ public void testIgnoreParseException() throws Exception null, null, parseExceptionIgnoreSpec, + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(Arrays.asList("d"), segments.get(0).getDimensions()); Assert.assertEquals(Arrays.asList("val"), segments.get(0).getMetrics()); @@ -755,9 +812,6 @@ public void testIgnoreParseException() throws Exception @Test public void testReportParseException() throws Exception { - expectedException.expect(ParseException.class); - expectedException.expectMessage("Unparseable timestamp found!"); - final File tmpDir = temporaryFolder.newFolder(); final File tmpFile = File.createTempFile("druid", "index", tmpDir); @@ -795,12 +849,356 @@ public void testReportParseException() throws Exception null, null, parseExceptionIgnoreSpec, + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + null + ); + + TaskStatus status = runTask(indexTask).lhs; + Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); + checkTaskStatusErrorMsgForParseExceptionsExceeded(status); + + Map expectedUnparseables = ImmutableMap.of( + "determinePartitions", + new ArrayList<>(), + "buildSegments", + Arrays.asList("Unparseable timestamp found! Event: {time=unparseable, d=a, val=1}") + ); + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + Assert.assertEquals(expectedUnparseables, reportData.getUnparseableEvents()); + } + + @Test + public void testMultipleParseExceptionsSuccess() throws Exception + { + final File tmpDir = temporaryFolder.newFolder(); + + final File tmpFile = File.createTempFile("druid", "index", tmpDir); + + try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { + writer.write("{\"time\":\"unparseable\",\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}\n"); // unparseable time + writer.write("{\"time\":\"2014-01-01T00:00:10Z\",\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}\n"); // valid row + writer.write("{\"time\":\"2014-01-01T00:00:10Z\",\"dim\":\"b\",\"dimLong\":\"notnumber\",\"dimFloat\":3.0,\"val\":1}\n"); // row with invalid long dimension + writer.write("{\"time\":\"2014-01-01T00:00:10Z\",\"dim\":\"b\",\"dimLong\":2,\"dimFloat\":\"notnumber\",\"val\":1}\n"); // row with invalid float dimension + writer.write("{\"time\":\"2014-01-01T00:00:10Z\",\"dim\":\"b\",\"dimLong\":2,\"dimFloat\":4.0,\"val\":\"notnumber\"}\n"); // row with invalid metric + writer.write("{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}\n"); // invalid JSON + writer.write("{\"time\":\"3014-03-01T00:00:10Z\",\"dim\":\"outsideofinterval\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}\n"); // thrown away + writer.write("{\"time\":\"99999999999-01-01T00:00:10Z\",\"dim\":\"b\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}\n"); // unparseable time + writer.write("this is not JSON\n"); // invalid JSON + } + + final IndexTask.IndexTuningConfig tuningConfig = new IndexTask.IndexTuningConfig( + 2, + null, + null, + null, + null, + indexSpec, + null, + true, + false, + true, + false, + null, + null, + null, + true, + 7, + 7 + ); + + final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + tmpDir, + new JSONParseSpec( + new TimestampSpec( + "time", + "auto", + null + ), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ), + Lists.newArrayList(), + Lists.newArrayList() + ), + null, + null + ), + null, + tuningConfig, + false + ); + + IndexTask indexTask = new IndexTask( + null, + null, + parseExceptionIgnoreSpec, + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + null + ); + + TaskStatus status = runTask(indexTask).lhs; + Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); + Assert.assertEquals(null, status.getErrorMsg()); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedMetrics = ImmutableMap.of( + "determinePartitions", + ImmutableMap.of( + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, 0, + TaskMetricsUtils.ROWS_PROCESSED, 4, + TaskMetricsUtils.ROWS_UNPARSEABLE, 4, + TaskMetricsUtils.ROWS_THROWN_AWAY, 1 + ), + "buildSegments", + ImmutableMap.of( + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, 3, + TaskMetricsUtils.ROWS_PROCESSED, 1, + TaskMetricsUtils.ROWS_UNPARSEABLE, 4, + TaskMetricsUtils.ROWS_THROWN_AWAY, 1 + ) + ); + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + + Map expectedUnparseables = ImmutableMap.of( + "determinePartitions", + Arrays.asList( + "Unable to parse row [this is not JSON]", + "Unparseable timestamp found! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}", + "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}]", + "Unparseable timestamp found! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}" + ), + "buildSegments", + Arrays.asList( + "Unable to parse row [this is not JSON]", + "Unparseable timestamp found! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}", + "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}]", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2014-01-01T00:00:10.000Z, event={time=2014-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=4.0, val=notnumber}, dimensions=[dim, dimLong, dimFloat]}], exceptions: [Unable to parse value[notnumber] for field[val],]", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2014-01-01T00:00:10.000Z, event={time=2014-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=notnumber, val=1}, dimensions=[dim, dimLong, dimFloat]}], exceptions: [could not convert value [notnumber] to float,]", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2014-01-01T00:00:10.000Z, event={time=2014-01-01T00:00:10Z, dim=b, dimLong=notnumber, dimFloat=3.0, val=1}, dimensions=[dim, dimLong, dimFloat]}], exceptions: [could not convert value [notnumber] to long,]", + "Unparseable timestamp found! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}" + ) + ); + + Assert.assertEquals(expectedUnparseables, reportData.getUnparseableEvents()); + } + + @Test + public void testMultipleParseExceptionsFailure() throws Exception + { + final File tmpDir = temporaryFolder.newFolder(); + + final File tmpFile = File.createTempFile("druid", "index", tmpDir); + + try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { + writer.write("time,dim,dimLong,dimFloat,val\n"); + writer.write("unparseable,a,2,3.0,1\n"); // unparseable + writer.write("2014-01-01T00:00:10Z,a,2,3.0,1\n"); // valid row + writer.write("9.0,a,2,3.0,1\n"); // unparseable + writer.write("3014-03-01T00:00:10Z,outsideofinterval,2,3.0,1\n"); // thrown away + writer.write("99999999999-01-01T00:00:10Z,b,2,3.0,1\n"); // unparseable + } + + // Allow up to 3 parse exceptions, and save up to 2 parse exceptions + final IndexTask.IndexTuningConfig tuningConfig = new IndexTask.IndexTuningConfig( + 2, + null, + null, + null, + null, + indexSpec, + null, + true, + false, + false, + false, + null, + null, + null, + true, + 2, + 5 + ); + + final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + tmpDir, + new CSVParseSpec( + new TimestampSpec( + "time", + "auto", + null + ), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ), + Lists.newArrayList(), + Lists.newArrayList() + ), + null, + Arrays.asList("time", "dim", "dimLong", "dimFloat", "val"), + true, + 0 + ), + null, + tuningConfig, + false + ); + + IndexTask indexTask = new IndexTask( + null, + null, + parseExceptionIgnoreSpec, + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + null + ); + + TaskStatus status = runTask(indexTask).lhs; + Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); + checkTaskStatusErrorMsgForParseExceptionsExceeded(status); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedMetrics = ImmutableMap.of( + "buildSegments", + ImmutableMap.of( + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, 0, + TaskMetricsUtils.ROWS_PROCESSED, 1, + TaskMetricsUtils.ROWS_UNPARSEABLE, 3, + TaskMetricsUtils.ROWS_THROWN_AWAY, 2 + ) + ); + + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + + Map expectedUnparseables = ImmutableMap.of( + "determinePartitions", + new ArrayList<>(), + "buildSegments", + Arrays.asList( + "Unparseable timestamp found! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}", + "Unparseable timestamp found! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1}", + "Unparseable timestamp found! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}" + ) + ); + + Assert.assertEquals(expectedUnparseables, reportData.getUnparseableEvents()); + } + + @Test + public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exception + { + final File tmpDir = temporaryFolder.newFolder(); + + final File tmpFile = File.createTempFile("druid", "index", tmpDir); + + try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { + writer.write("time,dim,dimLong,dimFloat,val\n"); + writer.write("unparseable,a,2,3.0,1\n"); // unparseable + writer.write("2014-01-01T00:00:10Z,a,2,3.0,1\n"); // valid row + writer.write("9.0,a,2,3.0,1\n"); // unparseable + writer.write("3014-03-01T00:00:10Z,outsideofinterval,2,3.0,1\n"); // thrown away + writer.write("99999999999-01-01T00:00:10Z,b,2,3.0,1\n"); // unparseable + } + + // Allow up to 3 parse exceptions, and save up to 2 parse exceptions + final IndexTask.IndexTuningConfig tuningConfig = new IndexTask.IndexTuningConfig( + 2, + null, + null, + null, + null, + indexSpec, + null, + true, + false, + true, + false, + null, + null, + null, + true, + 2, + 5 + ); + + final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + tmpDir, + new CSVParseSpec( + new TimestampSpec( + "time", + "auto", + null + ), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ), + Lists.newArrayList(), + Lists.newArrayList() + ), + null, + Arrays.asList("time", "dim", "dimLong", "dimFloat", "val"), + true, + 0 + ), + null, + tuningConfig, + false + ); + + IndexTask indexTask = new IndexTask( + null, + null, + parseExceptionIgnoreSpec, + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - runTask(indexTask); + TaskStatus status = runTask(indexTask).lhs; + Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); + checkTaskStatusErrorMsgForParseExceptionsExceeded(status); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedMetrics = ImmutableMap.of( + "determinePartitions", + ImmutableMap.of( + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, 0, + TaskMetricsUtils.ROWS_PROCESSED, 1, + TaskMetricsUtils.ROWS_UNPARSEABLE, 3, + TaskMetricsUtils.ROWS_THROWN_AWAY, 2 + ) + ); + + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + + Map expectedUnparseables = ImmutableMap.of( + "determinePartitions", + Arrays.asList( + "Unparseable timestamp found! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}", + "Unparseable timestamp found! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1}", + "Unparseable timestamp found! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}" + ), + "buildSegments", + new ArrayList<>() + ); + + Assert.assertEquals(expectedUnparseables, reportData.getUnparseableEvents()); } + @Test public void testCsvWithHeaderOfEmptyColumns() throws Exception { @@ -854,10 +1252,12 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception null, null, parseExceptionIgnoreSpec, + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - final List segments = runTask(indexTask); + final List segments = runTask(indexTask).rhs; // the order of result segments can be changed because hash shardSpec is used. // the below loop is to make this test deterministic. Assert.assertEquals(2, segments.size()); @@ -885,9 +1285,6 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception @Test public void testCsvWithHeaderOfEmptyTimestamp() throws Exception { - expectedException.expect(ParseException.class); - expectedException.expectMessage("Unparseable timestamp found!"); - final File tmpDir = temporaryFolder.newFolder(); final File tmpFile = File.createTempFile("druid", "index", tmpDir); @@ -924,13 +1321,34 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception null, null, parseExceptionIgnoreSpec, + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); - runTask(indexTask); + TaskStatus status = runTask(indexTask).lhs; + Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); + + checkTaskStatusErrorMsgForParseExceptionsExceeded(status); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedUnparseables = ImmutableMap.of( + "determinePartitions", + new ArrayList<>(), + "buildSegments", + Arrays.asList("Unparseable timestamp found! Event: {column_1=2014-01-01T00:00:10Z, column_2=a, column_3=1}") + ); + Assert.assertEquals(expectedUnparseables, reportData.getUnparseableEvents()); + } + + public static void checkTaskStatusErrorMsgForParseExceptionsExceeded(TaskStatus status) + { + // full stacktrace will be too long and make tests brittle (e.g. if line # changes), just match the main message + Assert.assertTrue(status.getErrorMsg().contains("Max parse exceptions exceeded, terminating task...")); } - private List runTask(IndexTask indexTask) throws Exception + private Pair> runTask(IndexTask indexTask) throws Exception { final List segments = Lists.newArrayList(); @@ -1044,15 +1462,15 @@ public Map makeLoadSpec(URI uri) null, null, null, - new NoopTestTaskFileWriter() + new TaskReportFileWriter(reportsFile) ); indexTask.isReady(box.getTaskActionClient()); - indexTask.run(box); + TaskStatus status = indexTask.run(box); Collections.sort(segments); - return segments; + return Pair.of(status, segments); } private IndexTask.IndexIngestionSpec createIngestionSpec( @@ -1153,7 +1571,23 @@ private static IndexTuningConfig createTuningConfig( reportParseException, null, null, - null + null, + null, + null, + 1 + ); + } + + private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException + { + Map taskReports = jsonMapper.readValue( + reportsFile, + new TypeReference>() + { + } + ); + return IngestionStatsAndErrorsTaskReportData.getPayloadFromTaskReports( + taskReports ); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/NoopTestTaskFileWriter.java b/indexing-service/src/test/java/io/druid/indexing/common/task/NoopTestTaskFileWriter.java index cebee6c624f6..f9bfe12d772c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/NoopTestTaskFileWriter.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/NoopTestTaskFileWriter.java @@ -22,6 +22,8 @@ import io.druid.indexing.common.TaskReport; import io.druid.indexing.common.TaskReportFileWriter; +import java.util.Map; + public class NoopTestTaskFileWriter extends TaskReportFileWriter { public NoopTestTaskFileWriter() @@ -30,7 +32,7 @@ public NoopTestTaskFileWriter() } @Override - public void write(TaskReport report) + public void write(Map reports) { } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index a8f10f9c9b42..9d02ed1c63ee 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -128,7 +128,6 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.junit.internal.matchers.ThrowableCauseMatcher; import org.junit.internal.matchers.ThrowableMessageMatcher; import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; @@ -478,21 +477,10 @@ public void testReportParseExceptionsOnBadMetric() throws Exception expectedException.expectCause(CoreMatchers.instanceOf(ParseException.class)); expectedException.expectCause( ThrowableMessageMatcher.hasMessage( - CoreMatchers.containsString("Encountered parse error for aggregator[met1]") - ) - ); - expectedException.expect( - ThrowableCauseMatcher.hasCause( - ThrowableCauseMatcher.hasCause( - CoreMatchers.allOf( - CoreMatchers.instanceOf(ParseException.class), - ThrowableMessageMatcher.hasMessage( - CoreMatchers.containsString("Unable to parse value[foo] for field[met1]") - ) - ) - ) + CoreMatchers.containsString("[Unable to parse value[foo] for field[met1]") ) ); + statusFuture.get(); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskReportSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskReportSerdeTest.java new file mode 100644 index 000000000000..62edbbbdc0c0 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskReportSerdeTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import io.druid.indexer.IngestionState; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReport; +import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import io.druid.indexing.common.TaskReport; +import io.druid.indexing.common.TestUtils; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +public class TaskReportSerdeTest +{ + private final ObjectMapper jsonMapper; + + public TaskReportSerdeTest() + { + TestUtils testUtils = new TestUtils(); + jsonMapper = testUtils.getTestObjectMapper(); + } + + @Test + public void testSerde() throws Exception + { + IngestionStatsAndErrorsTaskReport report1 = new IngestionStatsAndErrorsTaskReport( + "testID", + new IngestionStatsAndErrorsTaskReportData( + IngestionState.BUILD_SEGMENTS, + ImmutableMap.of( + "hello", "world" + ), + ImmutableMap.of( + "number", 1234 + ), + "an error message" + ) + ); + String report1serialized = jsonMapper.writeValueAsString(report1); + IngestionStatsAndErrorsTaskReport report2 = jsonMapper.readValue( + report1serialized, + IngestionStatsAndErrorsTaskReport.class + ); + Assert.assertEquals(report1, report2); + + + Map reportMap1 = TaskReport.buildTaskReports(report1); + String reportMapSerialized = jsonMapper.writeValueAsString(reportMap1); + Map reportMap2 = jsonMapper.readValue( + reportMapSerialized, + new TypeReference>() + { + } + ); + Assert.assertEquals(reportMap1, reportMap2); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 5c1ab4a7c9f4..8bea1819b841 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -52,6 +52,7 @@ import io.druid.segment.realtime.plumber.Plumber; import io.druid.segment.realtime.plumber.PlumberSchool; import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import io.druid.server.security.AuthTestUtils; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.hamcrest.CoreMatchers; @@ -205,9 +206,14 @@ public void testIndexTaskSerde() throws Exception null, null, null, + null, + null, + null, null ) ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); @@ -284,9 +290,14 @@ public void testIndexTaskwithResourceSerde() throws Exception null, null, null, + null, + null, + null, null ) ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); @@ -814,6 +825,8 @@ public void testHadoopIndexTaskSerde() throws Exception null, "blah", jsonMapper, + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 52823c83aa88..cde42246b0b6 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -115,6 +115,7 @@ import io.druid.server.coordination.ServerType; import io.druid.server.initialization.ServerConfig; import io.druid.server.metrics.NoopServiceEmitter; +import io.druid.server.security.AuthTestUtils; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -682,9 +683,14 @@ public void testIndexTask() throws Exception null, null, null, + null, + null, + null, null ) ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); @@ -756,9 +762,14 @@ public void testIndexTaskFailure() throws Exception null, null, null, + null, + null, + null, null ) ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); @@ -1137,9 +1148,14 @@ public void testResumeTasks() throws Exception null, null, null, + null, + null, + null, null ) ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, null ); diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/ObjectFlatteners.java b/java-util/src/main/java/io/druid/java/util/common/parsers/ObjectFlatteners.java index cae3eb686460..63c48987b9c9 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/ObjectFlatteners.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/ObjectFlatteners.java @@ -190,7 +190,7 @@ public Object setValue(final Object value) }; } ) - .collect(Collectors.toSet()); + .collect(Collectors.toCollection(LinkedHashSet::new)); } }; } diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/ParseException.java b/java-util/src/main/java/io/druid/java/util/common/parsers/ParseException.java index ca1f5ff67cf2..4c8b797ee593 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/ParseException.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/ParseException.java @@ -25,6 +25,8 @@ */ public class ParseException extends RuntimeException { + private boolean fromPartiallyValidRow = false; + public ParseException(String formatText, Object... arguments) { super(StringUtils.nonStrictFormat(formatText, arguments)); @@ -34,4 +36,14 @@ public ParseException(Throwable cause, String formatText, Object... arguments) { super(StringUtils.nonStrictFormat(formatText, arguments), cause); } + + public boolean isFromPartiallyValidRow() + { + return fromPartiallyValidRow; + } + + public void setFromPartiallyValidRow(boolean fromPartiallyValidRow) + { + this.fromPartiallyValidRow = fromPartiallyValidRow; + } } diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index fb09dbc97c53..1c08687476d9 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -44,6 +44,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularity; +import io.druid.java.util.common.parsers.ParseException; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; @@ -462,7 +463,7 @@ protected abstract AggregatorType[] initAggs( ); // Note: This method needs to be thread safe. - protected abstract Integer addToFacts( + protected abstract AddToFactsResult addToFacts( AggregatorFactory[] metrics, boolean deserializeComplexMetrics, boolean reportParseExceptions, @@ -491,6 +492,55 @@ protected abstract Integer addToFacts( protected abstract boolean isNull(int rowOffset, int aggOffset); + public static class TimeAndDimsResult + { + private TimeAndDims timeAndDims; + private List parseExceptionMessages; + + public TimeAndDimsResult( + TimeAndDims timeAndDims, + List parseExceptionMessages + ) + { + this.timeAndDims = timeAndDims; + this.parseExceptionMessages = parseExceptionMessages; + } + + public TimeAndDims getTimeAndDims() + { + return timeAndDims; + } + + public List getParseExceptionMessages() + { + return parseExceptionMessages; + } + } + + public static class AddToFactsResult + { + private int rowCount; + private List parseExceptionMessages; + + public AddToFactsResult( + int rowCount, + List parseExceptionMessages + ) + { + this.rowCount = rowCount; + this.parseExceptionMessages = parseExceptionMessages; + } + + public int getRowCount() + { + return rowCount; + } + + public List getParseExceptionMessages() + { + return parseExceptionMessages; + } + } @Override public void close() @@ -526,32 +576,37 @@ public Map getColumnCapabilities() * * @return the number of rows in the data set after adding the InputRow */ - public int add(InputRow row) throws IndexSizeExceededException + public IncrementalIndexAddResult add(InputRow row) throws IndexSizeExceededException { return add(row, false); } - public int add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException + public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException { - TimeAndDims key = toTimeAndDims(row); - final int rv = addToFacts( + TimeAndDimsResult timeAndDimsResult = toTimeAndDims(row); + final AddToFactsResult addToFactsResult = addToFacts( metrics, deserializeComplexMetrics, reportParseExceptions, row, numEntries, sizeInBytes, - key, + timeAndDimsResult.getTimeAndDims(), in, rowSupplier, skipMaxRowsInMemoryCheck ); updateMaxIngestedTime(row.getTimestamp()); - return rv; + ParseException parseException = getCombinedParseException( + row, + timeAndDimsResult.getParseExceptionMessages(), + addToFactsResult.getParseExceptionMessages() + ); + return new IncrementalIndexAddResult(addToFactsResult.getRowCount(), parseException); } @VisibleForTesting - TimeAndDims toTimeAndDims(InputRow row) + TimeAndDimsResult toTimeAndDims(InputRow row) { row = formatRow(row); if (row.getTimestampFromEpoch() < minTimestamp) { @@ -563,6 +618,7 @@ TimeAndDims toTimeAndDims(InputRow row) Object[] dims; List overflow = null; long dimsKeySize = 0; + List parseExceptionMessages = new ArrayList<>(); synchronized (dimensionDescs) { dims = new Object[dimensionDescs.size()]; for (String dimension : rowDimensions) { @@ -590,12 +646,17 @@ TimeAndDims toTimeAndDims(InputRow row) } DimensionHandler handler = desc.getHandler(); DimensionIndexer indexer = desc.getIndexer(); - Object dimsKey = indexer.processRowValsToUnsortedEncodedKeyComponent( - row.getRaw(dimension), - reportParseExceptions - ); + Object dimsKey = null; + try { + dimsKey = indexer.processRowValsToUnsortedEncodedKeyComponent( + row.getRaw(dimension), + true + ); + } + catch (ParseException pe) { + parseExceptionMessages.add(pe.getMessage()); + } dimsKeySize += indexer.estimateEncodedKeyComponentSize(dimsKey); - // Set column capabilities as data is coming in if (!capabilities.hasMultipleValues() && dimsKey != null && handler.getLengthOfEncodedKeyComponent(dimsKey) > 1) { capabilities.setHasMultipleValues(true); @@ -637,7 +698,45 @@ TimeAndDims toTimeAndDims(InputRow row) if (row.getTimestamp() != null) { truncated = gran.bucketStart(row.getTimestamp()).getMillis(); } - return new TimeAndDims(Math.max(truncated, minTimestamp), dims, dimensionDescsList, dimsKeySize); + + TimeAndDims timeAndDims = new TimeAndDims(Math.max(truncated, minTimestamp), dims, dimensionDescsList); + return new TimeAndDimsResult(timeAndDims, parseExceptionMessages); + } + + public static ParseException getCombinedParseException( + InputRow row, + List dimParseExceptionMessages, + List aggParseExceptionMessages + ) + { + int numAdded = 0; + StringBuilder stringBuilder = new StringBuilder(); + + if (dimParseExceptionMessages != null) { + for (String parseExceptionMessage : dimParseExceptionMessages) { + stringBuilder.append(parseExceptionMessage); + stringBuilder.append(","); + numAdded++; + } + } + if (aggParseExceptionMessages != null) { + for (String parseExceptionMessage : aggParseExceptionMessages) { + stringBuilder.append(parseExceptionMessage); + stringBuilder.append(","); + numAdded++; + } + } + + if (numAdded == 0) { + return null; + } + ParseException pe = new ParseException( + "Found unparseable columns in row: [%s], exceptions: [%s]", + row, + stringBuilder.toString() + ); + pe.setFromPartiallyValidRow(true); + return pe; } private synchronized void updateMaxIngestedTime(DateTime eventTime) diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAddResult.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAddResult.java new file mode 100644 index 000000000000..06c537a0aa36 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAddResult.java @@ -0,0 +1,52 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.incremental; + +import io.druid.java.util.common.parsers.ParseException; + +import javax.annotation.Nullable; + +public class IncrementalIndexAddResult +{ + private final int rowCount; + + @Nullable + private final ParseException parseException; + + public IncrementalIndexAddResult( + int rowCount, + @Nullable ParseException parseException + ) + { + this.rowCount = rowCount; + this.parseException = parseException; + } + + public int getRowCount() + { + return rowCount; + } + + @Nullable + public ParseException getParseException() + { + return parseException; + } +} diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java index c5553578403d..220b2de9cb4e 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -139,7 +139,7 @@ protected BufferAggregator[] initAggs( } @Override - protected Integer addToFacts( + protected AddToFactsResult addToFacts( AggregatorFactory[] metrics, boolean deserializeComplexMetrics, boolean reportParseExceptions, @@ -240,7 +240,7 @@ protected Integer addToFacts( } } rowContainer.set(null); - return numEntries.get(); + return new AddToFactsResult(numEntries.get(), new ArrayList<>()); } @Override diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 45defbfc7566..7d35bdeccda6 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -38,7 +38,9 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.Arrays; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; @@ -126,7 +128,7 @@ protected Aggregator[] initAggs( } @Override - protected Integer addToFacts( + protected AddToFactsResult addToFacts( AggregatorFactory[] metrics, boolean deserializeComplexMetrics, boolean reportParseExceptions, @@ -139,17 +141,18 @@ protected Integer addToFacts( boolean skipMaxRowsInMemoryCheck ) throws IndexSizeExceededException { + List parseExceptionMessages; final int priorIndex = facts.getPriorIndex(key); Aggregator[] aggs; if (TimeAndDims.EMPTY_ROW_INDEX != priorIndex) { aggs = concurrentGet(priorIndex); - doAggregate(metrics, aggs, rowContainer, row, reportParseExceptions); + parseExceptionMessages = doAggregate(metrics, aggs, rowContainer, row); } else { aggs = new Aggregator[metrics.length]; factorizeAggs(metrics, aggs, rowContainer, row); - doAggregate(metrics, aggs, rowContainer, row, reportParseExceptions); + parseExceptionMessages = doAggregate(metrics, aggs, rowContainer, row); final int rowIndex = indexIncrement.getAndIncrement(); concurrentSet(rowIndex, aggs); @@ -171,14 +174,14 @@ protected Integer addToFacts( } else { // We lost a race aggs = concurrentGet(prev); - doAggregate(metrics, aggs, rowContainer, row, reportParseExceptions); + parseExceptionMessages = doAggregate(metrics, aggs, rowContainer, row); // Free up the misfire concurrentRemove(rowIndex); // This is expected to occur ~80% of the time in the worst scenarios } } - return numEntries.get(); + return new AddToFactsResult(numEntries.get(), parseExceptionMessages); } private long estimateRowSizeInBytes(TimeAndDims key) @@ -207,14 +210,14 @@ private void factorizeAggs( rowContainer.set(null); } - private void doAggregate( + private List doAggregate( AggregatorFactory[] metrics, Aggregator[] aggs, ThreadLocal rowContainer, - InputRow row, - boolean reportParseExceptions + InputRow row ) { + List parseExceptionMessages = new ArrayList<>(); rowContainer.set(row); for (int i = 0; i < aggs.length; i++) { @@ -225,16 +228,14 @@ private void doAggregate( } catch (ParseException e) { // "aggregate" can throw ParseExceptions if a selector expects something but gets something else. - if (reportParseExceptions) { - throw new ParseException(e, "Encountered parse error for aggregator[%s]", metrics[i].getName()); - } else { - log.debug(e, "Encountered parse error, skipping aggregator[%s].", metrics[i].getName()); - } + log.debug(e, "Encountered parse error, skipping aggregator[%s].", metrics[i].getName()); + parseExceptionMessages.add(e.getMessage()); } } } rowContainer.set(null); + return parseExceptionMessages; } private void closeAggregators() diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java index b9e89fee9462..4f34520e63fb 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java @@ -215,11 +215,10 @@ public void testUnparseableNumerics() throws IndexSizeExceededException { IncrementalIndex index = closer.closeLater(indexCreator.createIndex()); - expectedException.expect(ParseException.class); - expectedException.expectMessage("could not convert value [asdj] to long"); - index.add( + IncrementalIndexAddResult result; + result = index.add( new MapBasedInputRow( - System.currentTimeMillis() - 1, + 0, Lists.newArrayList("string", "float", "long", "double"), ImmutableMap.of( "string", "A", @@ -229,12 +228,15 @@ public void testUnparseableNumerics() throws IndexSizeExceededException ) ) ); + Assert.assertEquals(ParseException.class, result.getParseException().getClass()); + Assert.assertEquals( + "Found unparseable columns in row: [MapBasedInputRow{timestamp=1970-01-01T00:00:00.000Z, event={string=A, float=19.0, long=asdj, double=21.0}, dimensions=[string, float, long, double]}], exceptions: [could not convert value [asdj] to long,]", + result.getParseException().getMessage() + ); - expectedException.expect(ParseException.class); - expectedException.expectMessage("could not convert value [aaa] to float"); - index.add( + result = index.add( new MapBasedInputRow( - System.currentTimeMillis() - 1, + 0, Lists.newArrayList("string", "float", "long", "double"), ImmutableMap.of( "string", "A", @@ -244,12 +246,15 @@ public void testUnparseableNumerics() throws IndexSizeExceededException ) ) ); + Assert.assertEquals(ParseException.class, result.getParseException().getClass()); + Assert.assertEquals( + "Found unparseable columns in row: [MapBasedInputRow{timestamp=1970-01-01T00:00:00.000Z, event={string=A, float=aaa, long=20, double=21.0}, dimensions=[string, float, long, double]}], exceptions: [could not convert value [aaa] to float,]", + result.getParseException().getMessage() + ); - expectedException.expect(ParseException.class); - expectedException.expectMessage("could not convert value [] to double"); - index.add( + result = index.add( new MapBasedInputRow( - System.currentTimeMillis() - 1, + 0, Lists.newArrayList("string", "float", "long", "double"), ImmutableMap.of( "string", "A", @@ -259,6 +264,11 @@ public void testUnparseableNumerics() throws IndexSizeExceededException ) ) ); + Assert.assertEquals(ParseException.class, result.getParseException().getClass()); + Assert.assertEquals( + "Found unparseable columns in row: [MapBasedInputRow{timestamp=1970-01-01T00:00:00.000Z, event={string=A, float=19.0, long=20, double=}, dimensions=[string, float, long, double]}], exceptions: [could not convert value [] to double,]", + result.getParseException().getMessage() + ); } @Test diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index 2f859c941991..df2713027f2c 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -171,7 +171,7 @@ protected void concurrentSet(int offset, Aggregator[] value) } @Override - protected Integer addToFacts( + protected AddToFactsResult addToFacts( AggregatorFactory[] metrics, boolean deserializeComplexMetrics, boolean reportParseExceptions, @@ -241,8 +241,7 @@ protected Integer addToFacts( rowContainer.set(null); - - return numEntries.get(); + return new AddToFactsResult(numEntries.get(), new ArrayList<>()); } @Override diff --git a/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsCompTest.java b/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsCompTest.java index bdf097da71d1..fa9490eae490 100644 --- a/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsCompTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsCompTest.java @@ -45,13 +45,13 @@ public void testBasic() .buildOnheap(); long time = System.currentTimeMillis(); - TimeAndDims td1 = index.toTimeAndDims(toMapRow(time, "billy", "A", "joe", "B")); - TimeAndDims td2 = index.toTimeAndDims(toMapRow(time, "billy", "A", "joe", "A")); - TimeAndDims td3 = index.toTimeAndDims(toMapRow(time, "billy", "A")); + TimeAndDims td1 = index.toTimeAndDims(toMapRow(time, "billy", "A", "joe", "B")).getTimeAndDims(); + TimeAndDims td2 = index.toTimeAndDims(toMapRow(time, "billy", "A", "joe", "A")).getTimeAndDims(); + TimeAndDims td3 = index.toTimeAndDims(toMapRow(time, "billy", "A")).getTimeAndDims(); - TimeAndDims td4 = index.toTimeAndDims(toMapRow(time + 1, "billy", "A", "joe", "B")); - TimeAndDims td5 = index.toTimeAndDims(toMapRow(time + 1, "billy", "A", "joe", Arrays.asList("A", "B"))); - TimeAndDims td6 = index.toTimeAndDims(toMapRow(time + 1)); + TimeAndDims td4 = index.toTimeAndDims(toMapRow(time + 1, "billy", "A", "joe", "B")).getTimeAndDims(); + TimeAndDims td5 = index.toTimeAndDims(toMapRow(time + 1, "billy", "A", "joe", Arrays.asList("A", "B"))).getTimeAndDims(); + TimeAndDims td6 = index.toTimeAndDims(toMapRow(time + 1)).getTimeAndDims(); Comparator comparator = index.dimsComparator(); diff --git a/server/src/main/java/io/druid/segment/indexing/TuningConfig.java b/server/src/main/java/io/druid/segment/indexing/TuningConfig.java index 998f07857527..7fd246d25732 100644 --- a/server/src/main/java/io/druid/segment/indexing/TuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/TuningConfig.java @@ -30,4 +30,7 @@ }) public interface TuningConfig { + boolean DEFAULT_LOG_PARSE_EXCEPTIONS = false; + int DEFAULT_MAX_PARSE_EXCEPTIONS = Integer.MAX_VALUE; + int DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS = 0; } diff --git a/server/src/main/java/io/druid/segment/realtime/FireDepartmentMetrics.java b/server/src/main/java/io/druid/segment/realtime/FireDepartmentMetrics.java index d61de1aee04e..302b58663ced 100644 --- a/server/src/main/java/io/druid/segment/realtime/FireDepartmentMetrics.java +++ b/server/src/main/java/io/druid/segment/realtime/FireDepartmentMetrics.java @@ -28,6 +28,7 @@ public class FireDepartmentMetrics { private final AtomicLong processedCount = new AtomicLong(0); + private final AtomicLong processedWithErrorsCount = new AtomicLong(0); private final AtomicLong thrownAwayCount = new AtomicLong(0); private final AtomicLong unparseableCount = new AtomicLong(0); private final AtomicLong rowOutputCount = new AtomicLong(0); @@ -49,6 +50,11 @@ public void incrementProcessed() processedCount.incrementAndGet(); } + public void incrementProcessedWithErrors() + { + processedWithErrorsCount.incrementAndGet(); + } + public void incrementThrownAway() { thrownAwayCount.incrementAndGet(); @@ -124,6 +130,11 @@ public long processed() return processedCount.get(); } + public long processedWithErrors() + { + return processedWithErrorsCount.get(); + } + public long thrownAway() { return thrownAwayCount.get(); @@ -203,6 +214,7 @@ public FireDepartmentMetrics snapshot() { final FireDepartmentMetrics retVal = new FireDepartmentMetrics(); retVal.processedCount.set(processedCount.get()); + retVal.processedWithErrorsCount.set(processedWithErrorsCount.get()); retVal.thrownAwayCount.set(thrownAwayCount.get()); retVal.unparseableCount.set(unparseableCount.get()); retVal.rowOutputCount.set(rowOutputCount.get()); @@ -231,6 +243,7 @@ public FireDepartmentMetrics merge(FireDepartmentMetrics other) Preconditions.checkNotNull(other, "Cannot merge a null FireDepartmentMetrics"); FireDepartmentMetrics otherSnapshot = other.snapshot(); processedCount.addAndGet(otherSnapshot.processed()); + processedWithErrorsCount.addAndGet(otherSnapshot.processedWithErrors()); thrownAwayCount.addAndGet(otherSnapshot.thrownAway()); rowOutputCount.addAndGet(otherSnapshot.rowOutput()); unparseableCount.addAndGet(otherSnapshot.unparseable()); @@ -248,5 +261,4 @@ public FireDepartmentMetrics merge(FireDepartmentMetrics other) messageGap.set(Math.max(messageGap(), otherSnapshot.messageGap())); return this; } - } diff --git a/server/src/main/java/io/druid/segment/realtime/FireDepartmentMetricsTaskMetricsGetter.java b/server/src/main/java/io/druid/segment/realtime/FireDepartmentMetricsTaskMetricsGetter.java new file mode 100644 index 000000000000..9c7ee60fde18 --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/FireDepartmentMetricsTaskMetricsGetter.java @@ -0,0 +1,64 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.realtime; + +import com.google.common.collect.ImmutableMap; +import io.druid.indexer.TaskMetricsGetter; +import io.druid.indexer.TaskMetricsUtils; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +public class FireDepartmentMetricsTaskMetricsGetter implements TaskMetricsGetter +{ + public static final List KEYS = Arrays.asList( + TaskMetricsUtils.ROWS_PROCESSED, + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, + TaskMetricsUtils.ROWS_THROWN_AWAY, + TaskMetricsUtils.ROWS_UNPARSEABLE + ); + + private final FireDepartmentMetrics fireDepartmentMetrics; + + public FireDepartmentMetricsTaskMetricsGetter( + FireDepartmentMetrics fireDepartmentMetrics + ) + { + this.fireDepartmentMetrics = fireDepartmentMetrics; + } + + @Override + public List getKeys() + { + return KEYS; + } + + @Override + public Map getTotalMetrics() + { + return ImmutableMap.of( + TaskMetricsUtils.ROWS_PROCESSED, fireDepartmentMetrics.processed(), + TaskMetricsUtils.ROWS_PROCESSED_WITH_ERRORS, fireDepartmentMetrics.processedWithErrors(), + TaskMetricsUtils.ROWS_THROWN_AWAY, fireDepartmentMetrics.thrownAway(), + TaskMetricsUtils.ROWS_UNPARSEABLE, fireDepartmentMetrics.unparseable() + ); + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java index dbd1ed831378..c8bba5cdeb84 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java @@ -23,6 +23,7 @@ import com.google.common.util.concurrent.ListenableFuture; import io.druid.data.input.Committer; import io.druid.data.input.InputRow; +import io.druid.java.util.common.parsers.ParseException; import io.druid.query.QuerySegmentWalker; import io.druid.segment.incremental.IndexSizeExceededException; @@ -228,11 +229,20 @@ class AppenderatorAddResult private final int numRowsInSegment; private final boolean isPersistRequired; - AppenderatorAddResult(SegmentIdentifier identifier, int numRowsInSegment, boolean isPersistRequired) + @Nullable + private final ParseException parseException; + + AppenderatorAddResult( + SegmentIdentifier identifier, + int numRowsInSegment, + boolean isPersistRequired, + @Nullable ParseException parseException + ) { this.segmentIdentifier = identifier; this.numRowsInSegment = numRowsInSegment; this.isPersistRequired = isPersistRequired; + this.parseException = parseException; } SegmentIdentifier getSegmentIdentifier() @@ -249,5 +259,11 @@ boolean isPersistRequired() { return isPersistRequired; } + + @Nullable + public ParseException getParseException() + { + return parseException; + } } } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverAddResult.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverAddResult.java index a1ec20baa5b7..07a01ab4d021 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverAddResult.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverAddResult.java @@ -19,6 +19,8 @@ package io.druid.segment.realtime.appenderator; +import io.druid.java.util.common.parsers.ParseException; + import javax.annotation.Nullable; /** @@ -33,32 +35,44 @@ public class AppenderatorDriverAddResult private final long totalNumRowsInAppenderator; private final boolean isPersistRequired; + @Nullable + private final ParseException parseException; + public static AppenderatorDriverAddResult ok( SegmentIdentifier segmentIdentifier, int numRowsInSegment, long totalNumRowsInAppenderator, - boolean isPersistRequired + boolean isPersistRequired, + @Nullable ParseException parseException ) { - return new AppenderatorDriverAddResult(segmentIdentifier, numRowsInSegment, totalNumRowsInAppenderator, isPersistRequired); + return new AppenderatorDriverAddResult( + segmentIdentifier, + numRowsInSegment, + totalNumRowsInAppenderator, + isPersistRequired, + parseException + ); } public static AppenderatorDriverAddResult fail() { - return new AppenderatorDriverAddResult(null, 0, 0, false); + return new AppenderatorDriverAddResult(null, 0, 0, false, null); } private AppenderatorDriverAddResult( @Nullable SegmentIdentifier segmentIdentifier, int numRowsInSegment, long totalNumRowsInAppenderator, - boolean isPersistRequired + boolean isPersistRequired, + @Nullable ParseException parseException ) { this.segmentIdentifier = segmentIdentifier; this.numRowsInSegment = numRowsInSegment; this.totalNumRowsInAppenderator = totalNumRowsInAppenderator; this.isPersistRequired = isPersistRequired; + this.parseException = parseException; } public boolean isOk() @@ -85,4 +99,10 @@ public boolean isPersistRequired() { return isPersistRequired; } + + @Nullable + public ParseException getParseException() + { + return parseException; + } } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 751f10a46b61..53ab67b85e29 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -63,6 +63,7 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; +import io.druid.segment.incremental.IncrementalIndexAddResult; import io.druid.segment.incremental.IndexSizeExceededException; import io.druid.segment.indexing.DataSchema; import io.druid.segment.loading.DataSegmentPusher; @@ -218,9 +219,11 @@ public AppenderatorAddResult add( metrics.reportMessageMaxTimestamp(row.getTimestampFromEpoch()); final int sinkRowsInMemoryBeforeAdd = sink.getNumRowsInMemory(); final int sinkRowsInMemoryAfterAdd; + final IncrementalIndexAddResult addResult; try { - sinkRowsInMemoryAfterAdd = sink.add(row, !allowIncrementalPersists); + addResult = sink.add(row, !allowIncrementalPersists); + sinkRowsInMemoryAfterAdd = addResult.getRowCount(); } catch (IndexSizeExceededException e) { // Uh oh, we can't do anything about this! We can't persist (commit metadata would be out of sync) and we @@ -250,7 +253,7 @@ public AppenderatorAddResult add( } } - return new AppenderatorAddResult(identifier, sink.getNumRows(), isPersistRequired); + return new AppenderatorAddResult(identifier, sink.getNumRows(), isPersistRequired, addResult.getParseException()); } @Override diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java index 24482d19c96c..963522117a6c 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java @@ -300,7 +300,8 @@ protected AppenderatorDriverAddResult append( identifier, result.getNumRowsInSegment(), appenderator.getTotalRowCount(), - result.isPersistRequired() + result.isPersistRequired(), + result.getParseException() ); } catch (SegmentNotWritableException e) { diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index dac2be05a11c..235f6067e789 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -61,6 +61,7 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; +import io.druid.segment.incremental.IncrementalIndexAddResult; import io.druid.segment.incremental.IndexSizeExceededException; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; @@ -216,13 +217,16 @@ public int add(InputRow row, Supplier committerSupplier) throws Index return -1; } - final int numRows = sink.add(row, false); + final IncrementalIndexAddResult addResult = sink.add(row, false); + if (config.isReportParseExceptions() && addResult.getParseException() != null) { + throw addResult.getParseException(); + } if (!sink.canAppendRow() || System.currentTimeMillis() > nextFlush) { persist(committerSupplier.get()); } - return numRows; + return addResult.getRowCount(); } private Sink getSink(long timestamp) diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index b655d9f7afe5..8ddd187f1434 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -34,6 +34,7 @@ import io.druid.segment.ReferenceCountingSegment; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexAddResult; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IndexSizeExceededException; import io.druid.segment.indexing.DataSchema; @@ -53,7 +54,7 @@ public class Sink implements Iterable { - private static final int ADD_FAILED = -1; + private static final IncrementalIndexAddResult ADD_FAILED = new IncrementalIndexAddResult(-1, null); private final Object hydrantLock = new Object(); private final Interval interval; @@ -144,7 +145,7 @@ public FireHydrant getCurrHydrant() return currHydrant; } - public int add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException + public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException { if (currHydrant == null) { throw new IAE("No currHydrant but given row[%s]", row); diff --git a/server/src/main/java/io/druid/server/coordination/ChangeRequestHistory.java b/server/src/main/java/io/druid/server/coordination/ChangeRequestHistory.java index afb9857dd556..31bf1f1e2e10 100644 --- a/server/src/main/java/io/druid/server/coordination/ChangeRequestHistory.java +++ b/server/src/main/java/io/druid/server/coordination/ChangeRequestHistory.java @@ -22,13 +22,13 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.AbstractFuture; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.druid.java.util.common.IAE; import io.druid.java.util.common.StringUtils; +import io.druid.utils.CircularBuffer; import java.util.ArrayList; import java.util.LinkedHashMap; @@ -327,46 +327,4 @@ public boolean cancel(boolean interruptIfRunning) return true; } } - - static class CircularBuffer - { - private final E[] buffer; - - private int start = 0; - private int size = 0; - - CircularBuffer(int capacity) - { - buffer = (E[]) new Object[capacity]; - } - - void add(E item) - { - buffer[start++] = item; - - if (start >= buffer.length) { - start = 0; - } - - if (size < buffer.length) { - size++; - } - } - - E get(int index) - { - Preconditions.checkArgument(index >= 0 && index < size, "invalid index"); - - int bufferIndex = (start - size + index) % buffer.length; - if (bufferIndex < 0) { - bufferIndex += buffer.length; - } - return buffer[bufferIndex]; - } - - int size() - { - return size; - } - } } diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 6b78d4005079..0f2c903f1c1f 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -1066,7 +1066,7 @@ public int add(InputRow row, Supplier committerSupplier) throws Index return -1; } - return sink.add(row, false); + return sink.add(row, false).getRowCount(); } public Sink getSink(long timestamp) diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java index 88bbe2b86c78..f031fe4540f7 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java @@ -309,7 +309,7 @@ public AppenderatorAddResult add( { rows.computeIfAbsent(identifier, k -> new ArrayList<>()).add(row); numRows++; - return new AppenderatorAddResult(identifier, numRows, false); + return new AppenderatorAddResult(identifier, numRows, false, null); } @Override diff --git a/server/src/test/java/io/druid/server/coordination/ChangeRequestHistoryTest.java b/server/src/test/java/io/druid/server/coordination/ChangeRequestHistoryTest.java index 05e726f23cbe..b09fedeaa8e6 100644 --- a/server/src/test/java/io/druid/server/coordination/ChangeRequestHistoryTest.java +++ b/server/src/test/java/io/druid/server/coordination/ChangeRequestHistoryTest.java @@ -22,6 +22,7 @@ import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import io.druid.utils.CircularBuffer; import org.junit.Assert; import org.junit.Test; @@ -173,7 +174,7 @@ public void testNonImmediateFuture() throws Exception @Test public void testCircularBuffer() { - ChangeRequestHistory.CircularBuffer circularBuffer = new ChangeRequestHistory.CircularBuffer<>( + CircularBuffer circularBuffer = new CircularBuffer<>( 3); circularBuffer.add(1); From 7f4188f9d74ce4de38d031d424be06812cbdc53e Mon Sep 17 00:00:00 2001 From: Niketh Sabbineni Date: Thu, 5 Apr 2018 22:12:30 -0700 Subject: [PATCH 05/31] Allow getDomain to return disjointed intervals (#5570) * Allow getDomain to return disjointed intervals * Indentation issues --- .../timeline/partition/NoneShardSpec.java | 14 ++------- .../druid/timeline/partition/ShardSpec.java | 4 +-- .../io/druid/timeline/DataSegmentTest.java | 4 +-- .../io/druid/query/filter/DimFilterUtils.java | 18 +++++++++-- .../query/filter/DimFilterUtilsTest.java | 5 ++- .../partition/HashBasedNumberedShardSpec.java | 28 ++++++----------- .../timeline/partition/LinearShardSpec.java | 13 ++------ .../timeline/partition/NumberedShardSpec.java | 13 ++------ .../partition/SingleDimensionShardSpec.java | 31 +++++++++---------- 9 files changed, 55 insertions(+), 75 deletions(-) diff --git a/api/src/main/java/io/druid/timeline/partition/NoneShardSpec.java b/api/src/main/java/io/druid/timeline/partition/NoneShardSpec.java index 1fcadfb42d23..d20c3b5458c2 100644 --- a/api/src/main/java/io/druid/timeline/partition/NoneShardSpec.java +++ b/api/src/main/java/io/druid/timeline/partition/NoneShardSpec.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; import io.druid.data.input.InputRow; import java.util.List; @@ -71,19 +71,11 @@ public int getPartitionNum() @Override public ShardSpecLookup getLookup(final List shardSpecs) { - - return new ShardSpecLookup() - { - @Override - public ShardSpec getShardSpec(long timestamp, InputRow row) - { - return shardSpecs.get(0); - } - }; + return (long timestamp, InputRow row) -> shardSpecs.get(0); } @Override - public Map> getDomain() + public Map> getDomain() { return ImmutableMap.of(); } diff --git a/api/src/main/java/io/druid/timeline/partition/ShardSpec.java b/api/src/main/java/io/druid/timeline/partition/ShardSpec.java index 5461544c609c..c691c3959dbb 100644 --- a/api/src/main/java/io/druid/timeline/partition/ShardSpec.java +++ b/api/src/main/java/io/druid/timeline/partition/ShardSpec.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; import io.druid.data.input.InputRow; import java.util.List; @@ -50,5 +50,5 @@ public interface ShardSpec * * @return map of dimensions to its possible range. Dimensions with unknown possible range are not mapped */ - Map> getDomain(); + Map> getDomain(); } diff --git a/api/src/test/java/io/druid/timeline/DataSegmentTest.java b/api/src/test/java/io/druid/timeline/DataSegmentTest.java index bda7947dd2b2..f2743d24b0a2 100644 --- a/api/src/test/java/io/druid/timeline/DataSegmentTest.java +++ b/api/src/test/java/io/druid/timeline/DataSegmentTest.java @@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; import com.google.common.collect.Sets; import io.druid.TestObjectMapper; import io.druid.data.input.InputRow; @@ -82,7 +82,7 @@ public ShardSpecLookup getLookup(List shardSpecs) } @Override - public Map> getDomain() + public Map> getDomain() { return ImmutableMap.of(); } diff --git a/processing/src/main/java/io/druid/query/filter/DimFilterUtils.java b/processing/src/main/java/io/druid/query/filter/DimFilterUtils.java index 42ee9eeba77a..25c991662ed2 100644 --- a/processing/src/main/java/io/druid/query/filter/DimFilterUtils.java +++ b/processing/src/main/java/io/druid/query/filter/DimFilterUtils.java @@ -122,12 +122,13 @@ public static Set filterShards(DimFilter dimFilter, Iterable input, Fu boolean include = true; if (dimFilter != null && shard != null) { - Map> domain = shard.getDomain(); - for (Map.Entry> entry : domain.entrySet()) { + Map> domain = shard.getDomain(); + for (Map.Entry> entry : domain.entrySet()) { String dimension = entry.getKey(); Optional> optFilterRangeSet = dimensionRangeCache .computeIfAbsent(dimension, d -> Optional.fromNullable(dimFilter.getDimensionRangeSet(d))); - if (optFilterRangeSet.isPresent() && optFilterRangeSet.get().subRangeSet(entry.getValue()).isEmpty()) { + + if (optFilterRangeSet.isPresent() && hasEmptyIntersection(optFilterRangeSet.get(), entry.getValue())) { include = false; } } @@ -139,4 +140,15 @@ public static Set filterShards(DimFilter dimFilter, Iterable input, Fu } return retSet; } + + private static boolean hasEmptyIntersection(RangeSet r1, RangeSet r2) + { + for (Range range : r2.asRanges()) { + if (!r1.subRangeSet(range).isEmpty()) { + return false; + } + } + + return true; + } } diff --git a/processing/src/test/java/io/druid/query/filter/DimFilterUtilsTest.java b/processing/src/test/java/io/druid/query/filter/DimFilterUtilsTest.java index 421aa47bae3c..480062510eaf 100644 --- a/processing/src/test/java/io/druid/query/filter/DimFilterUtilsTest.java +++ b/processing/src/test/java/io/druid/query/filter/DimFilterUtilsTest.java @@ -29,6 +29,7 @@ import com.google.common.collect.Range; import com.google.common.collect.RangeSet; import com.google.common.collect.Sets; +import com.google.common.collect.TreeRangeSet; import io.druid.timeline.partition.ShardSpec; import org.easymock.EasyMock; import org.junit.Assert; @@ -113,8 +114,10 @@ private static RangeSet rangeSet(List> ranges) private static ShardSpec shardSpec(String dimension, Range range) { ShardSpec shard = EasyMock.createMock(ShardSpec.class); + RangeSet rangeSet = TreeRangeSet.create(); + rangeSet.add(range); EasyMock.expect(shard.getDomain()) - .andReturn(ImmutableMap.of(dimension, range)) + .andReturn(ImmutableMap.of(dimension, rangeSet)) .anyTimes(); return shard; } diff --git a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java index b06ba203ff94..2f1d32e7964a 100644 --- a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java @@ -25,12 +25,11 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import io.druid.data.input.InputRow; @@ -90,14 +89,10 @@ List getGroupKey(final long timestamp, final InputRow inputRow) if (partitionDimensions.isEmpty()) { return Rows.toGroupKey(timestamp, inputRow); } else { - return Lists.transform(partitionDimensions, new Function() - { - @Override - public Object apply(final String dim) - { - return inputRow.getDimension(dim); - } - }); + return Lists.transform( + partitionDimensions, + dim -> inputRow.getDimension(dim) + ); } } @@ -114,19 +109,14 @@ public String toString() @Override public ShardSpecLookup getLookup(final List shardSpecs) { - return new ShardSpecLookup() - { - @Override - public ShardSpec getShardSpec(long timestamp, InputRow row) - { - int index = Math.abs(hash(timestamp, row) % getPartitions()); - return shardSpecs.get(index); - } + return (long timestamp, InputRow row) -> { + int index = Math.abs(hash(timestamp, row) % getPartitions()); + return shardSpecs.get(index); }; } @Override - public Map> getDomain() + public Map> getDomain() { return ImmutableMap.of(); } diff --git a/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java b/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java index 000f035921d8..b2358eb707db 100644 --- a/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; import io.druid.data.input.InputRow; import java.util.List; @@ -54,18 +54,11 @@ public int getPartitionNum() @Override public ShardSpecLookup getLookup(final List shardSpecs) { - return new ShardSpecLookup() - { - @Override - public ShardSpec getShardSpec(long timestamp, InputRow row) - { - return shardSpecs.get(0); - } - }; + return (long timestamp, InputRow row) -> shardSpecs.get(0); } @Override - public Map> getDomain() + public Map> getDomain() { return ImmutableMap.of(); } diff --git a/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java index 673bed3cf6a5..ea955e31f8a3 100644 --- a/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; import io.druid.data.input.InputRow; import java.util.List; @@ -67,18 +67,11 @@ public int getPartitionNum() @Override public ShardSpecLookup getLookup(final List shardSpecs) { - return new ShardSpecLookup() - { - @Override - public ShardSpec getShardSpec(long timestamp, InputRow row) - { - return shardSpecs.get(0); - } - }; + return (long timestamp, InputRow row) -> shardSpecs.get(0); } @Override - public Map> getDomain() + public Map> getDomain() { return ImmutableMap.of(); } diff --git a/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java b/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java index 8cbefe40f8a1..aa5527429f71 100644 --- a/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java @@ -23,6 +23,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; +import com.google.common.collect.TreeRangeSet; import io.druid.data.input.InputRow; import io.druid.java.util.common.ISE; @@ -102,35 +104,30 @@ public int getPartitionNum() @Override public ShardSpecLookup getLookup(final List shardSpecs) { - return new ShardSpecLookup() - { - @Override - public ShardSpec getShardSpec(long timestamp, InputRow row) - { - for (ShardSpec spec : shardSpecs) { - if (spec.isInChunk(timestamp, row)) { - return spec; - } + return (long timestamp, InputRow row) -> { + for (ShardSpec spec : shardSpecs) { + if (spec.isInChunk(timestamp, row)) { + return spec; } - throw new ISE("row[%s] doesn't fit in any shard[%s]", row, shardSpecs); } + throw new ISE("row[%s] doesn't fit in any shard[%s]", row, shardSpecs); }; } @Override - public Map> getDomain() + public Map> getDomain() { - Range range; + RangeSet rangeSet = TreeRangeSet.create(); if (start == null && end == null) { - range = Range.all(); + rangeSet.add(Range.all()); } else if (start == null) { - range = Range.atMost(end); + rangeSet.add(Range.atMost(end)); } else if (end == null) { - range = Range.atLeast(start); + rangeSet.add(Range.atLeast(start)); } else { - range = Range.closed(start, end); + rangeSet.add(Range.closed(start, end)); } - return ImmutableMap.of(dimension, range); + return ImmutableMap.of(dimension, rangeSet); } public void setPartitionNum(int partitionNum) From 83afb73afd8f4f99446c2739679c4e8b78331ca5 Mon Sep 17 00:00:00 2001 From: Senthil Kumar L S Date: Fri, 6 Apr 2018 11:26:59 +0530 Subject: [PATCH 06/31] Adding feature thetaSketchConstant to do some set operation in PostAgg (#5551) * Adding feature thetaSketchConstant to do some set operation in PostAggregator * Updated review comments for PR #5551 - Adding thetaSketchConstant * Fixed CI build issue * Updated review comments 2 for PR #5551 - Adding thetaSketchConstant --- .../theta/SketchConstantPostAggregator.java | 133 ++++++++++++++++++ .../datasketches/theta/SketchHolder.java | 14 +- .../datasketches/theta/SketchModule.java | 5 +- .../theta/SketchAggregationTest.java | 20 +++ .../SketchAggregationWithSimpleDataTest.java | 45 ++++++ .../resources/topn_query_sketch_const.json | 104 ++++++++++++++ .../post/ConstantPostAggregator.java | 3 +- .../aggregation/post/PostAggregatorIds.java | 1 + 8 files changed, 321 insertions(+), 4 deletions(-) create mode 100644 extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java create mode 100644 extensions-core/datasketches/src/test/resources/topn_query_sketch_const.json diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java new file mode 100644 index 000000000000..510cb78b80bd --- /dev/null +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java @@ -0,0 +1,133 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.aggregation.datasketches.theta; + +import java.util.Collections; +import java.util.Comparator; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.codec.digest.DigestUtils; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.post.PostAggregatorIds; +import io.druid.query.cache.CacheKeyBuilder; + +/** + */ +public class SketchConstantPostAggregator implements PostAggregator +{ + + private final String name; + private final String value; + private final SketchHolder sketchValue; + + @JsonCreator + public SketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value) + { + this.name = name; + Preconditions.checkArgument(value != null && !value.isEmpty(), + "Constant value cannot be null or empty, expecting base64 encoded sketch string"); + this.value = value; + this.sketchValue = SketchHolder.deserialize(value); + } + + @Override + public Set getDependentFields() + { + return Collections.emptySet(); + } + + @Override + public Comparator getComparator() + { + return SketchHolder.COMPARATOR; + } + + @Override + public Object compute(Map combinedAggregators) + { + return sketchValue; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + public SketchConstantPostAggregator decorate(Map aggregators) + { + return this; + } + + @JsonProperty("value") + public SketchHolder getSketchValue() + { + return sketchValue; + } + + @Override + public String toString() + { + return "SketchConstantPostAggregator{name='" + name + "', value='" + value + "'}"; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SketchConstantPostAggregator that = (SketchConstantPostAggregator) o; + if (!this.sketchValue.equals(that.sketchValue)) { + return false; + } + if (name != null ? !name.equals(that.name) : that.name != null) { + return false; + } + return true; + } + + @Override + public int hashCode() + { + int result = name != null ? name.hashCode() : 0; + result = 37 * result + sketchValue.hashCode(); + return result; + } + + @Override + public byte[] getCacheKey() + { + return new CacheKeyBuilder(PostAggregatorIds.THETA_SKETCH_CONSTANT) + .appendString(DigestUtils.sha1Hex(value)).build(); + } +} diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchHolder.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchHolder.java index cf5f1a3fb130..c273bd4a76aa 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchHolder.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchHolder.java @@ -36,6 +36,7 @@ import io.druid.java.util.common.StringUtils; import org.apache.commons.codec.binary.Base64; +import java.util.Arrays; import java.util.Comparator; /** @@ -286,6 +287,11 @@ public static SketchHolder sketchSetOperation(Func func, int sketchSize, Object. } } + /** + * Ideally make use of Sketch's equals and hashCode methods but which are not value based implementations. + * And yet need value based equals and hashCode implementations for SketchHolder. + * Hence using Arrays.equals() and Arrays.hashCode(). + */ @Override public boolean equals(Object o) { @@ -295,6 +301,12 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - return this.getSketch().equals(((SketchHolder) o).getSketch()); + return Arrays.equals(this.getSketch().toByteArray(), ((SketchHolder) o).getSketch().toByteArray()); + } + + @Override + public int hashCode() + { + return 31 * Arrays.hashCode(this.getSketch().toByteArray()); } } diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchModule.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchModule.java index 361dd4a2fbf9..18d699ec6c52 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchModule.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchModule.java @@ -38,6 +38,8 @@ public class SketchModule implements DruidModule public static final String THETA_SKETCH_ESTIMATE_POST_AGG = "thetaSketchEstimate"; public static final String THETA_SKETCH_SET_OP_POST_AGG = "thetaSketchSetOp"; + + public static final String THETA_SKETCH_CONSTANT_POST_AGG = "thetaSketchConstant"; @Override public void configure(Binder binder) @@ -63,7 +65,8 @@ public List getJacksonModules() .registerSubtypes( new NamedType(SketchMergeAggregatorFactory.class, THETA_SKETCH), new NamedType(SketchEstimatePostAggregator.class, THETA_SKETCH_ESTIMATE_POST_AGG), - new NamedType(SketchSetPostAggregator.class, THETA_SKETCH_SET_OP_POST_AGG) + new NamedType(SketchSetPostAggregator.class, THETA_SKETCH_SET_OP_POST_AGG), + new NamedType(SketchConstantPostAggregator.class, THETA_SKETCH_CONSTANT_POST_AGG) ) .addSerializer( SketchHolder.class, new SketchHolderJsonSerializer() diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java index 260dc12b3507..b5eb3cb0f6c7 100644 --- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java +++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java @@ -277,6 +277,14 @@ public void testSketchEstimatePostAggregatorSerde() throws Exception 2 ) ); + + assertPostAggregatorSerde( + new SketchEstimatePostAggregator( + "name", + new SketchConstantPostAggregator("name", "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI="), + null + ) + ); } @Test @@ -293,6 +301,18 @@ public void testSketchSetPostAggregatorSerde() throws Exception ) ) ); + + assertPostAggregatorSerde( + new SketchSetPostAggregator( + "name", + "INTERSECT", + null, + Lists.newArrayList( + new FieldAccessPostAggregator("name1", "fieldName1"), + new SketchConstantPostAggregator("name2", "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=") + ) + ) + ); } @Test diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java index 870fa9794b0c..d38166b4b915 100644 --- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java +++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java @@ -276,6 +276,51 @@ public void testSimpleDataIngestAndSelectQuery() throws Exception Assert.assertEquals(100, result.getValue().getEvents().size()); Assert.assertEquals("AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=", result.getValue().getEvents().get(0).getEvent().get("pty_country")); } + + @Test + public void testTopNQueryWithSketchConstant() throws Exception + { + AggregationTestHelper topNQueryAggregationTestHelper = AggregationTestHelper.createTopNQueryAggregationTestHelper( + sm.getJacksonModules(), + tempFolder + ); + + Sequence seq = topNQueryAggregationTestHelper.runQueryOnSegments( + ImmutableList.of(s1, s2), + readFileFromClasspathAsString("topn_query_sketch_const.json") + ); + + Result result = (Result) Iterables.getOnlyElement(seq.toList()); + + Assert.assertEquals(DateTimes.of("2014-10-20T00:00:00.000Z"), result.getTimestamp()); + + DimensionAndMetricValueExtractor value1 = Iterables.get(result.getValue().getValue(), 0); + Assert.assertEquals(38.0, value1.getDoubleMetric("sketch_count"), 0.01); + Assert.assertEquals(38.0, value1.getDoubleMetric("sketchEstimatePostAgg"), 0.01); + Assert.assertEquals(2.0, value1.getDoubleMetric("sketchEstimatePostAggForSketchConstant"), 0.01); + Assert.assertEquals(39.0, value1.getDoubleMetric("sketchUnionPostAggEstimate"), 0.01); + Assert.assertEquals(1.0, value1.getDoubleMetric("sketchIntersectionPostAggEstimate"), 0.01); + Assert.assertEquals(37.0, value1.getDoubleMetric("sketchAnotBPostAggEstimate"), 0.01); + Assert.assertEquals("product_3", value1.getDimensionValue("product")); + + DimensionAndMetricValueExtractor value2 = Iterables.get(result.getValue().getValue(), 1); + Assert.assertEquals(42.0, value2.getDoubleMetric("sketch_count"), 0.01); + Assert.assertEquals(42.0, value2.getDoubleMetric("sketchEstimatePostAgg"), 0.01); + Assert.assertEquals(2.0, value2.getDoubleMetric("sketchEstimatePostAggForSketchConstant"), 0.01); + Assert.assertEquals(42.0, value2.getDoubleMetric("sketchUnionPostAggEstimate"), 0.01); + Assert.assertEquals(2.0, value2.getDoubleMetric("sketchIntersectionPostAggEstimate"), 0.01); + Assert.assertEquals(40.0, value2.getDoubleMetric("sketchAnotBPostAggEstimate"), 0.01); + Assert.assertEquals("product_1", value2.getDimensionValue("product")); + + DimensionAndMetricValueExtractor value3 = Iterables.get(result.getValue().getValue(), 2); + Assert.assertEquals(42.0, value3.getDoubleMetric("sketch_count"), 0.01); + Assert.assertEquals(42.0, value3.getDoubleMetric("sketchEstimatePostAgg"), 0.01); + Assert.assertEquals(2.0, value3.getDoubleMetric("sketchEstimatePostAggForSketchConstant"), 0.01); + Assert.assertEquals(42.0, value3.getDoubleMetric("sketchUnionPostAggEstimate"), 0.01); + Assert.assertEquals(2.0, value3.getDoubleMetric("sketchIntersectionPostAggEstimate"), 0.01); + Assert.assertEquals(40.0, value3.getDoubleMetric("sketchAnotBPostAggEstimate"), 0.01); + Assert.assertEquals("product_2", value3.getDimensionValue("product")); + } public static final String readFileFromClasspathAsString(String fileName) throws IOException { diff --git a/extensions-core/datasketches/src/test/resources/topn_query_sketch_const.json b/extensions-core/datasketches/src/test/resources/topn_query_sketch_const.json new file mode 100644 index 000000000000..3dc47dca86c3 --- /dev/null +++ b/extensions-core/datasketches/src/test/resources/topn_query_sketch_const.json @@ -0,0 +1,104 @@ +{ + "queryType": "topN", + "dataSource": "test_datasource", + "granularity":"ALL", + "metric": { + "type": "inverted", + "metric": "sketch_count" + }, + "dimension": "product", + "threshold": 3, + "aggregations": [ + { + "type": "thetaSketch", + "name": "sketch_count", + "fieldName": "pty_country", + "size": 16384 + } + ], + "postAggregations": [ + { + "type": "thetaSketchEstimate", + "name": "sketchEstimatePostAgg", + "field": { + "type": "fieldAccess", + "fieldName": "sketch_count" + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchEstimatePostAggForSketchConstant", + "field": { + "type": "thetaSketchConstant", + "name": "theta_sketch_count", + "value": "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=" + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchIntersectionPostAggEstimate", + "field": { + "type": "thetaSketchSetOp", + "name": "sketchIntersectionPostAgg", + "func": "INTERSECT", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sketch_count" + }, + { + "type": "thetaSketchConstant", + "name": "theta_sketch_count", + "value": "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=" + } + ] + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchAnotBPostAggEstimate", + "field": { + "type": "thetaSketchSetOp", + "name": "sketchAnotBUnionPostAgg", + "func": "NOT", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sketch_count" + }, + { + "type": "thetaSketchConstant", + "name": "theta_sketch_count", + "value": "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=" + } + ] + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchUnionPostAggEstimate", + "field": { + "type": "thetaSketchSetOp", + "name": "sketchUnionPostAgg", + "func": "UNION", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sketch_count" + }, + { + "type": "thetaSketchConstant", + "name": "theta_sketch_count", + "value": "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=" + } + ] + } + } + ], + "intervals": [ + "2014-10-19T00:00:00.000Z/2014-10-22T00:00:00.000Z" + ] +} diff --git a/processing/src/main/java/io/druid/query/aggregation/post/ConstantPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/post/ConstantPostAggregator.java index b406b83a0fe4..80e665b9d833 100644 --- a/processing/src/main/java/io/druid/query/aggregation/post/ConstantPostAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/post/ConstantPostAggregator.java @@ -46,8 +46,7 @@ public ConstantPostAggregator( ) { this.name = name; - this.constantValue = constantValue; - Preconditions.checkNotNull(this.constantValue); + this.constantValue = Preconditions.checkNotNull(constantValue, "Constant value cannot be null"); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/post/PostAggregatorIds.java b/processing/src/main/java/io/druid/query/aggregation/post/PostAggregatorIds.java index bfb7d4df108e..3ef660c2cbe4 100644 --- a/processing/src/main/java/io/druid/query/aggregation/post/PostAggregatorIds.java +++ b/processing/src/main/java/io/druid/query/aggregation/post/PostAggregatorIds.java @@ -44,4 +44,5 @@ public class PostAggregatorIds public static final byte FINALIZING_FIELD_ACCESS = 20; public static final byte ZTEST = 21; public static final byte PVALUE_FROM_ZTEST = 22; + public static final byte THETA_SKETCH_CONSTANT = 23; } From 10dc1504413f15aa9cd4be6c9df0855a2f0894d9 Mon Sep 17 00:00:00 2001 From: Dylan Wylie Date: Fri, 6 Apr 2018 07:52:58 +0100 Subject: [PATCH 07/31] Fix taskDuration docs for KafkaIndexingService (#5572) * With incremental handoff the changed line is no longer true. --- docs/content/development/extensions-core/kafka-ingestion.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index e0d7f73c689d..2c09b64e15f0 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -164,7 +164,7 @@ For Roaring bitmaps: |`consumerProperties`|Map|A map of properties to be passed to the Kafka consumer. This must contain a property `bootstrap.servers` with a list of Kafka brokers in the form: `:,:,...`.|yes| |`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against node failure.|no (default == 1)| |`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See 'Capacity Planning' below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKafkaPartitions}`.|no (default == 1)| -|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment. Note that segments are only pushed to deep storage and loadable by historical nodes when the indexing task completes.|no (default == PT1H)| +|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment.|no (default == PT1H)| |`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)| |`period`|ISO8601 Period|How often the supervisor will execute its management logic. Note that the supervisor will also run in response to certain events (such as tasks succeeding, failing, and reaching their taskDuration) so this value specifies the maximum time between iterations.|no (default == PT30S)| |`useEarliestOffset`|Boolean|If a supervisor is managing a dataSource for the first time, it will obtain a set of starting offsets from Kafka. This flag determines whether it retrieves the earliest or latest offsets in Kafka. Under normal circumstances, subsequent tasks will start from where the previous segments ended so this flag will only be used on first run.|no (default == false)| From ea6b3471cd7af6aca18457b559c60afa0a1ad90a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 5 Apr 2018 23:53:43 -0700 Subject: [PATCH 08/31] Add doc for automatic pendingSegments (#5565) * Add missing doc for automatic pendingSegments * address comments --- docs/content/configuration/coordinator.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/content/configuration/coordinator.md b/docs/content/configuration/coordinator.md index d571dadcd3b0..9e4f38cf44e9 100644 --- a/docs/content/configuration/coordinator.md +++ b/docs/content/configuration/coordinator.md @@ -29,6 +29,7 @@ The coordinator node uses several of the global configs in [Configuration](../co |`druid.coordinator.merge.on`|Boolean flag for whether or not the coordinator should try and merge small segments into a more optimal segment size.|false| |`druid.coordinator.conversion.on`|Boolean flag for converting old segment indexing versions to the latest segment indexing version.|false| |`druid.coordinator.load.timeout`|The timeout duration for when the coordinator assigns a segment to a historical node.|PT15M| +|`druid.coordinator.kill.pendingSegments.on`|Boolean flag for whether or not the coordinator clean up old entries in the `pendingSegments` table of metadata store. If set to true, coordinator will check the created time of most recently complete task. If it doesn't exist, it finds the created time of the earlist running/pending/waiting tasks. Once the created time is found, then for all dataSources not in the `killPendingSegmentsSkipList` (see [Dynamic configuration](#dynamic-configuration)), coordinator will ask the overlord to clean up the entries 1 day or more older than the found created time in the `pendingSegments` table. This will be done periodically based on `druid.coordinator.period` specified.|false| |`druid.coordinator.kill.on`|Boolean flag for whether or not the coordinator should submit kill task for unused segments, that is, hard delete them from metadata store and deep storage. If set to true, then for all whitelisted dataSources (or optionally all), coordinator will submit tasks periodically based on `period` specified. These kill tasks will delete all segments except for the last `durationToRetain` period. Whitelist or All can be set via dynamic configuration `killAllDataSources` and `killDataSourceWhitelist` described later.|false| |`druid.coordinator.kill.period`|How often to send kill tasks to the indexing service. Value must be greater than `druid.coordinator.period.indexingPeriod`. Only applies if kill is turned on.|P1D (1 Day)| |`druid.coordinator.kill.durationToRetain`| Do not kill segments in last `durationToRetain`, must be greater or equal to 0. Only applies and MUST be specified if kill is turned on. Note that default value is invalid.|PT-1S (-1 seconds)| @@ -103,8 +104,9 @@ Issuing a GET request at the same URL will return the spec that is currently in |`replicantLifetime`|The maximum number of coordinator runs for a segment to be replicated before we start alerting.|15| |`replicationThrottleLimit`|The maximum number of segments that can be replicated at one time.|10| |`emitBalancingStats`|Boolean flag for whether or not we should emit balancing stats. This is an expensive operation.|false| -|`killDataSourceWhitelist`|List of dataSources for which kill tasks are sent if property `druid.coordinator.kill.on` is true.|none| +|`killDataSourceWhitelist`|List of dataSources for which kill tasks are sent if property `druid.coordinator.kill.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none| |`killAllDataSources`|Send kill tasks for ALL dataSources if property `druid.coordinator.kill.on` is true. If this is set to true then `killDataSourceWhitelist` must not be specified or be empty list.|false| +|`killPendingSegmentsSkipList`|List of dataSources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none| |`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of nodes. Value 1000 could be a start point for a rather big cluster. Default value is 0 (loading queue is unbounded) |0| To view the audit history of coordinator dynamic config issue a GET request to the URL - From 2bbc6d659c495fc9775ff62c14a463fd5896b8e3 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 5 Apr 2018 23:54:59 -0700 Subject: [PATCH 09/31] Fix indexTask to respect forceExtendableShardSpecs (#5509) * Fix indexTask to respect forceExtendableShardSpecs * add comments --- .../druid/indexing/common/task/IndexTask.java | 24 ++++++++++++------- .../indexing/common/task/IndexTaskTest.java | 2 +- 2 files changed, 16 insertions(+), 10 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 0b9f4c056994..f293d20d72cc 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -524,16 +524,15 @@ private static String findVersion(Map versions, Interval inter private static boolean isGuaranteedRollup(IndexIOConfig ioConfig, IndexTuningConfig tuningConfig) { Preconditions.checkState( - !(tuningConfig.isForceGuaranteedRollup() && - (tuningConfig.isForceExtendableShardSpecs() || ioConfig.isAppendToExisting())), - "Perfect rollup cannot be guaranteed with extendable shardSpecs" + !tuningConfig.isForceGuaranteedRollup() || !ioConfig.isAppendToExisting(), + "Perfect rollup cannot be guaranteed when appending to existing dataSources" ); return tuningConfig.isForceGuaranteedRollup(); } private static boolean isExtendableShardSpecs(IndexIOConfig ioConfig, IndexTuningConfig tuningConfig) { - return !isGuaranteedRollup(ioConfig, tuningConfig); + return tuningConfig.isForceExtendableShardSpecs() || ioConfig.isAppendToExisting(); } /** @@ -1251,7 +1250,19 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi private final IndexSpec indexSpec; private final File basePersistDirectory; private final int maxPendingPersists; + + /** + * This flag is to force to always use an extendableShardSpec (like {@link NumberedShardSpec} even if + * {@link #forceGuaranteedRollup} is set. + */ private final boolean forceExtendableShardSpecs; + + /** + * This flag is to force _perfect rollup mode_. {@link IndexTask} will scan the whole input data twice to 1) figure + * out proper shard specs for each segment and 2) generate segments. Note that perfect rollup mode basically assumes + * that no more data will be appended in the future. As a result, in perfect rollup mode, {@link NoneShardSpec} and + * {@link HashBasedNumberedShardSpec} are used for a single shard and two or shards, respectively. + */ private final boolean forceGuaranteedRollup; private final boolean reportParseExceptions; private final long pushTimeout; @@ -1351,11 +1362,6 @@ private IndexTuningConfig( this.pushTimeout = pushTimeout == null ? DEFAULT_PUSH_TIMEOUT : pushTimeout; this.basePersistDirectory = basePersistDirectory; - Preconditions.checkArgument( - !(this.forceExtendableShardSpecs && this.forceGuaranteedRollup), - "Perfect rollup cannot be guaranteed with extendable shardSpecs" - ); - this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; if (this.reportParseExceptions) { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 28cbf664da8a..dbd9af3279ac 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -222,7 +222,7 @@ public void testForceExtendableShardSpecs() throws Exception tmpDir, null, null, - createTuningConfig(2, null, true, false), + createTuningConfig(2, null, true, true), false ), null, From e9906e859012cfe853169565ec0fae5ce8be58d3 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 6 Apr 2018 05:37:16 -0700 Subject: [PATCH 10/31] Deprecate spark2 profile in pom.xml (#5581) Deprecated due to https://github.com/druid-io/druid/pull/5382 --- pom.xml | 7 ------- 1 file changed, 7 deletions(-) diff --git a/pom.xml b/pom.xml index 1830c8b7c7f8..dc401857aa22 100644 --- a/pom.xml +++ b/pom.xml @@ -1127,13 +1127,6 @@ - - spark2 - - 2.6.5 - 1.11.143 - - strict From 99315da49c7f64f6828351952681d622dffd8e55 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 6 Apr 2018 08:06:45 -0700 Subject: [PATCH 11/31] CompressionUtils: Add support for decompressing xz, bz2, zip. (#5586) Also switch various firehoses to the new method. Fixes #5585. --- .../StaticAzureBlobStoreFirehoseFactory.java | 2 +- .../StaticCloudFilesFirehoseFactory.java | 2 +- .../StaticGoogleBlobStoreFirehoseFactory.java | 2 +- .../lookup/namespace/UriCacheGenerator.java | 28 +++------- .../firehose/s3/StaticS3FirehoseFactory.java | 18 +++--- java-util/pom.xml | 8 +++ .../java/util/common/CompressionUtils.java | 46 +++++++++++++++- .../util/common/CompressionUtilsTest.java | 55 +++++++++++++++++-- pom.xml | 10 ++++ .../firehose/HttpFirehoseFactory.java | 2 +- .../firehose/LocalFirehoseFactory.java | 4 +- 11 files changed, 136 insertions(+), 41 deletions(-) diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java index e3ad8b432aa7..345cffd512cf 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java @@ -91,7 +91,7 @@ protected InputStream openObjectStream(AzureBlob object, long start) throws IOEx @Override protected InputStream wrapObjectStream(AzureBlob object, InputStream stream) throws IOException { - return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + return CompressionUtils.decompress(stream, object.getPath()); } private static AzureByteSource makeByteSource(AzureStorage azureStorage, AzureBlob object) diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java index 5f39e7e5a446..343635c4680e 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java @@ -101,7 +101,7 @@ private CloudFilesByteSource createCloudFilesByteSource(CloudFilesBlob object) @Override protected InputStream wrapObjectStream(CloudFilesBlob object, InputStream stream) throws IOException { - return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + return CompressionUtils.decompress(stream, object.getPath()); } @Override diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java index 0d5d99959609..38fb83870883 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java @@ -93,7 +93,7 @@ private GoogleByteSource createGoogleByteSource(GoogleBlob object) @Override protected InputStream wrapObjectStream(GoogleBlob object, InputStream stream) throws IOException { - return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + return CompressionUtils.decompress(stream, object.getPath()); } @Override diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/UriCacheGenerator.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/UriCacheGenerator.java index 27fd2eacbdfd..c2ca336c2eca 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/UriCacheGenerator.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/UriCacheGenerator.java @@ -134,28 +134,14 @@ public CacheScheduler.VersionedCache generateCache( catch (NumberFormatException ex) { log.debug(ex, "Failed to get last modified timestamp. Assuming no timestamp"); } - final ByteSource source; - if (CompressionUtils.isGz(uriPath)) { - // Simple gzip stream - log.debug("Loading gz"); - source = new ByteSource() + final ByteSource source = new ByteSource() + { + @Override + public InputStream openStream() throws IOException { - @Override - public InputStream openStream() throws IOException - { - return CompressionUtils.gzipInputStream(puller.getInputStream(uri)); - } - }; - } else { - source = new ByteSource() - { - @Override - public InputStream openStream() throws IOException - { - return puller.getInputStream(uri); - } - }; - } + return CompressionUtils.decompress(puller.getInputStream(uri), uri.getPath()); + } + }; final CacheScheduler.VersionedCache versionedCache = scheduler.createVersionedCache(entryId, version); try { diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index 8827fc9ae31d..8c41d6f1f129 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -114,14 +114,14 @@ protected Collection initObjects() throws IOException // Getting data is deferred until openObjectStream() is called for each object. if (!uris.isEmpty()) { return uris.stream() - .map( - uri -> { - final String s3Bucket = uri.getAuthority(); - final String key = S3Utils.extractS3Key(uri); - return S3Utils.getSingleObjectSummary(s3Client, s3Bucket, key); - } - ) - .collect(Collectors.toList()); + .map( + uri -> { + final String s3Bucket = uri.getAuthority(); + final String key = S3Utils.extractS3Key(uri); + return S3Utils.getSingleObjectSummary(s3Client, s3Bucket, key); + } + ) + .collect(Collectors.toList()); } else { final List objects = new ArrayList<>(); for (URI uri : prefixes) { @@ -212,7 +212,7 @@ protected InputStream openObjectStream(S3ObjectSummary object, long start) throw @Override protected InputStream wrapObjectStream(S3ObjectSummary object, InputStream stream) throws IOException { - return object.getKey().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + return CompressionUtils.decompress(stream, object.getKey()); } @Override diff --git a/java-util/pom.xml b/java-util/pom.xml index 150c332a3ca7..7f0b462d9c57 100644 --- a/java-util/pom.xml +++ b/java-util/pom.xml @@ -81,6 +81,14 @@ org.mozilla rhino + + org.apache.commons + commons-compress + + + org.tukaani + xz + com.jayway.jsonpath json-path diff --git a/java-util/src/main/java/io/druid/java/util/common/CompressionUtils.java b/java-util/src/main/java/io/druid/java/util/common/CompressionUtils.java index 876f26f2f589..c076ea6e8def 100644 --- a/java-util/src/main/java/io/druid/java/util/common/CompressionUtils.java +++ b/java-util/src/main/java/io/druid/java/util/common/CompressionUtils.java @@ -28,14 +28,18 @@ import com.google.common.io.Files; import io.druid.java.util.common.io.NativeIO; import io.druid.java.util.common.logger.Logger; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; +import org.apache.commons.compress.compressors.xz.XZCompressorInputStream; import java.io.BufferedInputStream; +import java.io.ByteArrayInputStream; import java.io.File; import java.io.FileOutputStream; import java.io.FilterInputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.charset.StandardCharsets; import java.util.Enumeration; import java.util.zip.GZIPInputStream; import java.util.zip.GZIPOutputStream; @@ -48,7 +52,9 @@ public class CompressionUtils { private static final Logger log = new Logger(CompressionUtils.class); private static final int DEFAULT_RETRY_COUNT = 3; + private static final String BZ2_SUFFIX = ".bz2"; private static final String GZ_SUFFIX = ".gz"; + private static final String XZ_SUFFIX = ".xz"; private static final String ZIP_SUFFIX = ".zip"; /** @@ -313,7 +319,7 @@ public static FileUtils.FileCopyResult gunzip(InputStream in, File outFile) thro * * @return A GZIPInputStream that can handle concatenated gzip streams in the input */ - public static GZIPInputStream gzipInputStream(final InputStream in) throws IOException + private static GZIPInputStream gzipInputStream(final InputStream in) throws IOException { return new GZIPInputStream( new FilterInputStream(in) @@ -516,4 +522,42 @@ public static String getGzBaseName(String fname) } throw new IAE("[%s] is not a valid gz file name", fname); } + + /** + * Decompress an input stream from a file, based on the filename. + */ + public static InputStream decompress(final InputStream in, final String fileName) throws IOException + { + if (fileName.endsWith(GZ_SUFFIX)) { + return gzipInputStream(in); + } else if (fileName.endsWith(BZ2_SUFFIX)) { + return new BZip2CompressorInputStream(in, true); + } else if (fileName.endsWith(XZ_SUFFIX)) { + return new XZCompressorInputStream(in, true); + } else if (fileName.endsWith(ZIP_SUFFIX)) { + // This reads the first file in the archive. + final ZipInputStream zipIn = new ZipInputStream(in, StandardCharsets.UTF_8); + try { + final ZipEntry nextEntry = zipIn.getNextEntry(); + if (nextEntry == null) { + zipIn.close(); + + // No files in the archive - return an empty stream. + return new ByteArrayInputStream(new byte[0]); + } + return zipIn; + } + catch (IOException e) { + try { + zipIn.close(); + } + catch (IOException e2) { + e.addSuppressed(e2); + } + throw e; + } + } else { + return in; + } + } } diff --git a/java-util/src/test/java/io/druid/java/util/common/CompressionUtilsTest.java b/java-util/src/test/java/io/druid/java/util/common/CompressionUtilsTest.java index c00e75b92249..d8f878b9ed70 100644 --- a/java-util/src/test/java/io/druid/java/util/common/CompressionUtilsTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/CompressionUtilsTest.java @@ -25,6 +25,8 @@ import com.google.common.io.ByteSource; import com.google.common.io.ByteStreams; import com.google.common.io.Files; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; +import org.apache.commons.compress.compressors.xz.XZCompressorOutputStream; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -53,6 +55,8 @@ import java.util.regex.Pattern; import java.util.zip.GZIPInputStream; import java.util.zip.GZIPOutputStream; +import java.util.zip.ZipEntry; +import java.util.zip.ZipOutputStream; public class CompressionUtilsTest { @@ -221,7 +225,6 @@ public void testGoodZipStream() throws IOException } } - @Test public void testGoodGzipByteSource() throws IOException { @@ -230,7 +233,7 @@ public void testGoodGzipByteSource() throws IOException Assert.assertFalse(gzFile.exists()); CompressionUtils.gzip(Files.asByteSource(testFile), Files.asByteSink(gzFile), Predicates.alwaysTrue()); Assert.assertTrue(gzFile.exists()); - try (final InputStream inputStream = CompressionUtils.gzipInputStream(new FileInputStream(gzFile))) { + try (final InputStream inputStream = CompressionUtils.decompress(new FileInputStream(gzFile), gzFile.getName())) { assertGoodDataStream(inputStream); } if (!testFile.delete()) { @@ -244,6 +247,50 @@ public void testGoodGzipByteSource() throws IOException } } + @Test + public void testDecompressBzip2() throws IOException + { + final File tmpDir = temporaryFolder.newFolder("testDecompressBzip2"); + final File bzFile = new File(tmpDir, testFile.getName() + ".bz2"); + Assert.assertFalse(bzFile.exists()); + try (final OutputStream out = new BZip2CompressorOutputStream(new FileOutputStream(bzFile))) { + ByteStreams.copy(new FileInputStream(testFile), out); + } + try (final InputStream inputStream = CompressionUtils.decompress(new FileInputStream(bzFile), bzFile.getName())) { + assertGoodDataStream(inputStream); + } + } + + @Test + public void testDecompressXz() throws IOException + { + final File tmpDir = temporaryFolder.newFolder("testDecompressXz"); + final File xzFile = new File(tmpDir, testFile.getName() + ".xz"); + Assert.assertFalse(xzFile.exists()); + try (final OutputStream out = new XZCompressorOutputStream(new FileOutputStream(xzFile))) { + ByteStreams.copy(new FileInputStream(testFile), out); + } + try (final InputStream inputStream = CompressionUtils.decompress(new FileInputStream(xzFile), xzFile.getName())) { + assertGoodDataStream(inputStream); + } + } + + @Test + public void testDecompressZip() throws IOException + { + final File tmpDir = temporaryFolder.newFolder("testDecompressZip"); + final File zipFile = new File(tmpDir, testFile.getName() + ".zip"); + Assert.assertFalse(zipFile.exists()); + try (final ZipOutputStream out = new ZipOutputStream(new FileOutputStream(zipFile))) { + out.putNextEntry(new ZipEntry("cool.file")); + ByteStreams.copy(new FileInputStream(testFile), out); + out.closeEntry(); + } + try (final InputStream inputStream = CompressionUtils.decompress(new FileInputStream(zipFile), zipFile.getName())) { + assertGoodDataStream(inputStream); + } + } + @Test public void testGoodGZStream() throws IOException { @@ -490,7 +537,7 @@ public void flush() throws IOException }, Predicates.alwaysTrue() ); Assert.assertTrue(gzFile.exists()); - try (final InputStream inputStream = CompressionUtils.gzipInputStream(new FileInputStream(gzFile))) { + try (final InputStream inputStream = CompressionUtils.decompress(new FileInputStream(gzFile), "file.gz")) { assertGoodDataStream(inputStream); } if (!testFile.delete()) { @@ -536,7 +583,7 @@ public void testStreamErrorGunzip() throws Exception Assert.assertFalse(gzFile.exists()); CompressionUtils.gzip(Files.asByteSource(testFile), Files.asByteSink(gzFile), Predicates.alwaysTrue()); Assert.assertTrue(gzFile.exists()); - try (final InputStream inputStream = CompressionUtils.gzipInputStream(new FileInputStream(gzFile))) { + try (final InputStream inputStream = CompressionUtils.decompress(new FileInputStream(gzFile), "file.gz")) { assertGoodDataStream(inputStream); } if (testFile.exists() && !testFile.delete()) { diff --git a/pom.xml b/pom.xml index dc401857aa22..8fc12df9adc4 100644 --- a/pom.xml +++ b/pom.xml @@ -325,6 +325,16 @@ rhino 1.7R5 + + org.apache.commons + commons-compress + 1.16 + + + org.tukaani + xz + 1.8 + com.fasterxml.jackson.core jackson-annotations diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/HttpFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/HttpFirehoseFactory.java index aaab6f9dae55..949cb1db7e47 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/HttpFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/HttpFirehoseFactory.java @@ -105,7 +105,7 @@ protected InputStream openObjectStream(URI object, long start) throws IOExceptio @Override protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException { - return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + return CompressionUtils.decompress(stream, object.getPath()); } @Override diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java index 6db1e8c30941..e9e7b40bb730 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java @@ -22,10 +22,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import io.druid.java.util.emitter.EmittingLogger; import io.druid.data.input.impl.AbstractTextFilesFirehoseFactory; import io.druid.data.input.impl.StringInputRowParser; import io.druid.java.util.common.CompressionUtils; +import io.druid.java.util.emitter.EmittingLogger; import org.apache.commons.io.FileUtils; import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.commons.io.filefilter.WildcardFileFilter; @@ -97,6 +97,6 @@ protected InputStream openObjectStream(File object) throws IOException @Override protected InputStream wrapObjectStream(File object, InputStream stream) throws IOException { - return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + return CompressionUtils.decompress(stream, object.getPath()); } } From 1d8d14ed6bd1bb96fdf9dbd07f828c8564eb4b65 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Thu, 5 Apr 2018 11:26:42 -0700 Subject: [PATCH 12/31] This commit introduces a new tuning config called 'maxBytesInMemory' for ingestion tasks Currently a config called 'maxRowsInMemory' is present which affects how much memory gets used for indexing.If this value is not optimal for your JVM heap size, it could lead to OutOfMemoryError sometimes. A lower value will lead to frequent persists which might be bad for query performance and a higher value will limit number of persists but require more jvm heap space and could lead to OOM. 'maxBytesInMemory' is an attempt to solve this problem. It limits the total number of bytes kept in memory before persisting. * The default value is 1/3(Runtime.maxMemory()) * To maintain the current behaviour set 'maxBytesInMemory' to -1 * If both 'maxRowsInMemory' and 'maxBytesInMemory' are present, both of them will be respected i.e. the first one to go above threshold will trigger persist --- .../java/io/druid/segment/incremental/IncrementalIndex.java | 2 +- .../io/druid/segment/incremental/OnheapIncrementalIndex.java | 1 + .../src/main/java/io/druid/segment/realtime/plumber/Sink.java | 2 ++ 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 1c08687476d9..3e9492f300b5 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -699,7 +699,7 @@ TimeAndDimsResult toTimeAndDims(InputRow row) truncated = gran.bucketStart(row.getTimestamp()).getMillis(); } - TimeAndDims timeAndDims = new TimeAndDims(Math.max(truncated, minTimestamp), dims, dimensionDescsList); + TimeAndDims timeAndDims = new TimeAndDims(Math.max(truncated, minTimestamp), dims, dimensionDescsList, dimsKeySize); return new TimeAndDimsResult(timeAndDims, parseExceptionMessages); } diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 7d35bdeccda6..97a791a2023e 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -39,6 +39,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index 8ddd187f1434..f336d4eff3a5 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -178,6 +178,8 @@ public boolean isEmpty() } } + //add methods for byte mem checks + public boolean isWritable() { return writable; From 8c85a65c1227c4b443ee0b5c6250ae0641629cd7 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Fri, 6 Apr 2018 13:12:52 -0700 Subject: [PATCH 13/31] Address code review comments * Fix the coding style according to druid conventions * Add more javadocs * Rename some variables/methods * Other minor issues --- .../indexing/kafka/KafkaTuningConfig.java | 4 +- .../indexing/kafka/KafkaIndexTaskTest.java | 6 +- .../indexing/kafka/KafkaTuningConfigTest.java | 3 +- .../kafka/supervisor/KafkaSupervisorTest.java | 3 +- .../RealtimeAppenderatorTuningConfig.java | 2 +- .../indexing/common/task/TaskSerdeTest.java | 3 +- .../druid/segment/StringDimensionIndexer.java | 9 ++- .../segment/incremental/IncrementalIndex.java | 14 +++- .../incremental/OffheapIncrementalIndex.java | 2 +- .../incremental/OnheapIncrementalIndex.java | 64 ++++++++++++++----- .../appenderator/AppenderatorImpl.java | 2 +- .../druid/segment/realtime/plumber/Sink.java | 3 +- .../appenderator/AppenderatorTest.java | 10 +-- 13 files changed, 87 insertions(+), 38 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index b953da4fcb4b..e61cc979d215 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -281,7 +281,7 @@ public boolean equals(Object o) KafkaTuningConfig that = (KafkaTuningConfig) o; return maxRowsInMemory == that.maxRowsInMemory && maxRowsPerSegment == that.maxRowsPerSegment && - maxBytesInMemory == this.maxBytesInMemory && + maxBytesInMemory == that.maxBytesInMemory && maxPendingPersists == that.maxPendingPersists && reportParseExceptions == that.reportParseExceptions && handoffConditionTimeout == that.handoffConditionTimeout && @@ -324,7 +324,7 @@ public String toString() return "KafkaTuningConfig{" + "maxRowsInMemory=" + maxRowsInMemory + ", maxRowsPerSegment=" + maxRowsPerSegment + - ",maxBytesInMemory=" + maxBytesInMemory + + ", maxBytesInMemory=" + maxBytesInMemory + ", intermediatePersistPeriod=" + intermediatePersistPeriod + ", basePersistDirectory=" + basePersistDirectory + ", maxPendingPersists=" + maxPendingPersists + diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index cf6cb1317ad7..2d0a5588d538 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -1964,7 +1964,8 @@ private KafkaIndexTask createTask( ) { final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( - 1000, null, + 1000, + null, maxRowsPerSegment, new Period("P1Y"), null, @@ -2006,7 +2007,8 @@ private KafkaIndexTask createTask( ) { final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( - 1000, null, + 1000, + null, maxRowsPerSegment, new Period("P1Y"), null, diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java index 38c6537318c8..369396010702 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java @@ -102,7 +102,8 @@ public void testSerdeWithNonDefaults() throws Exception public void testCopyOf() { KafkaTuningConfig original = new KafkaTuningConfig( - 1, null, + 1, + null, 2, new Period("PT3S"), new File("/tmp/xxx"), diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 21c51d5a899a..cbea2344b22e 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -184,7 +184,8 @@ public void setupTest() taskQueue = createMock(TaskQueue.class); tuningConfig = new KafkaSupervisorTuningConfig( - 1000, null, + 1000, + null, 50000, new Period("P1Y"), new File("/test"), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index ca83fb51e59a..77a780579732 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -37,7 +37,7 @@ @JsonTypeName("realtime_appenderator") public class RealtimeAppenderatorTuningConfig implements TuningConfig, AppenderatorConfig { - private static final int defaultMaxRowsInMemory = 75000; + private static final int defaultMaxRowsInMemory = 1000000; private static final int defaultMaxRowsPerSegment = 5_000_000; private static final long defaultMaxBytesInMemory = getDefaultMaxBytesInMemory(); private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 8bea1819b841..b23bc5f6ba6c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -538,7 +538,8 @@ public Plumber findPlumber( ), new RealtimeTuningConfig( - 1, null, + 1, + null, new Period("PT10M"), null, null, diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java index 911153613b89..5649a3f25aca 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -268,11 +268,14 @@ public int[] processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boole @Override public long estimateEncodedKeyComponentSize(int[] key) { - // the size is calculated conservatively, first key may be accounted for twice - long estimatedSize = (key.length * Integer.BYTES); + /** + * string length is being accounted for each time they are referenced, even though they are stored just once + */ + long estimatedSize = key.length * Integer.BYTES; estimatedSize += Arrays.stream(key) .filter(element -> dimLookup.getValue(element) != null) - .mapToLong(element -> dimLookup.getValue(element).length() * Character.BYTES).sum(); + .mapToLong(element -> dimLookup.getValue(element).length() * Character.BYTES) + .sum(); return estimatedSize; } diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 3e9492f300b5..d0e85fad1747 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -250,7 +250,7 @@ public ColumnCapabilities getColumnCapabilities(String columnName) /** * Setting deserializeComplexMetrics to false is necessary for intermediate aggregation such as groupBy that * should not deserialize input columns using ComplexMetricSerde for aggregators that return complex metrics. - *

+ * * Set concurrentEventAdd to true to indicate that adding of input row should be thread-safe (for example, groupBy * where the multiple threads can add concurrently to the IncrementalIndex). * @@ -1178,9 +1178,19 @@ private void setRowIndex(int rowIndex) this.rowIndex = rowIndex; } + /** + * sizeInBytes estimates the size of TimeAndDims key, it takes into account the timestamp(long), + * dims(Object Array) and dimensionDescsList(List). Each of these are calculated as follows: + *

    + *
  • timestamp : Long.BYTES + *
  • dims array : Integer.BYTES * array length + Long.BYTES (dims object) + dimsKeySize(passed via constructor) + *
  • dimensionDescList : Long.BYTES (shared pointer) + *
+ * + * @return long estimated sizeInBytes + */ public long estimateBytesInMemory() { - //timestamp + dims length + dimensionDescsList shared pointer long sizeInBytes = Long.BYTES + Integer.BYTES * dims.length + Long.BYTES + Long.BYTES; sizeInBytes += dimsKeySize; return sizeInBytes; diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java index 220b2de9cb4e..54b772360551 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -145,7 +145,7 @@ protected AddToFactsResult addToFacts( boolean reportParseExceptions, InputRow row, AtomicInteger numEntries, - AtomicLong sizeInBytes, + AtomicLong sizeInBytes, // ignored, added to make abstract class method impl happy TimeAndDims key, ThreadLocal rowContainer, Supplier rowSupplier, diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 97a791a2023e..98daddb3709b 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -57,7 +57,6 @@ public class OnheapIncrementalIndex extends IncrementalIndex * KeyHash + next pointer + key pointer + value pointer + safe extra */ private static final int ROUGH_OVERHEAD_PER_MAP_ENTRY = Long.BYTES * 5 + Integer.BYTES; - private static final long defaultMaxBytesInMemory = Runtime.getRuntime().maxMemory() / 3; private final ConcurrentHashMap aggregators = new ConcurrentHashMap<>(); private final FactsHolder facts; private final AtomicInteger indexIncrement = new AtomicInteger(0); @@ -80,7 +79,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex { super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions, concurrentEventAdd); this.maxRowCount = maxRowCount; - this.maxBytesInMemory = maxBytesInMemory == 0 ? defaultMaxBytesInMemory : maxBytesInMemory; + this.maxBytesInMemory = maxBytesInMemory; this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) : new PlainFactsHolder(sortFacts); if (maxBytesInMemory != -1) { @@ -88,15 +87,25 @@ public class OnheapIncrementalIndex extends IncrementalIndex } } - private long getMaxBytesPerRowForAggregators(IncrementalIndexSchema incrementalIndexSchema) + /** + * Gives estimated max size per aggregator. + * size estimation consists of: + *
    + *
  • metrics length : Integer.BYTES * len + *
  • maxAggregatorIntermediateSize : getMaxIntermediateSize per aggregator + overhead(16 bytes) + *
+ * + * @param incrementalIndexSchema + * + * @return long max aggregator size in bytes + */ + private static long getMaxBytesPerRowForAggregators(IncrementalIndexSchema incrementalIndexSchema) { long maxAggregatorIntermediateSize = Integer.BYTES * incrementalIndexSchema.getMetrics().length; maxAggregatorIntermediateSize += Arrays.stream(incrementalIndexSchema.getMetrics()) - .reduce( - 0, - (sum, aggregator) -> sum += aggregator.getMaxIntermediateSize(), - (sum1, sum2) -> sum1 + sum2 - ); + .mapToLong(aggregator -> aggregator.getMaxIntermediateSize() + + Long.BYTES * 2) + .sum(); return maxAggregatorIntermediateSize; } @@ -169,7 +178,7 @@ protected AddToFactsResult addToFacts( if (TimeAndDims.EMPTY_ROW_INDEX == prev) { numEntries.incrementAndGet(); if (maxBytesInMemory != -1) { - long estimatedRowSize = estimateRowSizeInBytes(key) + maxBytesPerRowForAggregators; + long estimatedRowSize = estimateRowSizeInBytes(key, maxBytesPerRowForAggregators); sizeInBytes.addAndGet(estimatedRowSize); } } else { @@ -185,9 +194,22 @@ protected AddToFactsResult addToFacts( return new AddToFactsResult(numEntries.get(), parseExceptionMessages); } - private long estimateRowSizeInBytes(TimeAndDims key) + /** + * Gives an estimated size of row in bytes, it accounts for: + *
    + *
  • overhead per Map Entry + *
  • TimeAndDims key size + *
  • aggregator size + *
+ * + * @param key TimeAndDims key + * @param maxBytesPerRowForAggregators max size per aggregator + * + * @return estimated size of row + */ + private long estimateRowSizeInBytes(TimeAndDims key, long maxBytesPerRowForAggregators) { - return ROUGH_OVERHEAD_PER_MAP_ENTRY + key.estimateBytesInMemory(); + return ROUGH_OVERHEAD_PER_MAP_ENTRY + key.estimateBytesInMemory() + maxBytesPerRowForAggregators; } @Override @@ -277,16 +299,24 @@ public boolean canAppendRow() { final boolean countCheck = size() < maxRowCount; boolean sizeCheck = true; - if (maxBytesInMemory != -1) { + if (maxBytesInMemory > 0 ) { sizeCheck = sizeInBytes() < maxBytesInMemory; } final boolean canAdd = countCheck && sizeCheck; - if (!countCheck) { - outOfRowsReason = StringUtils.format("Maximum number of rows [%d] reached", maxRowCount); - } - if (!sizeCheck) { - outOfRowsReason = StringUtils.format("Maximum size in bytes [%d] reached", maxBytesInMemory); + if (!countCheck && !sizeCheck) { + outOfRowsReason = StringUtils.format( + "Maximum number of rows [%d] and maximum size in bytes [%d] reached", + maxRowCount, + maxBytesInMemory + ); + } else { + if (!countCheck) { + outOfRowsReason = StringUtils.format("Maximum number of rows [%d] reached", maxRowCount); + } else if (!sizeCheck) { + outOfRowsReason = StringUtils.format("Maximum size in bytes [%d] reached", maxBytesInMemory); + } } + return canAdd; } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 53ab67b85e29..e906fae5e0c1 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -287,7 +287,7 @@ int getRowsInMemory() } @VisibleForTesting - long getRowSizeInMemory(SegmentIdentifier identifier) + long getBytesInMemory(SegmentIdentifier identifier) { final Sink sink = sinks.get(identifier); diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index f336d4eff3a5..c3d0064018ee 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -283,7 +283,8 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) final IncrementalIndex newIndex = new IncrementalIndex.Builder() .setIndexSchema(indexSchema) .setReportParseExceptions(reportParseExceptions) - .setMaxRowCount(maxRowsInMemory).setMaxBytesInMemory(maxBytesInMemory) + .setMaxRowCount(maxRowsInMemory) + .setMaxBytesInMemory(maxBytesInMemory) .buildOnheap(); final FireHydrant old; diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java index 9aa4f85ccff5..66b4d2184bc4 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java @@ -171,10 +171,10 @@ public void run() appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); - //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 24 (aggregator metrics) + 10 (dimsKeySize) = 106 - Assert.assertEquals(106, ((AppenderatorImpl) appenderator).getRowSizeInMemory(IDENTIFIERS.get(0))); + //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 10 (dimsKeySize) = 138 + Assert.assertEquals(138, ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0))); appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); - Assert.assertEquals(106, ((AppenderatorImpl) appenderator).getRowSizeInMemory(IDENTIFIERS.get(1))); + Assert.assertEquals(138, ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1))); appenderator.close(); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); } @@ -209,10 +209,10 @@ public void run() appenderator.startJob(); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowSizeInMemory(IDENTIFIERS.get(0))); + Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0))); Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowSizeInMemory(IDENTIFIERS.get(1))); + Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1))); Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.close(); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); From 55a3d2be39735b65030a6a31523153c8fc42bfc4 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Fri, 6 Apr 2018 16:17:49 -0700 Subject: [PATCH 14/31] Address more code review comments * Some refactoring to put defaults in IndexTaskUtils * Added check for maxBytesInMemory in AppenderatorImpl * Decrement bytes in abandonSegment * Test unit test for multiple sinks in single appenderator * Fix some merge conflicts after rebase --- .../druid/indexing/common/task/IndexTask.java | 8 ++-- .../indexing/common/task/IndexTaskUtils.java | 2 + .../indexing/common/task/IndexTaskTest.java | 3 ++ .../segment/incremental/IncrementalIndex.java | 19 ++++++++- .../IncrementalIndexAddResult.java | 8 ++++ .../incremental/OnheapIncrementalIndex.java | 15 +++---- .../OnheapIncrementalIndexBenchmark.java | 7 ++-- .../incremental/TimeAndDimsKeySizeTest.java | 9 ++-- .../appenderator/AppenderatorImpl.java | 22 ++++++++-- .../druid/segment/realtime/plumber/Sink.java | 2 +- .../appenderator/AppenderatorTest.java | 41 ++++++++++++++++++- 11 files changed, 109 insertions(+), 27 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index f293d20d72cc..53a56b983162 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -1230,9 +1230,7 @@ public boolean isAppendToExisting() @JsonTypeName("index") public static class IndexTuningConfig implements TuningConfig, AppenderatorConfig { - private static final int DEFAULT_MAX_ROWS_IN_MEMORY = 75_000; private static final int DEFAULT_MAX_TOTAL_ROWS = 20_000_000; - private static final long DEFAULT_MAX_BYTES_IN_MEMORY = Runtime.getRuntime().maxMemory() / 3; private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; private static final boolean DEFAULT_FORCE_EXTENDABLE_SHARD_SPECS = false; @@ -1299,7 +1297,7 @@ public IndexTuningConfig( this( targetPartitionSize, maxRowsInMemory != null ? maxRowsInMemory : rowFlushBoundary_forBackCompatibility, - maxBytesInMemory != null ? maxBytesInMemory : DEFAULT_MAX_BYTES_IN_MEMORY, + maxBytesInMemory != null ? maxBytesInMemory : IndexTaskUtils.DEFAULT_MAX_BYTES_IN_MEMORY, maxTotalRows, numShards, indexSpec, @@ -1346,8 +1344,8 @@ private IndexTuningConfig( ); this.targetPartitionSize = initializeTargetPartitionSize(numShards, targetPartitionSize); - this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - this.maxBytesInMemory = maxBytesInMemory == null ? DEFAULT_MAX_BYTES_IN_MEMORY : maxBytesInMemory; + this.maxRowsInMemory = maxRowsInMemory == null ? IndexTaskUtils.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory == null ? IndexTaskUtils.DEFAULT_MAX_BYTES_IN_MEMORY : maxBytesInMemory; this.maxTotalRows = initializeMaxTotalRows(numShards, maxTotalRows); this.numShards = numShards == null || numShards.equals(-1) ? null : numShards; this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskUtils.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskUtils.java index aee89b58a5aa..0afb390dabc9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskUtils.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskUtils.java @@ -36,6 +36,8 @@ public class IndexTaskUtils { + public static final int DEFAULT_MAX_ROWS_IN_MEMORY = 75_000; + public static final long DEFAULT_MAX_BYTES_IN_MEMORY = Runtime.getRuntime().maxMemory() / 3; @Nullable public static List getMessagesFromSavedParseExceptions(CircularBuffer savedParseExceptions) { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index dbd9af3279ac..a6da30cbb783 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -893,6 +893,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception null, null, null, + null, indexSpec, null, true, @@ -1011,6 +1012,7 @@ public void testMultipleParseExceptionsFailure() throws Exception null, null, null, + null, indexSpec, null, true, @@ -1116,6 +1118,7 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc null, null, null, + null, indexSpec, null, true, diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index d0e85fad1747..e36f2370919f 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -520,6 +520,7 @@ public List getParseExceptionMessages() public static class AddToFactsResult { private int rowCount; + private long bytesInMemory; private List parseExceptionMessages; public AddToFactsResult( @@ -531,11 +532,27 @@ public AddToFactsResult( this.parseExceptionMessages = parseExceptionMessages; } + public AddToFactsResult( + int rowCount, + long bytesInMemory, + List parseExceptionMessages + ) + { + this.rowCount = rowCount; + this.bytesInMemory = bytesInMemory; + this.parseExceptionMessages = parseExceptionMessages; + } + public int getRowCount() { return rowCount; } + public long getBytesInMemory() + { + return bytesInMemory; + } + public List getParseExceptionMessages() { return parseExceptionMessages; @@ -602,7 +619,7 @@ public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCh timeAndDimsResult.getParseExceptionMessages(), addToFactsResult.getParseExceptionMessages() ); - return new IncrementalIndexAddResult(addToFactsResult.getRowCount(), parseException); + return new IncrementalIndexAddResult(addToFactsResult.getRowCount(), addToFactsResult.getBytesInMemory(), parseException); } @VisibleForTesting diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAddResult.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAddResult.java index 06c537a0aa36..e76d3c15a1ef 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAddResult.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAddResult.java @@ -26,16 +26,19 @@ public class IncrementalIndexAddResult { private final int rowCount; + private final long bytesInMemory; @Nullable private final ParseException parseException; public IncrementalIndexAddResult( int rowCount, + long bytesInMemory, @Nullable ParseException parseException ) { this.rowCount = rowCount; + this.bytesInMemory = bytesInMemory; this.parseException = parseException; } @@ -44,6 +47,11 @@ public int getRowCount() return rowCount; } + public long getBytesInMemory() + { + return bytesInMemory; + } + @Nullable public ParseException getParseException() { diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 98daddb3709b..04554d260ccd 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -79,12 +79,11 @@ public class OnheapIncrementalIndex extends IncrementalIndex { super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions, concurrentEventAdd); this.maxRowCount = maxRowCount; - this.maxBytesInMemory = maxBytesInMemory; + this.maxBytesInMemory = maxBytesInMemory; this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) : new PlainFactsHolder(sortFacts); - if (maxBytesInMemory != -1) { - maxBytesPerRowForAggregators = getMaxBytesPerRowForAggregators(incrementalIndexSchema); - } + maxBytesPerRowForAggregators = getMaxBytesPerRowForAggregators(incrementalIndexSchema); + } /** @@ -177,10 +176,8 @@ protected AddToFactsResult addToFacts( final int prev = facts.putIfAbsent(key, rowIndex); if (TimeAndDims.EMPTY_ROW_INDEX == prev) { numEntries.incrementAndGet(); - if (maxBytesInMemory != -1) { - long estimatedRowSize = estimateRowSizeInBytes(key, maxBytesPerRowForAggregators); - sizeInBytes.addAndGet(estimatedRowSize); - } + long estimatedRowSize = estimateRowSizeInBytes(key, maxBytesPerRowForAggregators); + sizeInBytes.addAndGet(estimatedRowSize); } else { // We lost a race aggs = concurrentGet(prev); @@ -191,7 +188,7 @@ protected AddToFactsResult addToFacts( } } - return new AddToFactsResult(numEntries.get(), parseExceptionMessages); + return new AddToFactsResult(numEntries.get(), sizeInBytes.get(), parseExceptionMessages); } /** diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index df2713027f2c..d40141f7a2ce 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -208,12 +208,13 @@ protected AddToFactsResult addToFacts( // Last ditch sanity checks - if (numEntries.get() >= maxRowCount && getFacts().getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX) { - throw new IndexSizeExceededException("Maximum number of rows reached"); + if (numEntries.get() >= maxRowCount || sizeInBytes.get() >= maxBytesInMemory && getFacts().getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX) { + throw new IndexSizeExceededException("Maximum number of rows or max bytes reached"); } final int prev = getFacts().putIfAbsent(key, rowIndex); if (TimeAndDims.EMPTY_ROW_INDEX == prev) { numEntries.incrementAndGet(); + sizeInBytes.incrementAndGet(); } else { // We lost a race aggs = indexedMap.get(prev); @@ -241,7 +242,7 @@ protected AddToFactsResult addToFacts( rowContainer.set(null); - return new AddToFactsResult(numEntries.get(), new ArrayList<>()); + return new AddToFactsResult(numEntries.get(), sizeInBytes.get(), new ArrayList<>()); } @Override diff --git a/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsKeySizeTest.java b/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsKeySizeTest.java index e8b59652faf4..b85d2c93a1b0 100644 --- a/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsKeySizeTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsKeySizeTest.java @@ -41,7 +41,8 @@ public void testTimeAndDimsKeySizeBasic() .setMaxBytesInMemory(1000) .buildOnheap(); long time = System.currentTimeMillis(); - IncrementalIndex.TimeAndDims td1 = index.toTimeAndDims(toMapRow(time, "billy", "A", "joe", "B")); + IncrementalIndex.TimeAndDimsResult tndResult = index.toTimeAndDims(toMapRow(time, "billy", "A", "joe", "B")); + IncrementalIndex.TimeAndDims td1 = tndResult.getTimeAndDims(); Assert.assertEquals(44, td1.estimateBytesInMemory()); } @@ -54,13 +55,14 @@ public void testTimeAndDimsKeySizeArr() .setMaxBytesInMemory(1000) .buildOnheap(); long time = System.currentTimeMillis(); - IncrementalIndex.TimeAndDims td1 = index.toTimeAndDims(toMapRow( + IncrementalIndex.TimeAndDimsResult tndResult = index.toTimeAndDims(toMapRow( time + 1, "billy", "A", "joe", Arrays.asList("A", "B") )); + IncrementalIndex.TimeAndDims td1 = tndResult.getTimeAndDims(); Assert.assertEquals(50, td1.estimateBytesInMemory()); } @@ -73,13 +75,14 @@ public void testTimeAndDimsKeySizeComplex() .setMaxBytesInMemory(1000) .buildOnheap(); long time = System.currentTimeMillis(); - IncrementalIndex.TimeAndDims td1 = index.toTimeAndDims(toMapRow( + IncrementalIndex.TimeAndDimsResult tndResult = index.toTimeAndDims(toMapRow( time + 1, "billy", "nelson", "joe", Arrays.asList("123", "abcdef") )); + IncrementalIndex.TimeAndDims td1 = tndResult.getTimeAndDims(); Assert.assertEquals(74, td1.estimateBytesInMemory()); } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index e906fae5e0c1..cf00120a7c7d 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -97,6 +97,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; @@ -128,6 +129,7 @@ public class AppenderatorImpl implements Appenderator // This variable updated in add(), persist(), and drop() private final AtomicInteger rowsCurrentlyInMemory = new AtomicInteger(); private final AtomicInteger totalRows = new AtomicInteger(); + private final AtomicLong currentBytesInMemory = new AtomicLong(); // Synchronize persisting commitMetadata so that multiple persist threads (if present) // and abandon threads do not step over each other private final Lock commitLock = new ReentrantLock(); @@ -219,11 +221,14 @@ public AppenderatorAddResult add( metrics.reportMessageMaxTimestamp(row.getTimestampFromEpoch()); final int sinkRowsInMemoryBeforeAdd = sink.getNumRowsInMemory(); final int sinkRowsInMemoryAfterAdd; - final IncrementalIndexAddResult addResult; + final long bytesInMemoryBeforeAdd = sink.getBytesInMemory(); + final long bytesInMemoryAfterAdd; + final IncrementalIndexAddResult addResult; try { addResult = sink.add(row, !allowIncrementalPersists); sinkRowsInMemoryAfterAdd = addResult.getRowCount(); + bytesInMemoryAfterAdd = addResult.getBytesInMemory(); } catch (IndexSizeExceededException e) { // Uh oh, we can't do anything about this! We can't persist (commit metadata would be out of sync) and we @@ -240,11 +245,13 @@ public AppenderatorAddResult add( final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd; rowsCurrentlyInMemory.addAndGet(numAddedRows); totalRows.addAndGet(numAddedRows); + currentBytesInMemory.addAndGet(bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd); boolean isPersistRequired = false; if (!sink.canAppendRow() || System.currentTimeMillis() > nextFlush - || rowsCurrentlyInMemory.get() >= tuningConfig.getMaxRowsInMemory()) { + || rowsCurrentlyInMemory.get() >= tuningConfig.getMaxRowsInMemory() + || (tuningConfig.getMaxBytesInMemory() > 0 && currentBytesInMemory.get() >= tuningConfig.getMaxBytesInMemory())) { if (allowIncrementalPersists) { // persistAll clears rowsCurrentlyInMemory, no need to update it. persistAll(committerSupplier == null ? null : committerSupplier.get()); @@ -286,6 +293,12 @@ int getRowsInMemory() return rowsCurrentlyInMemory.get(); } + @VisibleForTesting + long getCurrentBytesInMemory() + { + return currentBytesInMemory.get(); + } + @VisibleForTesting long getBytesInMemory(SegmentIdentifier identifier) { @@ -410,6 +423,7 @@ public ListenableFuture persist(Collection identifier final Map currentHydrants = Maps.newHashMap(); final List> indexesToPersist = Lists.newArrayList(); int numPersistedRows = 0; + long bytesPersisted = 0L; for (SegmentIdentifier identifier : identifiers) { final Sink sink = sinks.get(identifier); if (sink == null) { @@ -418,6 +432,7 @@ public ListenableFuture persist(Collection identifier final List hydrants = Lists.newArrayList(sink); currentHydrants.put(identifier.getIdentifierAsString(), hydrants.size()); numPersistedRows += sink.getNumRowsInMemory(); + bytesPersisted += sink.getBytesInMemory(); final int limit = sink.isWritable() ? hydrants.size() - 1 : hydrants.size(); @@ -508,7 +523,7 @@ public Object doCall() // NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes. rowsCurrentlyInMemory.addAndGet(-numPersistedRows); - + currentBytesInMemory.addAndGet(-bytesPersisted); return future; } @@ -1048,6 +1063,7 @@ private ListenableFuture abandonSegment( // Decrement this sink's rows from rowsCurrentlyInMemory (we only count active sinks). rowsCurrentlyInMemory.addAndGet(-sink.getNumRowsInMemory()); totalRows.addAndGet(-sink.getNumRows()); + currentBytesInMemory.addAndGet(-sink.getBytesInMemory()); // Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread. return Futures.transform( diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index c3d0064018ee..e02f2f211e6f 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -54,7 +54,7 @@ public class Sink implements Iterable { - private static final IncrementalIndexAddResult ADD_FAILED = new IncrementalIndexAddResult(-1, null); + private static final IncrementalIndexAddResult ADD_FAILED = new IncrementalIndexAddResult(-1, -1,null); private final Object hydrantLock = new Object(); private final Interval interval; diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java index 66b4d2184bc4..9dfb269abbc5 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java @@ -180,6 +180,42 @@ public void run() } } + @Test + public void testMaxBytesInMemoryInMultipleSinks() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester(100, 1024, true)) { + final Appenderator appenderator = tester.getAppenderator(); + final AtomicInteger eventCount = new AtomicInteger(0); + final Supplier committerSupplier = () -> { + final Object metadata = ImmutableMap.of(eventCount, eventCount.get()); + + return new Committer() + { + @Override + public Object getMetadata() + { + return metadata; + } + + @Override + public void run() + { + //Do nothing + } + }; + }; + + appenderator.startJob(); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); + //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 10 (dimsKeySize) = 138 + Assert.assertEquals(138, ((AppenderatorImpl) appenderator).getCurrentBytesInMemory()); + appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); + Assert.assertEquals(276, ((AppenderatorImpl) appenderator).getCurrentBytesInMemory()); + appenderator.close(); + Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); + } + } + @Test public void testIgnoreMaxBytesInMemory() throws Exception { @@ -209,10 +245,11 @@ public void run() appenderator.startJob(); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0))); + //we still calculate the size even when ignoring it to make persist decision + Assert.assertEquals(138, ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0))); Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(1))); + Assert.assertEquals(276, ((AppenderatorImpl) appenderator).getCurrentBytesInMemory()); Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.close(); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); From c40678b64a1c617152a1bae7f41f7cfd9acd5302 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Fri, 6 Apr 2018 16:46:14 -0700 Subject: [PATCH 15/31] Fix some style checks --- .../io/druid/segment/incremental/OnheapIncrementalIndex.java | 3 +-- .../druid/segment/realtime/appenderator/AppenderatorImpl.java | 2 +- .../src/main/java/io/druid/segment/realtime/plumber/Sink.java | 2 +- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 04554d260ccd..51a9360a1dea 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -37,7 +37,6 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.util.Arrays; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -296,7 +295,7 @@ public boolean canAppendRow() { final boolean countCheck = size() < maxRowCount; boolean sizeCheck = true; - if (maxBytesInMemory > 0 ) { + if (maxBytesInMemory > 0) { sizeCheck = sizeInBytes() < maxBytesInMemory; } final boolean canAdd = countCheck && sizeCheck; diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index cf00120a7c7d..7ccd99805ecc 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -223,7 +223,7 @@ public AppenderatorAddResult add( final int sinkRowsInMemoryAfterAdd; final long bytesInMemoryBeforeAdd = sink.getBytesInMemory(); final long bytesInMemoryAfterAdd; - final IncrementalIndexAddResult addResult; + final IncrementalIndexAddResult addResult; try { addResult = sink.add(row, !allowIncrementalPersists); diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index e02f2f211e6f..d520574b9cfd 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -54,7 +54,7 @@ public class Sink implements Iterable { - private static final IncrementalIndexAddResult ADD_FAILED = new IncrementalIndexAddResult(-1, -1,null); + private static final IncrementalIndexAddResult ADD_FAILED = new IncrementalIndexAddResult(-1, -1, null); private final Object hydrantLock = new Object(); private final Interval interval; From 9f87c2fb89523337f7c36a7ccdd72c1f8e62f80e Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Mon, 9 Apr 2018 11:32:29 -0700 Subject: [PATCH 16/31] Merge conflicts --- .../druid/indexing/common/task/IndexTask.java | 56 ------------------- .../OnheapIncrementalIndexBenchmark.java | 4 -- 2 files changed, 60 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index f7ef5a1bc186..53a56b983162 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -460,7 +460,6 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception if (chatHandlerProvider.isPresent()) { chatHandlerProvider.get().unregister(getId()); } -<<<<<<< HEAD } } @@ -495,57 +494,6 @@ private Map getTaskCompletionUnparseableEvents() return unparseableEventsMap; } - private Map getTaskCompletionRowStats() - { - Map metrics = Maps.newHashMap(); - if (determinePartitionsMetricsGetter != null) { - metrics.put( - "determinePartitions", - determinePartitionsMetricsGetter.getTotalMetrics() - ); - } - if (buildSegmentsMetricsGetter != null) { - metrics.put( - "buildSegments", - buildSegmentsMetricsGetter.getTotalMetrics() - ); -======= ->>>>>>> 685f4063d4e63fba5f9ae11bf496c4c2575a8ebf - } - return metrics; - } - - private Map getTaskCompletionReports() - { - return TaskReport.buildTaskReports( - new IngestionStatsAndErrorsTaskReport( - getId(), - new IngestionStatsAndErrorsTaskReportData( - ingestionState, - getTaskCompletionUnparseableEvents(), - getTaskCompletionRowStats(), - errorMsg - ) - ) - ); - } - - private Map getTaskCompletionUnparseableEvents() - { - Map unparseableEventsMap = Maps.newHashMap(); - List determinePartitionsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( - determinePartitionsSavedParseExceptions); - List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( - buildSegmentsSavedParseExceptions); - - if (determinePartitionsParseExceptionMessages != null || buildSegmentsParseExceptionMessages != null) { - unparseableEventsMap.put("determinePartitions", determinePartitionsParseExceptionMessages); - unparseableEventsMap.put("buildSegments", buildSegmentsParseExceptionMessages); - } - - return unparseableEventsMap; - } - private Map getTaskCompletionRowStats() { Map metrics = Maps.newHashMap(); @@ -1368,11 +1316,7 @@ public IndexTuningConfig( private IndexTuningConfig() { -<<<<<<< HEAD this(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); -======= - this(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); ->>>>>>> 685f4063d4e63fba5f9ae11bf496c4c2575a8ebf } private IndexTuningConfig( diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index 9d91cf3464e6..d40141f7a2ce 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -242,11 +242,7 @@ protected AddToFactsResult addToFacts( rowContainer.set(null); -<<<<<<< HEAD return new AddToFactsResult(numEntries.get(), sizeInBytes.get(), new ArrayList<>()); -======= - return new AddToFactsResult(numEntries.get(), new ArrayList<>()); ->>>>>>> 685f4063d4e63fba5f9ae11bf496c4c2575a8ebf } @Override From c45bf3bcc0a643c242212ab06922a4afd494b8ff Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Mon, 9 Apr 2018 13:41:05 -0700 Subject: [PATCH 17/31] Fix failing tests Add back check for 0 maxBytesInMemory in OnHeapIncrementalIndex --- .../io/druid/segment/incremental/OnheapIncrementalIndex.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 51a9360a1dea..496a1006194e 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -56,6 +56,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex * KeyHash + next pointer + key pointer + value pointer + safe extra */ private static final int ROUGH_OVERHEAD_PER_MAP_ENTRY = Long.BYTES * 5 + Integer.BYTES; + private static final long defaultMaxBytesInMemory = Runtime.getRuntime().maxMemory() / 3; private final ConcurrentHashMap aggregators = new ConcurrentHashMap<>(); private final FactsHolder facts; private final AtomicInteger indexIncrement = new AtomicInteger(0); @@ -78,7 +79,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex { super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions, concurrentEventAdd); this.maxRowCount = maxRowCount; - this.maxBytesInMemory = maxBytesInMemory; + this.maxBytesInMemory = maxBytesInMemory == 0 ? defaultMaxBytesInMemory : maxBytesInMemory; this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) : new PlainFactsHolder(sortFacts); maxBytesPerRowForAggregators = getMaxBytesPerRowForAggregators(incrementalIndexSchema); From 5363f0f954e31bf4729de8fad9a015628d8de9d4 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Thu, 12 Apr 2018 18:08:21 -0700 Subject: [PATCH 18/31] Address PR comments * Put defaults for maxRows and maxBytes in TuningConfig * Change/add javadocs * Refactoring and renaming some variables/methods --- .../indexing/kafka/KafkaTuningConfigTest.java | 2 +- .../KafkaSupervisorTuningConfigTest.java | 2 +- .../RealtimeAppenderatorTuningConfig.java | 8 +-- .../druid/indexing/common/task/IndexTask.java | 6 +-- .../indexing/common/task/IndexTaskUtils.java | 2 - .../indexing/common/task/TaskSerdeTest.java | 2 +- .../druid/segment/StringDimensionIndexer.java | 4 +- .../segment/incremental/IncrementalIndex.java | 40 ++++++++------- .../IncrementalIndexAddResult.java | 2 - .../incremental/OffheapIncrementalIndex.java | 2 +- .../incremental/OnheapIncrementalIndex.java | 31 +++++++----- .../indexing/RealtimeTuningConfig.java | 9 +--- .../druid/segment/indexing/TuningConfig.java | 2 + .../appenderator/AppenderatorImpl.java | 49 +++++++++++++++---- .../druid/segment/realtime/plumber/Sink.java | 4 +- .../indexing/RealtimeTuningConfigTest.java | 2 +- .../appenderator/AppenderatorTest.java | 6 +-- 17 files changed, 101 insertions(+), 72 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java index 369396010702..04a07b81ed70 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java @@ -56,7 +56,7 @@ public void testSerdeWithDefaults() throws Exception ); Assert.assertNotNull(config.getBasePersistDirectory()); - Assert.assertEquals(75000, config.getMaxRowsInMemory()); + Assert.assertEquals(1000000, config.getMaxRowsInMemory()); Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); Assert.assertEquals(0, config.getMaxPendingPersists()); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java index 34477c143727..36fe290d2a2d 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java @@ -58,7 +58,7 @@ public void testSerdeWithDefaults() throws Exception ); Assert.assertNotNull(config.getBasePersistDirectory()); - Assert.assertEquals(75000, config.getMaxRowsInMemory()); + Assert.assertEquals(1000000, config.getMaxRowsInMemory()); Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); Assert.assertEquals(0, config.getMaxPendingPersists()); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index 77a780579732..b20de38f29d6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -37,9 +37,9 @@ @JsonTypeName("realtime_appenderator") public class RealtimeAppenderatorTuningConfig implements TuningConfig, AppenderatorConfig { - private static final int defaultMaxRowsInMemory = 1000000; + private static final int defaultMaxRowsInMemory = TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY; private static final int defaultMaxRowsPerSegment = 5_000_000; - private static final long defaultMaxBytesInMemory = getDefaultMaxBytesInMemory(); + private static final long defaultMaxBytesInMemory = TuningConfig.DEFAULT_MAX_BYTES_IN_MEMORY; private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); private static final int defaultMaxPendingPersists = 0; private static final ShardSpec defaultShardSpec = NoneShardSpec.instance(); @@ -53,10 +53,6 @@ private static File createNewBasePersistDirectory() return Files.createTempDir(); } - private static long getDefaultMaxBytesInMemory() - { - return (Runtime.getRuntime().maxMemory()) / 3; - } private final int maxRowsInMemory; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 53a56b983162..a652d38a4200 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -1297,7 +1297,7 @@ public IndexTuningConfig( this( targetPartitionSize, maxRowsInMemory != null ? maxRowsInMemory : rowFlushBoundary_forBackCompatibility, - maxBytesInMemory != null ? maxBytesInMemory : IndexTaskUtils.DEFAULT_MAX_BYTES_IN_MEMORY, + maxBytesInMemory != null ? maxBytesInMemory : TuningConfig.DEFAULT_MAX_BYTES_IN_MEMORY, maxTotalRows, numShards, indexSpec, @@ -1344,8 +1344,8 @@ private IndexTuningConfig( ); this.targetPartitionSize = initializeTargetPartitionSize(numShards, targetPartitionSize); - this.maxRowsInMemory = maxRowsInMemory == null ? IndexTaskUtils.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - this.maxBytesInMemory = maxBytesInMemory == null ? IndexTaskUtils.DEFAULT_MAX_BYTES_IN_MEMORY : maxBytesInMemory; + this.maxRowsInMemory = maxRowsInMemory == null ? TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory == null ? TuningConfig.DEFAULT_MAX_BYTES_IN_MEMORY : maxBytesInMemory; this.maxTotalRows = initializeMaxTotalRows(numShards, maxTotalRows); this.numShards = numShards == null || numShards.equals(-1) ? null : numShards; this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskUtils.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskUtils.java index 0afb390dabc9..aee89b58a5aa 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskUtils.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskUtils.java @@ -36,8 +36,6 @@ public class IndexTaskUtils { - public static final int DEFAULT_MAX_ROWS_IN_MEMORY = 75_000; - public static final long DEFAULT_MAX_BYTES_IN_MEMORY = Runtime.getRuntime().maxMemory() / 3; @Nullable public static List getMessagesFromSavedParseExceptions(CircularBuffer savedParseExceptions) { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index b23bc5f6ba6c..545bb9ec08e7 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -108,7 +108,7 @@ public void testIndexTaskTuningConfigDefaults() throws Exception Assert.assertEquals(new IndexSpec(), tuningConfig.getIndexSpec()); Assert.assertEquals(new Period(Integer.MAX_VALUE), tuningConfig.getIntermediatePersistPeriod()); Assert.assertEquals(0, tuningConfig.getMaxPendingPersists()); - Assert.assertEquals(75000, tuningConfig.getMaxRowsInMemory()); + Assert.assertEquals(1000000, tuningConfig.getMaxRowsInMemory()); Assert.assertEquals(null, tuningConfig.getNumShards()); Assert.assertEquals(5000000, (int) tuningConfig.getTargetPartitionSize()); } diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java index 5649a3f25aca..165003295a55 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -268,9 +268,7 @@ public int[] processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boole @Override public long estimateEncodedKeyComponentSize(int[] key) { - /** - * string length is being accounted for each time they are referenced, even though they are stored just once - */ + // string length is being accounted for each time they are referenced, even though they are stored just once long estimatedSize = key.length * Integer.BYTES; estimatedSize += Arrays.stream(key) .filter(element -> dimLookup.getValue(element) != null) diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index e36f2370919f..072cdf3f1757 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -241,7 +241,7 @@ public ColumnCapabilities getColumnCapabilities(String columnName) private final List dimensionDescsList; private final Map columnCapabilities; private final AtomicInteger numEntries = new AtomicInteger(); - private final AtomicLong sizeInBytes = new AtomicLong(); + private final AtomicLong bytesInMemory = new AtomicLong(); // This is modified on add() in a critical section. private final ThreadLocal in = new ThreadLocal<>(); @@ -523,15 +523,6 @@ public static class AddToFactsResult private long bytesInMemory; private List parseExceptionMessages; - public AddToFactsResult( - int rowCount, - List parseExceptionMessages - ) - { - this.rowCount = rowCount; - this.parseExceptionMessages = parseExceptionMessages; - } - public AddToFactsResult( int rowCount, long bytesInMemory, @@ -607,7 +598,7 @@ public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCh reportParseExceptions, row, numEntries, - sizeInBytes, + bytesInMemory, timeAndDimsResult.getTimeAndDims(), in, rowSupplier, @@ -716,7 +707,12 @@ TimeAndDimsResult toTimeAndDims(InputRow row) truncated = gran.bucketStart(row.getTimestamp()).getMillis(); } - TimeAndDims timeAndDims = new TimeAndDims(Math.max(truncated, minTimestamp), dims, dimensionDescsList, dimsKeySize); + TimeAndDims timeAndDims = TimeAndDims.createTimeAndDimswithDimsKeySize( + Math.max(truncated, minTimestamp), + dims, + dimensionDescsList, + dimsKeySize + ); return new TimeAndDimsResult(timeAndDims, parseExceptionMessages); } @@ -773,9 +769,9 @@ public int size() return numEntries.get(); } - public long sizeInBytes() + public long getBytesInMemory() { - return sizeInBytes.get(); + return bytesInMemory.get(); } private long getMinTimeMillis() @@ -1162,7 +1158,7 @@ public static final class TimeAndDims this.rowIndex = rowIndex; } - TimeAndDims( + private TimeAndDims( long timestamp, Object[] dims, List dimensionDescsList, @@ -1175,6 +1171,16 @@ public static final class TimeAndDims this.dimsKeySize = dimsKeySize; } + public static TimeAndDims createTimeAndDimswithDimsKeySize( + long timestamp, + Object[] dims, + List dimensionDescsList, + long dimsKeySize + ) + { + return new TimeAndDims(timestamp, dims, dimensionDescsList, dimsKeySize); + } + public long getTimestamp() { return timestamp; @@ -1196,7 +1202,7 @@ private void setRowIndex(int rowIndex) } /** - * sizeInBytes estimates the size of TimeAndDims key, it takes into account the timestamp(long), + * bytesInMemory estimates the size of TimeAndDims key, it takes into account the timestamp(long), * dims(Object Array) and dimensionDescsList(List). Each of these are calculated as follows: *
    *
  • timestamp : Long.BYTES @@ -1204,7 +1210,7 @@ private void setRowIndex(int rowIndex) *
  • dimensionDescList : Long.BYTES (shared pointer) *
* - * @return long estimated sizeInBytes + * @return long estimated bytesInMemory */ public long estimateBytesInMemory() { diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAddResult.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAddResult.java index 594365c79e77..e76d3c15a1ef 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAddResult.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAddResult.java @@ -47,13 +47,11 @@ public int getRowCount() return rowCount; } - public long getBytesInMemory() { return bytesInMemory; } - @Nullable public ParseException getParseException() { diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java index 54b772360551..757a3cf42847 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -240,7 +240,7 @@ protected AddToFactsResult addToFacts( } } rowContainer.set(null); - return new AddToFactsResult(numEntries.get(), new ArrayList<>()); + return new AddToFactsResult(numEntries.get(), 0, new ArrayList<>()); } @Override diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 496a1006194e..03304ce7d251 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -52,11 +52,10 @@ public class OnheapIncrementalIndex extends IncrementalIndex { private static final Logger log = new Logger(OnheapIncrementalIndex.class); /** - * overhead per Map$Entry object - * KeyHash + next pointer + key pointer + value pointer + safe extra + * overhead per Map$Entry object + * KeyHash + next pointer + key pointer + value pointer + safe extra */ private static final int ROUGH_OVERHEAD_PER_MAP_ENTRY = Long.BYTES * 5 + Integer.BYTES; - private static final long defaultMaxBytesInMemory = Runtime.getRuntime().maxMemory() / 3; private final ConcurrentHashMap aggregators = new ConcurrentHashMap<>(); private final FactsHolder facts; private final AtomicInteger indexIncrement = new AtomicInteger(0); @@ -79,19 +78,25 @@ public class OnheapIncrementalIndex extends IncrementalIndex { super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions, concurrentEventAdd); this.maxRowCount = maxRowCount; - this.maxBytesInMemory = maxBytesInMemory == 0 ? defaultMaxBytesInMemory : maxBytesInMemory; + this.maxBytesInMemory = maxBytesInMemory == 0 ? -1 : maxBytesInMemory; this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) : new PlainFactsHolder(sortFacts); - maxBytesPerRowForAggregators = getMaxBytesPerRowForAggregators(incrementalIndexSchema); + maxBytesPerRowForAggregators = this.getMaxBytesPerRowForAggregators(incrementalIndexSchema); } /** - * Gives estimated max size per aggregator. - * size estimation consists of: + * Gives estimated max size per aggregator. It is assumed that every aggregator will have enough overhead for its own object header + * and for a pointer to a selector. We are adding a overhead-factor for each object as additional 16 bytes. + * These 16 bytes or 128 bits is the object metadata for 64-bit JVM process and consists of: + *
    + *
  • Class pointer which describes the object type: 64 bits + *
  • Flags which describe state of the object including hashcode: 64 bits + *
      + * total size estimation consists of: *
        *
      • metrics length : Integer.BYTES * len - *
      • maxAggregatorIntermediateSize : getMaxIntermediateSize per aggregator + overhead(16 bytes) + *
      • maxAggregatorIntermediateSize : getMaxIntermediateSize per aggregator + overhead-factor(16 bytes) *
      * * @param incrementalIndexSchema @@ -168,10 +173,14 @@ protected AddToFactsResult addToFacts( // Last ditch sanity checks if (numEntries.get() >= maxRowCount - || maxBytesInMemory != -1 && sizeInBytes.get() >= maxBytesInMemory + || (maxBytesInMemory != -1 && sizeInBytes.get() >= maxBytesInMemory) && facts.getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX && !skipMaxRowsInMemoryCheck) { - throw new IndexSizeExceededException("Maximum number of rows [%d] or max size in bytes [%d] reached", maxRowCount, maxBytesInMemory); + throw new IndexSizeExceededException( + "Maximum number of rows [%d] or max size in bytes [%d] reached", + maxRowCount, + maxBytesInMemory + ); } final int prev = facts.putIfAbsent(key, rowIndex); if (TimeAndDims.EMPTY_ROW_INDEX == prev) { @@ -297,7 +306,7 @@ public boolean canAppendRow() final boolean countCheck = size() < maxRowCount; boolean sizeCheck = true; if (maxBytesInMemory > 0) { - sizeCheck = sizeInBytes() < maxBytesInMemory; + sizeCheck = getBytesInMemory() < maxBytesInMemory; } final boolean canAdd = countCheck && sizeCheck; if (!countCheck && !sizeCheck) { diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index 42899b33ecb3..afb3fe525226 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -41,8 +41,8 @@ */ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig { - private static final int defaultMaxRowsInMemory = 75000; - private static final long defaultMaxBytesInMemory = getDefaultMaxBytesInMemory(); + private static final int defaultMaxRowsInMemory = TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY; + private static final long defaultMaxBytesInMemory = TuningConfig.DEFAULT_MAX_BYTES_IN_MEMORY; private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); private static final Period defaultWindowPeriod = new Period("PT10M"); private static final VersioningPolicy defaultVersioningPolicy = new IntervalStartVersioningPolicy(); @@ -59,11 +59,6 @@ private static File createNewBasePersistDirectory() return Files.createTempDir(); } - private static long getDefaultMaxBytesInMemory() - { - return (Runtime.getRuntime().maxMemory()) / 3; - } - // Might make sense for this to be a builder public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File basePersistDirectory) { diff --git a/server/src/main/java/io/druid/segment/indexing/TuningConfig.java b/server/src/main/java/io/druid/segment/indexing/TuningConfig.java index 7fd246d25732..e4d3d362f177 100644 --- a/server/src/main/java/io/druid/segment/indexing/TuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/TuningConfig.java @@ -33,4 +33,6 @@ public interface TuningConfig boolean DEFAULT_LOG_PARSE_EXCEPTIONS = false; int DEFAULT_MAX_PARSE_EXCEPTIONS = Integer.MAX_VALUE; int DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS = 0; + static final int DEFAULT_MAX_ROWS_IN_MEMORY = 1_000_000; + static final long DEFAULT_MAX_BYTES_IN_MEMORY = Runtime.getRuntime().maxMemory() / 3; } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 7ccd99805ecc..f2dcd9dd80ea 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -129,7 +129,7 @@ public class AppenderatorImpl implements Appenderator // This variable updated in add(), persist(), and drop() private final AtomicInteger rowsCurrentlyInMemory = new AtomicInteger(); private final AtomicInteger totalRows = new AtomicInteger(); - private final AtomicLong currentBytesInMemory = new AtomicLong(); + private final AtomicLong bytesCurrentlyInMemory = new AtomicLong(); // Synchronize persisting commitMetadata so that multiple persist threads (if present) // and abandon threads do not step over each other private final Lock commitLock = new ReentrantLock(); @@ -245,15 +245,44 @@ public AppenderatorAddResult add( final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd; rowsCurrentlyInMemory.addAndGet(numAddedRows); totalRows.addAndGet(numAddedRows); - currentBytesInMemory.addAndGet(bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd); + bytesCurrentlyInMemory.addAndGet(bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd); boolean isPersistRequired = false; - if (!sink.canAppendRow() - || System.currentTimeMillis() > nextFlush - || rowsCurrentlyInMemory.get() >= tuningConfig.getMaxRowsInMemory() - || (tuningConfig.getMaxBytesInMemory() > 0 && currentBytesInMemory.get() >= tuningConfig.getMaxBytesInMemory())) { + boolean persist = false; + StringBuilder persistReason = new StringBuilder(); + + if (!sink.canAppendRow()) { + persist = true; + persistReason.append(" No more rows can be appended to sink"); + } + if (System.currentTimeMillis() > nextFlush) { + persist = true; + persistReason.append(StringUtils.format( + " current time[%d] is greater than nextFlush[%d],", + System.currentTimeMillis(), + nextFlush + )); + } + if (rowsCurrentlyInMemory.get() >= tuningConfig.getMaxRowsInMemory()) { + persist = true; + persistReason.append(StringUtils.format( + " rowsCurrentlyInMemory[%d] is greater than maxRowsInMemory[%d],", + rowsCurrentlyInMemory.get(), + tuningConfig.getMaxRowsInMemory() + )); + } + if (tuningConfig.getMaxBytesInMemory() > 0 && bytesCurrentlyInMemory.get() >= tuningConfig.getMaxBytesInMemory()) { + persist = true; + persistReason.append(StringUtils.format( + " bytesCurrentlyInMemory[%d] is greater than maxBytesInMemory[%d]", + bytesCurrentlyInMemory.get(), + tuningConfig.getMaxBytesInMemory() + )); + } + if (persist) { if (allowIncrementalPersists) { // persistAll clears rowsCurrentlyInMemory, no need to update it. + log.info("Persisting rows in memory due to: [%s]", persistReason.toString()); persistAll(committerSupplier == null ? null : committerSupplier.get()); } else { isPersistRequired = true; @@ -294,9 +323,9 @@ int getRowsInMemory() } @VisibleForTesting - long getCurrentBytesInMemory() + long getBytesCurrentlyInMemory() { - return currentBytesInMemory.get(); + return bytesCurrentlyInMemory.get(); } @VisibleForTesting @@ -523,7 +552,7 @@ public Object doCall() // NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes. rowsCurrentlyInMemory.addAndGet(-numPersistedRows); - currentBytesInMemory.addAndGet(-bytesPersisted); + bytesCurrentlyInMemory.addAndGet(-bytesPersisted); return future; } @@ -1063,7 +1092,7 @@ private ListenableFuture abandonSegment( // Decrement this sink's rows from rowsCurrentlyInMemory (we only count active sinks). rowsCurrentlyInMemory.addAndGet(-sink.getNumRowsInMemory()); totalRows.addAndGet(-sink.getNumRows()); - currentBytesInMemory.addAndGet(-sink.getBytesInMemory()); + bytesCurrentlyInMemory.addAndGet(-sink.getBytesInMemory()); // Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread. return Futures.transform( diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index 6936b0272200..39f03fcf629c 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -179,8 +179,6 @@ public boolean isEmpty() } } - //add methods for byte mem checks - public boolean isWritable() { return writable; @@ -267,7 +265,7 @@ public long getBytesInMemory() return 0; } - return currHydrant.getIndex().sizeInBytes(); + return currHydrant.getIndex().getBytesInMemory(); } } diff --git a/server/src/test/java/io/druid/segment/indexing/RealtimeTuningConfigTest.java b/server/src/test/java/io/druid/segment/indexing/RealtimeTuningConfigTest.java index 12878424309c..5559e1163fe1 100644 --- a/server/src/test/java/io/druid/segment/indexing/RealtimeTuningConfigTest.java +++ b/server/src/test/java/io/druid/segment/indexing/RealtimeTuningConfigTest.java @@ -71,7 +71,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); Assert.assertEquals(NoneShardSpec.instance(), config.getShardSpec()); Assert.assertEquals(0, config.getMaxPendingPersists()); - Assert.assertEquals(75000, config.getMaxRowsInMemory()); + Assert.assertEquals(1000000, config.getMaxRowsInMemory()); Assert.assertEquals(0, config.getMergeThreadPriority()); Assert.assertEquals(0, config.getPersistThreadPriority()); Assert.assertEquals(new Period("PT10M"), config.getWindowPeriod()); diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java index 9dfb269abbc5..f18133141173 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java @@ -208,9 +208,9 @@ public void run() appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 10 (dimsKeySize) = 138 - Assert.assertEquals(138, ((AppenderatorImpl) appenderator).getCurrentBytesInMemory()); + Assert.assertEquals(138, ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()); appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); - Assert.assertEquals(276, ((AppenderatorImpl) appenderator).getCurrentBytesInMemory()); + Assert.assertEquals(276, ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()); appenderator.close(); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); } @@ -249,7 +249,7 @@ public void run() Assert.assertEquals(138, ((AppenderatorImpl) appenderator).getBytesInMemory(IDENTIFIERS.get(0))); Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); - Assert.assertEquals(276, ((AppenderatorImpl) appenderator).getCurrentBytesInMemory()); + Assert.assertEquals(276, ((AppenderatorImpl) appenderator).getBytesCurrentlyInMemory()); Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.close(); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); From 813a261cef7cfdd1db6ef75ee602d51304dd3767 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Fri, 13 Apr 2018 11:13:55 -0700 Subject: [PATCH 19/31] Fix TeamCity inspection warnings --- .../java/io/druid/segment/incremental/IncrementalIndex.java | 4 ++-- .../io/druid/segment/incremental/OnheapIncrementalIndex.java | 2 +- .../src/main/java/io/druid/segment/indexing/TuningConfig.java | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 072cdf3f1757..b120e680148a 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -520,7 +520,7 @@ public List getParseExceptionMessages() public static class AddToFactsResult { private int rowCount; - private long bytesInMemory; + private final long bytesInMemory; private List parseExceptionMessages; public AddToFactsResult( @@ -1171,7 +1171,7 @@ private TimeAndDims( this.dimsKeySize = dimsKeySize; } - public static TimeAndDims createTimeAndDimswithDimsKeySize( + static TimeAndDims createTimeAndDimswithDimsKeySize( long timestamp, Object[] dims, List dimensionDescsList, diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 03304ce7d251..058b1ec0fbd5 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -81,7 +81,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex this.maxBytesInMemory = maxBytesInMemory == 0 ? -1 : maxBytesInMemory; this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) : new PlainFactsHolder(sortFacts); - maxBytesPerRowForAggregators = this.getMaxBytesPerRowForAggregators(incrementalIndexSchema); + maxBytesPerRowForAggregators = getMaxBytesPerRowForAggregators(incrementalIndexSchema); } diff --git a/server/src/main/java/io/druid/segment/indexing/TuningConfig.java b/server/src/main/java/io/druid/segment/indexing/TuningConfig.java index e4d3d362f177..5ada9ff90f70 100644 --- a/server/src/main/java/io/druid/segment/indexing/TuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/TuningConfig.java @@ -33,6 +33,6 @@ public interface TuningConfig boolean DEFAULT_LOG_PARSE_EXCEPTIONS = false; int DEFAULT_MAX_PARSE_EXCEPTIONS = Integer.MAX_VALUE; int DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS = 0; - static final int DEFAULT_MAX_ROWS_IN_MEMORY = 1_000_000; - static final long DEFAULT_MAX_BYTES_IN_MEMORY = Runtime.getRuntime().maxMemory() / 3; + int DEFAULT_MAX_ROWS_IN_MEMORY = 1_000_000; + long DEFAULT_MAX_BYTES_IN_MEMORY = Runtime.getRuntime().maxMemory() / 3; } From ec24d3acd01aa9e6f4c7533165d4a8e834289936 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Fri, 13 Apr 2018 15:33:00 -0700 Subject: [PATCH 20/31] Added maxBytesInMemory config to HadoopTuningConfig --- .../data/input/orc/OrcIndexGeneratorJobTest.java | 1 + .../io/druid/indexer/HadoopTuningConfig.java | 16 +++++++++++++++- .../java/io/druid/indexer/IndexGeneratorJob.java | 1 + .../druid/indexer/BatchDeltaIngestionTest.java | 1 + .../DetermineHashedPartitionsJobTest.java | 1 + .../indexer/DeterminePartitionsJobTest.java | 1 + .../indexer/HadoopDruidIndexerConfigTest.java | 2 ++ .../io/druid/indexer/HadoopTuningConfigTest.java | 1 + .../io/druid/indexer/IndexGeneratorJobTest.java | 12 +++++++++++- .../java/io/druid/indexer/JobHelperTest.java | 1 + .../indexer/path/GranularityPathSpecTest.java | 1 + .../indexer/updater/HadoopConverterJobTest.java | 1 + 12 files changed, 37 insertions(+), 2 deletions(-) diff --git a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java index bf5383b5fbf6..6cb6fe9b1584 100644 --- a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java @@ -222,6 +222,7 @@ public void setUp() throws Exception null, null, null, + null, false, false, false, diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java index a997e40d2994..c2826879b8d7 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java @@ -43,7 +43,8 @@ public class HadoopTuningConfig implements TuningConfig private static final PartitionsSpec DEFAULT_PARTITIONS_SPEC = HashedPartitionsSpec.makeDefaultHashedPartitionsSpec(); private static final Map> DEFAULT_SHARD_SPECS = ImmutableMap.of(); private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); - private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 75000; + private static final int DEFAULT_ROW_FLUSH_BOUNDARY = TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY; + private static final long DEFAULT_MAX_BYTES_IN_MEMORY = TuningConfig.DEFAULT_MAX_BYTES_IN_MEMORY; private static final boolean DEFAULT_USE_COMBINER = false; private static final int DEFAULT_NUM_BACKGROUND_PERSIST_THREADS = 0; @@ -56,6 +57,7 @@ public static HadoopTuningConfig makeDefaultTuningConfig() DEFAULT_SHARD_SPECS, DEFAULT_INDEX_SPEC, DEFAULT_ROW_FLUSH_BOUNDARY, + DEFAULT_MAX_BYTES_IN_MEMORY, false, true, false, @@ -80,6 +82,7 @@ public static HadoopTuningConfig makeDefaultTuningConfig() private final Map> shardSpecs; private final IndexSpec indexSpec; private final int rowFlushBoundary; + private final long maxBytesInMemory; private final boolean leaveIntermediate; private final Boolean cleanupOnFailure; private final boolean overwriteFiles; @@ -102,6 +105,7 @@ public HadoopTuningConfig( final @JsonProperty("shardSpecs") Map> shardSpecs, final @JsonProperty("indexSpec") IndexSpec indexSpec, final @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + final @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, final @JsonProperty("leaveIntermediate") boolean leaveIntermediate, final @JsonProperty("cleanupOnFailure") Boolean cleanupOnFailure, final @JsonProperty("overwriteFiles") boolean overwriteFiles, @@ -129,6 +133,7 @@ public HadoopTuningConfig( this.rowFlushBoundary = maxRowsInMemory == null ? maxRowsInMemoryCOMPAT == null ? DEFAULT_ROW_FLUSH_BOUNDARY : maxRowsInMemoryCOMPAT : maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory == null ? DEFAULT_MAX_BYTES_IN_MEMORY : maxBytesInMemory; this.leaveIntermediate = leaveIntermediate; this.cleanupOnFailure = cleanupOnFailure == null ? true : cleanupOnFailure; this.overwriteFiles = overwriteFiles; @@ -190,6 +195,12 @@ public int getRowFlushBoundary() return rowFlushBoundary; } + @JsonProperty + public long getMaxBytesInMemory() + { + return maxBytesInMemory; + } + @JsonProperty public boolean isLeaveIntermediate() { @@ -288,6 +299,7 @@ public HadoopTuningConfig withWorkingPath(String path) shardSpecs, indexSpec, rowFlushBoundary, + maxBytesInMemory, leaveIntermediate, cleanupOnFailure, overwriteFiles, @@ -315,6 +327,7 @@ public HadoopTuningConfig withVersion(String ver) shardSpecs, indexSpec, rowFlushBoundary, + maxBytesInMemory, leaveIntermediate, cleanupOnFailure, overwriteFiles, @@ -342,6 +355,7 @@ public HadoopTuningConfig withShardSpecs(Map> specs specs, indexSpec, rowFlushBoundary, + maxBytesInMemory, leaveIntermediate, cleanupOnFailure, overwriteFiles, diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index b5708b94354c..80a23e71f83f 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -288,6 +288,7 @@ private static IncrementalIndex makeIncrementalIndex( .setIndexSchema(indexSchema) .setReportParseExceptions(!tuningConfig.isIgnoreInvalidRows()) .setMaxRowCount(tuningConfig.getRowFlushBoundary()) + .setMaxBytesInMemory(tuningConfig.getMaxBytesInMemory()) .buildOnheap(); if (oldDimOrder != null && !indexSchema.getDimensionsSpec().hasCustomDimensions()) { diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java index b7bb444c1cc7..c011c73bc0d4 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java @@ -480,6 +480,7 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig( null, null, null, + null, false, false, false, diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java index 8d656f20d4b8..7f1e76f547e9 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java @@ -198,6 +198,7 @@ public DetermineHashedPartitionsJobTest( null, null, null, + null, false, false, false, diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java index 908425198783..f96c5957db08 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java @@ -259,6 +259,7 @@ public DeterminePartitionsJobTest( null, null, null, + null, false, false, false, diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java index 2ed052eafec9..6678b13d075c 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -84,6 +84,7 @@ public void testHashedBucketSelection() ImmutableMap.of(DateTimes.of("2010-01-01T01:00:00").getMillis(), specs), null, null, + null, false, false, false, @@ -160,6 +161,7 @@ public void testNoneShardSpecBucketSelection() ), null, null, + null, false, false, false, diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java index 9e4a26a22b43..b389d8ecc91c 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java @@ -46,6 +46,7 @@ public void testSerde() throws Exception null, null, 100, + null, true, true, true, diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java index 1b422b6c223e..af1d7d944b74 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java @@ -91,7 +91,7 @@ public class IndexGeneratorJobTest @Parameterized.Parameters(name = "useCombiner={0}, partitionType={1}, interval={2}, shardInfoForEachSegment={3}, " + "data={4}, inputFormatName={5}, inputRowParser={6}, maxRowsInMemory={7}, " + - "aggs={8}, datasourceName={9}, forceExtendableShardSpecs={10}") + "maxBytesInMemory={8}, aggs={9}, datasourceName={10}, forceExtendableShardSpecs={11}") public static Collection constructFeed() { final List baseConstructors = Arrays.asList( @@ -151,6 +151,7 @@ public static Collection constructFeed() null ), null, + null, aggs1, "website" }, @@ -198,6 +199,7 @@ public static Collection constructFeed() ) ), null, + null, aggs1, "website" }, @@ -246,6 +248,7 @@ public static Collection constructFeed() null ), null, + null, aggs1, "website" }, @@ -303,6 +306,7 @@ public static Collection constructFeed() ) ), null, + null, aggs1, "website" }, @@ -335,6 +339,7 @@ public static Collection constructFeed() null ), 1, // force 1 row max per index for easier testing + null, aggs2, "inherit_dims" }, @@ -367,6 +372,7 @@ public static Collection constructFeed() null ), 1, // force 1 row max per index for easier testing + null, aggs2, "inherit_dims2" } @@ -398,6 +404,7 @@ public static Collection constructFeed() private final String inputFormatName; private final InputRowParser inputRowParser; private final Integer maxRowsInMemory; + private final Long maxBytesInMemory; private final AggregatorFactory[] aggs; private final String datasourceName; private final boolean forceExtendableShardSpecs; @@ -416,6 +423,7 @@ public IndexGeneratorJobTest( String inputFormatName, InputRowParser inputRowParser, Integer maxRowsInMemory, + Long maxBytesInMemory, AggregatorFactory[] aggs, String datasourceName, boolean forceExtendableShardSpecs @@ -429,6 +437,7 @@ public IndexGeneratorJobTest( this.inputFormatName = inputFormatName; this.inputRowParser = inputRowParser; this.maxRowsInMemory = maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory; this.aggs = aggs; this.datasourceName = datasourceName; this.forceExtendableShardSpecs = forceExtendableShardSpecs; @@ -511,6 +520,7 @@ public void setUp() throws Exception null, null, maxRowsInMemory, + maxBytesInMemory, false, false, false, diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java index c768e2c8e10d..a7613249d050 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java @@ -108,6 +108,7 @@ public void setup() throws Exception null, null, null, + null, false, false, false, diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java index b4caeed21f43..a24d2615dbcb 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java @@ -61,6 +61,7 @@ public class GranularityPathSpecTest null, null, null, + null, false, false, false, diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java index c8d763544340..e673c3e661e7 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java @@ -200,6 +200,7 @@ public InputStream openStream() throws IOException null, null, null, + null, false, false, false, From be9a0c17f8ec8ff683e5bd46d368290c906fb425 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Mon, 16 Apr 2018 10:41:29 -0700 Subject: [PATCH 21/31] Updated the docs and examples * Added maxBytesInMemory config in docs * Removed references to maxRowsInMemory under tuningConfig in examples --- docs/content/development/extensions-core/kafka-ingestion.md | 3 ++- docs/content/ingestion/batch-ingestion.md | 3 ++- docs/content/ingestion/stream-pull.md | 6 ++++-- docs/content/ingestion/tasks.md | 5 +++-- examples/conf-quickstart/tranquility/kafka.json | 1 - examples/conf-quickstart/tranquility/server.json | 1 - examples/conf/tranquility/kafka.json | 1 - examples/conf/tranquility/server.json | 1 - 8 files changed, 11 insertions(+), 10 deletions(-) diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index 2c09b64e15f0..8ac0730a39e5 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -115,7 +115,8 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |Field|Type|Description|Required| |-----|----|-----------|--------| |`type`|String|The indexing task type, this should always be `kafka`.|yes| -|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 75000)| +|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 1000000)| +|`maxBytesInMemory`|Long|The maximum number of bytes to keep in memory to aggregate before persisting. This is used to manage the required JVM heap size. |no (default == One-third of max JVM memory)| |`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == 5000000)| |`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)| |`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)| diff --git a/docs/content/ingestion/batch-ingestion.md b/docs/content/ingestion/batch-ingestion.md index 743ead5b071e..29358cd8b196 100644 --- a/docs/content/ingestion/batch-ingestion.md +++ b/docs/content/ingestion/batch-ingestion.md @@ -154,7 +154,8 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |workingPath|String|The working path to use for intermediate results (results between Hadoop jobs).|no (default == '/tmp/druid-indexing')| |version|String|The version of created segments. Ignored for HadoopIndexTask unless useExplicitVersion is set to true|no (default == datetime that indexing starts at)| |partitionsSpec|Object|A specification of how to partition each time bucket into segments. Absence of this property means no partitioning will occur. See 'Partitioning specification' below.|no (default == 'hashed')| -|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. Note that this is the number of post-aggregation rows which may not be equal to the number of input events due to roll-up. This is used to manage the required JVM heap size.|no (default == 75000)| +|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. Note that this is the number of post-aggregation rows which may not be equal to the number of input events due to roll-up. This is used to manage the required JVM heap size.|no (default == 1000000)| +|maxBytesInMemory|Long|The maximum number of bytes to keep in memory to aggregate before persisting. This is used to manage the required JVM heap size.|no (default == One-third of max JVM memory)| |leaveIntermediate|Boolean|Leave behind intermediate files (for debugging) in the workingPath when a job completes, whether it passes or fails.|no (default == false)| |cleanupOnFailure|Boolean|Clean up intermediate files when a job fails (unless leaveIntermediate is on).|no (default == true)| |overwriteFiles|Boolean|Override existing files found during indexing.|no (default == false)| diff --git a/docs/content/ingestion/stream-pull.md b/docs/content/ingestion/stream-pull.md index e57975276871..863f287a19e2 100644 --- a/docs/content/ingestion/stream-pull.md +++ b/docs/content/ingestion/stream-pull.md @@ -92,7 +92,7 @@ The property `druid.realtime.specFile` has the path of a file (absolute or relat }, "tuningConfig": { "type" : "realtime", - "maxRowsInMemory": 75000, + "maxRowsInMemory": 1000000, "intermediatePersistPeriod": "PT10m", "windowPeriod": "PT10m", "basePersistDirectory": "\/tmp\/realtime\/basePersist", @@ -141,7 +141,8 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |Field|Type|Description|Required| |-----|----|-----------|--------| |type|String|This should always be 'realtime'.|no| -|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 75000)| +|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 1000000)| +|maxBytesInMemory|Long|The maximum number of bytes to keep in memory to aggregate before persisting. This is used to manage the required JVM heap size.|no (default == One-third of max JVM memory)| |windowPeriod|ISO 8601 Period String|The amount of lag time to allow events. This is configured with a 10 minute window, meaning that any event more than 10 minutes ago will be thrown away and not included in the segment generated by the realtime server.|no (default == PT10m)| |intermediatePersistPeriod|ISO8601 Period String|The period that determines the rate at which intermediate persists occur. These persists determine how often commits happen against the incoming realtime stream. If the realtime data loading process is interrupted at time T, it should be restarted to re-read data that arrived at T minus this period.|no (default == PT10m)| |basePersistDirectory|String|The directory to put things that need persistence. The plumber is responsible for the actual intermediate persists and this tells it where to store those persists.|no (default == java tmp dir)| @@ -288,6 +289,7 @@ The following table summarizes constraints between settings in the spec file for |queryGranularity| Time granularity (minute, hour, day, week, month) for rollup | less than segmentGranularity| minute, hour, day, week, month | |intermediatePersistPeriod| The max time (ISO8601 Period) between flushes of ingested rows from memory to disk | avoid excessive flushing | number of un-persisted rows in memory also constrained by maxRowsInMemory | |maxRowsInMemory| The max number of ingested rows to hold in memory before a flush to disk | number of un-persisted post-aggregation rows in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod | +|maxBytesInMemory| The maximum number of bytes to keep in memory before a flush to disk | number of un-persisted post-aggregation bytes in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod | The normal, expected use cases have the following overall constraints: `intermediatePersistPeriod ≤ windowPeriod < segmentGranularity` and `queryGranularity ≤ segmentGranularity` diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index 4d424efcadb4..61503bf3d523 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -77,7 +77,7 @@ The Index Task is a simpler variation of the Index Hadoop task that is designed "tuningConfig" : { "type" : "index", "targetPartitionSize" : 5000000, - "maxRowsInMemory" : 75000 + "maxRowsInMemory" : 1000000 } } } @@ -137,7 +137,8 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |--------|-----------|-------|---------| |type|The task type, this should always be "index".|none|yes| |targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no| -|maxRowsInMemory|Used in determining when intermediate persists to disk should occur.|75000|no| +|maxRowsInMemory|Used in determining when intermediate persists to disk should occur.|1000000|no| +|maxBytesInMemory|Used in determining when intermediate persists to disk should occur.|1/3 of max JVM memory|no| |maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|150000|no| |numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no| |indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| diff --git a/examples/conf-quickstart/tranquility/kafka.json b/examples/conf-quickstart/tranquility/kafka.json index ffe537870d56..38858d257845 100644 --- a/examples/conf-quickstart/tranquility/kafka.json +++ b/examples/conf-quickstart/tranquility/kafka.json @@ -53,7 +53,6 @@ }, "tuningConfig" : { "type" : "realtime", - "maxRowsInMemory" : "100000", "intermediatePersistPeriod" : "PT10M", "windowPeriod" : "PT10M" } diff --git a/examples/conf-quickstart/tranquility/server.json b/examples/conf-quickstart/tranquility/server.json index cbc1d14e7724..a17f71665502 100644 --- a/examples/conf-quickstart/tranquility/server.json +++ b/examples/conf-quickstart/tranquility/server.json @@ -53,7 +53,6 @@ }, "tuningConfig" : { "type" : "realtime", - "maxRowsInMemory" : "100000", "intermediatePersistPeriod" : "PT10M", "windowPeriod" : "PT10M" } diff --git a/examples/conf/tranquility/kafka.json b/examples/conf/tranquility/kafka.json index b0adb3248931..fb7c9aeb4ca3 100644 --- a/examples/conf/tranquility/kafka.json +++ b/examples/conf/tranquility/kafka.json @@ -53,7 +53,6 @@ }, "tuningConfig" : { "type" : "realtime", - "maxRowsInMemory" : "100000", "intermediatePersistPeriod" : "PT10M", "windowPeriod" : "PT10M" } diff --git a/examples/conf/tranquility/server.json b/examples/conf/tranquility/server.json index dabeed18228b..ff810f72fbb4 100644 --- a/examples/conf/tranquility/server.json +++ b/examples/conf/tranquility/server.json @@ -53,7 +53,6 @@ }, "tuningConfig" : { "type" : "realtime", - "maxRowsInMemory" : "100000", "intermediatePersistPeriod" : "PT10M", "windowPeriod" : "PT10M" } From 1cc919477049b94ea0c50551cb26add0fd60b036 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Sun, 22 Apr 2018 17:04:17 -0700 Subject: [PATCH 22/31] Set maxBytesInMemory to 0 until used Set the maxBytesInMemory to 0 if user does not set it as part of tuningConfing and set to part of max jvm memory when ingestion task starts --- .../indexing/kafka/KafkaTuningConfig.java | 2 +- .../io/druid/indexer/HadoopTuningConfig.java | 3 +- .../io/druid/indexer/IndexGeneratorJob.java | 3 +- .../RealtimeAppenderatorTuningConfig.java | 2 +- .../common/index/YeOldePlumberSchool.java | 3 +- .../druid/indexing/common/task/IndexTask.java | 4 +-- .../druid/segment/indexing/TuningConfigs.java | 34 +++++++++++++++++++ .../appenderator/AppenderatorImpl.java | 7 ++-- .../realtime/plumber/RealtimePlumber.java | 5 +-- .../segment/realtime/RealtimeManagerTest.java | 5 +-- .../plumber/RealtimePlumberSchoolTest.java | 7 ++-- .../segment/realtime/plumber/SinkTest.java | 3 +- 12 files changed, 59 insertions(+), 19 deletions(-) create mode 100644 server/src/main/java/io/druid/segment/indexing/TuningConfigs.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index e61cc979d215..ead3d92247c6 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -82,7 +82,7 @@ public KafkaTuningConfig( this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; this.maxRowsPerSegment = maxRowsPerSegment == null ? DEFAULT_MAX_ROWS_PER_SEGMENT : maxRowsPerSegment; - this.maxBytesInMemory = maxBytesInMemory == null ? defaults.getMaxBytesInMemory() : maxBytesInMemory; + this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaults.getIntermediatePersistPeriod() : intermediatePersistPeriod; diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java index c2826879b8d7..8386e4a837f4 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java @@ -44,7 +44,6 @@ public class HadoopTuningConfig implements TuningConfig private static final Map> DEFAULT_SHARD_SPECS = ImmutableMap.of(); private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); private static final int DEFAULT_ROW_FLUSH_BOUNDARY = TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY; - private static final long DEFAULT_MAX_BYTES_IN_MEMORY = TuningConfig.DEFAULT_MAX_BYTES_IN_MEMORY; private static final boolean DEFAULT_USE_COMBINER = false; private static final int DEFAULT_NUM_BACKGROUND_PERSIST_THREADS = 0; @@ -133,7 +132,7 @@ public HadoopTuningConfig( this.rowFlushBoundary = maxRowsInMemory == null ? maxRowsInMemoryCOMPAT == null ? DEFAULT_ROW_FLUSH_BOUNDARY : maxRowsInMemoryCOMPAT : maxRowsInMemory; - this.maxBytesInMemory = maxBytesInMemory == null ? DEFAULT_MAX_BYTES_IN_MEMORY : maxBytesInMemory; + this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.leaveIntermediate = leaveIntermediate; this.cleanupOnFailure = cleanupOnFailure == null ? true : cleanupOnFailure; this.overwriteFiles = overwriteFiles; diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index 80a23e71f83f..da61e0648eff 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -51,6 +51,7 @@ import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.indexing.TuningConfigs; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.ShardSpec; @@ -288,7 +289,7 @@ private static IncrementalIndex makeIncrementalIndex( .setIndexSchema(indexSchema) .setReportParseExceptions(!tuningConfig.isIgnoreInvalidRows()) .setMaxRowCount(tuningConfig.getRowFlushBoundary()) - .setMaxBytesInMemory(tuningConfig.getMaxBytesInMemory()) + .setMaxBytesInMemory(TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory())) .buildOnheap(); if (oldDimOrder != null && !indexSchema.getDimensionsSpec().hasCustomDimensions()) { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index b20de38f29d6..11372f2bded1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -94,7 +94,7 @@ public RealtimeAppenderatorTuningConfig( { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; this.maxRowsPerSegment = maxRowsPerSegment == null ? defaultMaxRowsPerSegment : maxRowsPerSegment; - this.maxBytesInMemory = maxBytesInMemory == null ? defaultMaxBytesInMemory : maxBytesInMemory; + this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaultIntermediatePersistPeriod : intermediatePersistPeriod; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java index 536500bd191e..d63f5e7ba889 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java @@ -43,6 +43,7 @@ import io.druid.segment.incremental.IndexSizeExceededException; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.TuningConfigs; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.FireHydrant; @@ -105,7 +106,7 @@ public Plumber findPlumber( config.getShardSpec(), version, config.getMaxRowsInMemory(), - config.getMaxBytesInMemory(), + TuningConfigs.getMaxBytesInMemoryOrDefault(config.getMaxBytesInMemory()), config.isReportParseExceptions() ); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index a652d38a4200..c025abd1be50 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -1297,7 +1297,7 @@ public IndexTuningConfig( this( targetPartitionSize, maxRowsInMemory != null ? maxRowsInMemory : rowFlushBoundary_forBackCompatibility, - maxBytesInMemory != null ? maxBytesInMemory : TuningConfig.DEFAULT_MAX_BYTES_IN_MEMORY, + maxBytesInMemory != null ? maxBytesInMemory : 0, maxTotalRows, numShards, indexSpec, @@ -1345,7 +1345,7 @@ private IndexTuningConfig( this.targetPartitionSize = initializeTargetPartitionSize(numShards, targetPartitionSize); this.maxRowsInMemory = maxRowsInMemory == null ? TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - this.maxBytesInMemory = maxBytesInMemory == null ? TuningConfig.DEFAULT_MAX_BYTES_IN_MEMORY : maxBytesInMemory; + this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.maxTotalRows = initializeMaxTotalRows(numShards, maxTotalRows); this.numShards = numShards == null || numShards.equals(-1) ? null : numShards; this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; diff --git a/server/src/main/java/io/druid/segment/indexing/TuningConfigs.java b/server/src/main/java/io/druid/segment/indexing/TuningConfigs.java new file mode 100644 index 000000000000..571005f4370c --- /dev/null +++ b/server/src/main/java/io/druid/segment/indexing/TuningConfigs.java @@ -0,0 +1,34 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.indexing; + +public class TuningConfigs +{ + private TuningConfigs() + { + } + + public static long getMaxBytesInMemoryOrDefault(final long maxBytesInMemory) + { + // In the main tuningConfig class constructor, we set the maxBytes to 0 if null to avoid setting + // maxBytes to max jvm memory of the process that starts first. Instead we set the default based on + // the actual task node's jvm memory. + return maxBytesInMemory == 0 ? TuningConfig.DEFAULT_MAX_BYTES_IN_MEMORY : maxBytesInMemory; + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index f2dcd9dd80ea..2941aef3d9d7 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -66,6 +66,7 @@ import io.druid.segment.incremental.IncrementalIndexAddResult; import io.druid.segment.incremental.IndexSizeExceededException; import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.TuningConfigs; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.FireHydrant; @@ -271,7 +272,7 @@ public AppenderatorAddResult add( tuningConfig.getMaxRowsInMemory() )); } - if (tuningConfig.getMaxBytesInMemory() > 0 && bytesCurrentlyInMemory.get() >= tuningConfig.getMaxBytesInMemory()) { + if (tuningConfig.getMaxBytesInMemory() != -1 && bytesCurrentlyInMemory.get() >= tuningConfig.getMaxBytesInMemory()) { persist = true; persistReason.append(StringUtils.format( " bytesCurrentlyInMemory[%d] is greater than maxBytesInMemory[%d]", @@ -351,7 +352,7 @@ private Sink getOrCreateSink(final SegmentIdentifier identifier) identifier.getShardSpec(), identifier.getVersion(), tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemory(), + TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions() ); @@ -1035,7 +1036,7 @@ public int compare(File o1, File o2) identifier.getShardSpec(), identifier.getVersion(), tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemory(), + TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions(), hydrants ); diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 235f6067e789..62b4d71cb2ff 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -65,6 +65,7 @@ import io.druid.segment.incremental.IndexSizeExceededException; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.TuningConfigs; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.FireHydrant; @@ -255,7 +256,7 @@ private Sink getSink(long timestamp) config.getShardSpec(), versioningPolicy.getVersion(sinkInterval), config.getMaxRowsInMemory(), - config.getMaxBytesInMemory(), + TuningConfigs.getMaxBytesInMemoryOrDefault(config.getMaxBytesInMemory()), config.isReportParseExceptions() ); addSink(retVal); @@ -737,7 +738,7 @@ public int compare(File o1, File o2) config.getShardSpec(), versioningPolicy.getVersion(sinkInterval), config.getMaxRowsInMemory(), - config.getMaxBytesInMemory(), + TuningConfigs.getMaxBytesInMemoryOrDefault(config.getMaxBytesInMemory()), config.isReportParseExceptions(), hydrants ); diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 0f2c903f1c1f..4d98bbccddab 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -69,6 +69,7 @@ import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.TuningConfigs; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.realtime.plumber.Plumber; import io.druid.segment.realtime.plumber.PlumberSchool; @@ -222,7 +223,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws ParseExcept tuningConfig.getShardSpec(), DateTimes.nowUtc().toString(), tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemory(), + TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions() )); @@ -243,7 +244,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws ParseExcept tuningConfig.getShardSpec(), DateTimes.nowUtc().toString(), tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemory(), + TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions() )); diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index f75cccfb1382..10e5bd1085a2 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -50,6 +50,7 @@ import io.druid.segment.TestHelper; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.TuningConfigs; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.FireDepartmentMetrics; @@ -270,7 +271,7 @@ private void testPersist(final Object commitMetadata) throws Exception tuningConfig.getShardSpec(), DateTimes.of("2014-12-01T12:34:56.789").toString(), tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemory(), + TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions() ); plumber.getSinks().put(0L, sink); @@ -315,7 +316,7 @@ public void testPersistFails() throws Exception tuningConfig.getShardSpec(), DateTimes.of("2014-12-01T12:34:56.789").toString(), tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemory(), + TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions() ); plumber.getSinks().put(0L, sink); @@ -370,7 +371,7 @@ private void testPersistHydrantGapsHelper(final Object commitMetadata) throws Ex tuningConfig.getShardSpec(), DateTimes.of("2014-12-01T12:34:56.789").toString(), tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemory(), + TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions() ); plumber2.getSinks().put(0L, sink); diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index f9bf8f03e9ba..a8bf1be421e2 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -31,6 +31,7 @@ import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.TuningConfigs; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.realtime.FireHydrant; import org.joda.time.DateTime; @@ -84,7 +85,7 @@ public void testSwap() throws Exception tuningConfig.getShardSpec(), version, tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemory(), + TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions() ); From 78227218c1f6ed58ab37abe87cc2c84723d6d8df Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Mon, 23 Apr 2018 13:41:06 -0700 Subject: [PATCH 23/31] Update toString in KafkaSupervisorTuningConfig --- .../indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 71a2a801baed..4fb2b2409b02 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.indexing.kafka.KafkaTuningConfig; +import io.druid.segment.indexing.TuningConfigs; import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexSpec; import org.joda.time.Duration; @@ -133,7 +134,7 @@ public String toString() return "KafkaSupervisorTuningConfig{" + "maxRowsInMemory=" + getMaxRowsInMemory() + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + - ", maxBytesInMemory=" + getMaxBytesInMemory() + + ", maxBytesInMemory=" + TuningConfigs.getMaxBytesInMemoryOrDefault(getMaxBytesInMemory()) + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + ", maxPendingPersists=" + getMaxPendingPersists() + From dcea72bff9d38c5c30720e47a8b7fdf38089ac75 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Tue, 24 Apr 2018 13:50:14 -0700 Subject: [PATCH 24/31] Use correct maxBytesInMemory value in AppenderatorImpl --- .../segment/realtime/appenderator/AppenderatorImpl.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 2941aef3d9d7..b7951598ec00 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -146,7 +146,7 @@ public class AppenderatorImpl implements Appenderator private volatile FileChannel basePersistDirLockChannel = null; private AtomicBoolean closed = new AtomicBoolean(false); - public AppenderatorImpl( + AppenderatorImpl( DataSchema schema, AppenderatorConfig tuningConfig, FireDepartmentMetrics metrics, @@ -272,12 +272,14 @@ public AppenderatorAddResult add( tuningConfig.getMaxRowsInMemory() )); } - if (tuningConfig.getMaxBytesInMemory() != -1 && bytesCurrentlyInMemory.get() >= tuningConfig.getMaxBytesInMemory()) { + if (tuningConfig.getMaxBytesInMemory() != -1 + && bytesCurrentlyInMemory.get() + >= TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory())) { persist = true; persistReason.append(StringUtils.format( " bytesCurrentlyInMemory[%d] is greater than maxBytesInMemory[%d]", bytesCurrentlyInMemory.get(), - tuningConfig.getMaxBytesInMemory() + TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()) )); } if (persist) { From 98ec6948b090cb1e790f62beb35399e37175bf84 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Wed, 25 Apr 2018 17:56:36 -0700 Subject: [PATCH 25/31] Update DEFAULT_MAX_BYTES_IN_MEMORY to 1/6 max jvm memory Experimenting with various defaults, 1/3 jvm memory causes OOM --- .../src/main/java/io/druid/segment/indexing/TuningConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/segment/indexing/TuningConfig.java b/server/src/main/java/io/druid/segment/indexing/TuningConfig.java index 5ada9ff90f70..e8f3d171d3c6 100644 --- a/server/src/main/java/io/druid/segment/indexing/TuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/TuningConfig.java @@ -34,5 +34,5 @@ public interface TuningConfig int DEFAULT_MAX_PARSE_EXCEPTIONS = Integer.MAX_VALUE; int DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS = 0; int DEFAULT_MAX_ROWS_IN_MEMORY = 1_000_000; - long DEFAULT_MAX_BYTES_IN_MEMORY = Runtime.getRuntime().maxMemory() / 3; + long DEFAULT_MAX_BYTES_IN_MEMORY = Runtime.getRuntime().maxMemory() / 6; } From 7b11f21d186fbba5868f49ec43750ebc0ad7a52a Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Thu, 26 Apr 2018 09:58:31 -0700 Subject: [PATCH 26/31] Update docs to correct maxBytesInMemory default value --- docs/content/development/extensions-core/kafka-ingestion.md | 2 +- docs/content/ingestion/batch-ingestion.md | 2 +- docs/content/ingestion/stream-pull.md | 2 +- docs/content/ingestion/tasks.md | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index 8ac0730a39e5..d1ded0ae5d9b 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -116,7 +116,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |-----|----|-----------|--------| |`type`|String|The indexing task type, this should always be `kafka`.|yes| |`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 1000000)| -|`maxBytesInMemory`|Long|The maximum number of bytes to keep in memory to aggregate before persisting. This is used to manage the required JVM heap size. |no (default == One-third of max JVM memory)| +|`maxBytesInMemory`|Long|The maximum number of bytes to keep in memory to aggregate before persisting. This is used to manage the required JVM heap size. |no (default == One-sixth of max JVM memory)| |`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == 5000000)| |`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)| |`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)| diff --git a/docs/content/ingestion/batch-ingestion.md b/docs/content/ingestion/batch-ingestion.md index 29358cd8b196..c4192b34a18e 100644 --- a/docs/content/ingestion/batch-ingestion.md +++ b/docs/content/ingestion/batch-ingestion.md @@ -155,7 +155,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |version|String|The version of created segments. Ignored for HadoopIndexTask unless useExplicitVersion is set to true|no (default == datetime that indexing starts at)| |partitionsSpec|Object|A specification of how to partition each time bucket into segments. Absence of this property means no partitioning will occur. See 'Partitioning specification' below.|no (default == 'hashed')| |maxRowsInMemory|Integer|The number of rows to aggregate before persisting. Note that this is the number of post-aggregation rows which may not be equal to the number of input events due to roll-up. This is used to manage the required JVM heap size.|no (default == 1000000)| -|maxBytesInMemory|Long|The maximum number of bytes to keep in memory to aggregate before persisting. This is used to manage the required JVM heap size.|no (default == One-third of max JVM memory)| +|maxBytesInMemory|Long|The maximum number of bytes to keep in memory to aggregate before persisting. This is used to manage the required JVM heap size.|no (default == One-sixth of max JVM memory)| |leaveIntermediate|Boolean|Leave behind intermediate files (for debugging) in the workingPath when a job completes, whether it passes or fails.|no (default == false)| |cleanupOnFailure|Boolean|Clean up intermediate files when a job fails (unless leaveIntermediate is on).|no (default == true)| |overwriteFiles|Boolean|Override existing files found during indexing.|no (default == false)| diff --git a/docs/content/ingestion/stream-pull.md b/docs/content/ingestion/stream-pull.md index 863f287a19e2..e0fecbca9056 100644 --- a/docs/content/ingestion/stream-pull.md +++ b/docs/content/ingestion/stream-pull.md @@ -142,7 +142,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |-----|----|-----------|--------| |type|String|This should always be 'realtime'.|no| |maxRowsInMemory|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 1000000)| -|maxBytesInMemory|Long|The maximum number of bytes to keep in memory to aggregate before persisting. This is used to manage the required JVM heap size.|no (default == One-third of max JVM memory)| +|maxBytesInMemory|Long|The maximum number of bytes to keep in memory to aggregate before persisting. This is used to manage the required JVM heap size.|no (default == One-sixth of max JVM memory)| |windowPeriod|ISO 8601 Period String|The amount of lag time to allow events. This is configured with a 10 minute window, meaning that any event more than 10 minutes ago will be thrown away and not included in the segment generated by the realtime server.|no (default == PT10m)| |intermediatePersistPeriod|ISO8601 Period String|The period that determines the rate at which intermediate persists occur. These persists determine how often commits happen against the incoming realtime stream. If the realtime data loading process is interrupted at time T, it should be restarted to re-read data that arrived at T minus this period.|no (default == PT10m)| |basePersistDirectory|String|The directory to put things that need persistence. The plumber is responsible for the actual intermediate persists and this tells it where to store those persists.|no (default == java tmp dir)| diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index 61503bf3d523..f3329f03c7ba 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -138,7 +138,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |type|The task type, this should always be "index".|none|yes| |targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no| |maxRowsInMemory|Used in determining when intermediate persists to disk should occur.|1000000|no| -|maxBytesInMemory|Used in determining when intermediate persists to disk should occur.|1/3 of max JVM memory|no| +|maxBytesInMemory|Used in determining when intermediate persists to disk should occur.|1/6 of max JVM memory|no| |maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|150000|no| |numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no| |indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| From 27f98b51bf38762c502c33a124fbd4459be6ef79 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Mon, 30 Apr 2018 10:53:02 -0700 Subject: [PATCH 27/31] Minor to rename and add comment --- ...zeTest.java => IncrementalIndexRowSizeTest.java} | 8 ++++---- .../io/druid/segment/indexing/TuningConfig.java | 3 +++ .../realtime/appenderator/AppenderatorImpl.java | 13 ++++++------- 3 files changed, 13 insertions(+), 11 deletions(-) rename processing/src/test/java/io/druid/segment/incremental/{TimeAndDimsKeySizeTest.java => IncrementalIndexRowSizeTest.java} (94%) diff --git a/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsKeySizeTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexRowSizeTest.java similarity index 94% rename from processing/src/test/java/io/druid/segment/incremental/TimeAndDimsKeySizeTest.java rename to processing/src/test/java/io/druid/segment/incremental/IncrementalIndexRowSizeTest.java index 582f0ef6def8..9d77bbb7693b 100644 --- a/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsKeySizeTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexRowSizeTest.java @@ -30,10 +30,10 @@ /** */ -public class TimeAndDimsKeySizeTest +public class IncrementalIndexRowSizeTest { @Test - public void testTimeAndDimsKeySizeBasic() + public void testIncrementalIndexRowSizeBasic() { IncrementalIndex index = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) @@ -47,7 +47,7 @@ public void testTimeAndDimsKeySizeBasic() } @Test - public void testTimeAndDimsKeySizeArr() + public void testIncrementalIndexRowSizeArr() { IncrementalIndex index = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) @@ -67,7 +67,7 @@ public void testTimeAndDimsKeySizeArr() } @Test - public void testTimeAndDimsKeySizeComplex() + public void testIncrementalIndexRowSizeComplex() { IncrementalIndex index = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) diff --git a/server/src/main/java/io/druid/segment/indexing/TuningConfig.java b/server/src/main/java/io/druid/segment/indexing/TuningConfig.java index e8f3d171d3c6..651fea80ad85 100644 --- a/server/src/main/java/io/druid/segment/indexing/TuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/TuningConfig.java @@ -34,5 +34,8 @@ public interface TuningConfig int DEFAULT_MAX_PARSE_EXCEPTIONS = Integer.MAX_VALUE; int DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS = 0; int DEFAULT_MAX_ROWS_IN_MEMORY = 1_000_000; + // We initially estimated this to be 1/3(max jvm memory), but bytesCurrentlyInMemory only + // tracks active index and not the index being flushed to disk, to account for that + // we doubled default to 1/6(max jvm memory) long DEFAULT_MAX_BYTES_IN_MEMORY = Runtime.getRuntime().maxMemory() / 6; } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 81786387e3a4..52b47b648af8 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -250,15 +250,15 @@ public AppenderatorAddResult add( boolean isPersistRequired = false; boolean persist = false; - StringBuilder persistReason = new StringBuilder(); + List persistReasons = Lists.newArrayList(); if (!sink.canAppendRow()) { persist = true; - persistReason.append(" No more rows can be appended to sink"); + persistReasons.add("No more rows can be appended to sink"); } if (System.currentTimeMillis() > nextFlush) { persist = true; - persistReason.append(StringUtils.format( + persistReasons.add(StringUtils.format( " current time[%d] is greater than nextFlush[%d],", System.currentTimeMillis(), nextFlush @@ -266,7 +266,7 @@ public AppenderatorAddResult add( } if (rowsCurrentlyInMemory.get() >= tuningConfig.getMaxRowsInMemory()) { persist = true; - persistReason.append(StringUtils.format( + persistReasons.add(StringUtils.format( " rowsCurrentlyInMemory[%d] is greater than maxRowsInMemory[%d],", rowsCurrentlyInMemory.get(), tuningConfig.getMaxRowsInMemory() @@ -276,7 +276,7 @@ public AppenderatorAddResult add( && bytesCurrentlyInMemory.get() >= TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory())) { persist = true; - persistReason.append(StringUtils.format( + persistReasons.add(StringUtils.format( " bytesCurrentlyInMemory[%d] is greater than maxBytesInMemory[%d]", bytesCurrentlyInMemory.get(), TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()) @@ -285,13 +285,12 @@ public AppenderatorAddResult add( if (persist) { if (allowIncrementalPersists) { // persistAll clears rowsCurrentlyInMemory, no need to update it. - log.info("Persisting rows in memory due to: [%s]", persistReason.toString()); + log.info("Persisting rows in memory due to: [%s]", String.join(",", persistReasons)); persistAll(committerSupplier == null ? null : committerSupplier.get()); } else { isPersistRequired = true; } } - return new AppenderatorAddResult(identifier, sink.getNumRows(), isPersistRequired, addResult.getParseException()); } From 9b8b39f2313ee6895b8dc7385c4821c5a57b1344 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Mon, 30 Apr 2018 11:40:50 -0700 Subject: [PATCH 28/31] Add more details in docs --- docs/content/development/extensions-core/kafka-ingestion.md | 2 +- docs/content/ingestion/batch-ingestion.md | 2 +- docs/content/ingestion/stream-pull.md | 2 +- docs/content/ingestion/tasks.md | 2 +- .../src/main/java/io/druid/segment/indexing/TuningConfig.java | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index d1ded0ae5d9b..9ad029c50f3d 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -116,7 +116,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |-----|----|-----------|--------| |`type`|String|The indexing task type, this should always be `kafka`.|yes| |`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 1000000)| -|`maxBytesInMemory`|Long|The maximum number of bytes to keep in memory to aggregate before persisting. This is used to manage the required JVM heap size. |no (default == One-sixth of max JVM memory)| +|`maxBytesInMemory`|Long|The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists). |no (default == One-sixth of max JVM memory)| |`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == 5000000)| |`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)| |`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)| diff --git a/docs/content/ingestion/batch-ingestion.md b/docs/content/ingestion/batch-ingestion.md index c4192b34a18e..2ea7e8014c8a 100644 --- a/docs/content/ingestion/batch-ingestion.md +++ b/docs/content/ingestion/batch-ingestion.md @@ -155,7 +155,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |version|String|The version of created segments. Ignored for HadoopIndexTask unless useExplicitVersion is set to true|no (default == datetime that indexing starts at)| |partitionsSpec|Object|A specification of how to partition each time bucket into segments. Absence of this property means no partitioning will occur. See 'Partitioning specification' below.|no (default == 'hashed')| |maxRowsInMemory|Integer|The number of rows to aggregate before persisting. Note that this is the number of post-aggregation rows which may not be equal to the number of input events due to roll-up. This is used to manage the required JVM heap size.|no (default == 1000000)| -|maxBytesInMemory|Long|The maximum number of bytes to keep in memory to aggregate before persisting. This is used to manage the required JVM heap size.|no (default == One-sixth of max JVM memory)| +|maxBytesInMemory|Long|The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists).|no (default == One-sixth of max JVM memory)| |leaveIntermediate|Boolean|Leave behind intermediate files (for debugging) in the workingPath when a job completes, whether it passes or fails.|no (default == false)| |cleanupOnFailure|Boolean|Clean up intermediate files when a job fails (unless leaveIntermediate is on).|no (default == true)| |overwriteFiles|Boolean|Override existing files found during indexing.|no (default == false)| diff --git a/docs/content/ingestion/stream-pull.md b/docs/content/ingestion/stream-pull.md index e0fecbca9056..6f899f01bfa8 100644 --- a/docs/content/ingestion/stream-pull.md +++ b/docs/content/ingestion/stream-pull.md @@ -289,7 +289,7 @@ The following table summarizes constraints between settings in the spec file for |queryGranularity| Time granularity (minute, hour, day, week, month) for rollup | less than segmentGranularity| minute, hour, day, week, month | |intermediatePersistPeriod| The max time (ISO8601 Period) between flushes of ingested rows from memory to disk | avoid excessive flushing | number of un-persisted rows in memory also constrained by maxRowsInMemory | |maxRowsInMemory| The max number of ingested rows to hold in memory before a flush to disk | number of un-persisted post-aggregation rows in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod | -|maxBytesInMemory| The maximum number of bytes to keep in memory before a flush to disk | number of un-persisted post-aggregation bytes in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod | +|maxBytesInMemory| The number of bytes to keep in memory before a flush to disk. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)| number of un-persisted post-aggregation bytes in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod | The normal, expected use cases have the following overall constraints: `intermediatePersistPeriod ≤ windowPeriod < segmentGranularity` and `queryGranularity ≤ segmentGranularity` diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index f3329f03c7ba..a734505da516 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -138,7 +138,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |type|The task type, this should always be "index".|none|yes| |targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no| |maxRowsInMemory|Used in determining when intermediate persists to disk should occur.|1000000|no| -|maxBytesInMemory|Used in determining when intermediate persists to disk should occur.|1/6 of max JVM memory|no| +|maxBytesInMemory|Used in determining when intermediate persists to disk should occur. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no| |maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|150000|no| |numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no| |indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| diff --git a/server/src/main/java/io/druid/segment/indexing/TuningConfig.java b/server/src/main/java/io/druid/segment/indexing/TuningConfig.java index 651fea80ad85..e8cc02fc4503 100644 --- a/server/src/main/java/io/druid/segment/indexing/TuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/TuningConfig.java @@ -36,6 +36,6 @@ public interface TuningConfig int DEFAULT_MAX_ROWS_IN_MEMORY = 1_000_000; // We initially estimated this to be 1/3(max jvm memory), but bytesCurrentlyInMemory only // tracks active index and not the index being flushed to disk, to account for that - // we doubled default to 1/6(max jvm memory) + // we halved default to 1/6(max jvm memory) long DEFAULT_MAX_BYTES_IN_MEMORY = Runtime.getRuntime().maxMemory() / 6; } From 28adb60ca0962c98ea0626281b844e89e7d075a9 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Wed, 2 May 2018 11:46:24 -0700 Subject: [PATCH 29/31] Address new PR comments --- docs/content/ingestion/tasks.md | 4 +-- .../indexing/kafka/KafkaTuningConfig.java | 2 ++ .../io/druid/indexer/HadoopTuningConfig.java | 4 ++- .../RealtimeAppenderatorTuningConfig.java | 3 ++- .../druid/indexing/common/task/IndexTask.java | 2 ++ .../druid/segment/StringDimensionIndexer.java | 4 ++- .../segment/incremental/IncrementalIndex.java | 1 + .../incremental/IncrementalIndexRow.java | 3 ++- .../incremental/OnheapIncrementalIndex.java | 25 ++++++++----------- .../OnheapIncrementalIndexBenchmark.java | 5 ++-- .../indexing/RealtimeTuningConfig.java | 6 +++-- .../appenderator/AppenderatorImpl.java | 12 +++------ .../druid/segment/realtime/plumber/Sink.java | 2 -- 13 files changed, 36 insertions(+), 37 deletions(-) diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index a734505da516..22e4cfddae22 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -137,8 +137,8 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |--------|-----------|-------|---------| |type|The task type, this should always be "index".|none|yes| |targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no| -|maxRowsInMemory|Used in determining when intermediate persists to disk should occur.|1000000|no| -|maxBytesInMemory|Used in determining when intermediate persists to disk should occur. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no| +|maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no| +|maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no| |maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|150000|no| |numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no| |indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index ead3d92247c6..91b0027b07ac 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -82,6 +82,8 @@ public KafkaTuningConfig( this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; this.maxRowsPerSegment = maxRowsPerSegment == null ? DEFAULT_MAX_ROWS_PER_SEGMENT : maxRowsPerSegment; + // initializing this to 0, it will be lazily intialized to a value + // @see server.src.main.java.io.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaults.getIntermediatePersistPeriod() diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java index 8386e4a837f4..3664d4edefcd 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java @@ -56,7 +56,7 @@ public static HadoopTuningConfig makeDefaultTuningConfig() DEFAULT_SHARD_SPECS, DEFAULT_INDEX_SPEC, DEFAULT_ROW_FLUSH_BOUNDARY, - DEFAULT_MAX_BYTES_IN_MEMORY, + 0L, false, true, false, @@ -132,6 +132,8 @@ public HadoopTuningConfig( this.rowFlushBoundary = maxRowsInMemory == null ? maxRowsInMemoryCOMPAT == null ? DEFAULT_ROW_FLUSH_BOUNDARY : maxRowsInMemoryCOMPAT : maxRowsInMemory; + // initializing this to 0, it will be lazily intialized to a value + // @see server.src.main.java.io.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.leaveIntermediate = leaveIntermediate; this.cleanupOnFailure = cleanupOnFailure == null ? true : cleanupOnFailure; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index 11372f2bded1..800c0300d43e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -39,7 +39,6 @@ public class RealtimeAppenderatorTuningConfig implements TuningConfig, Appendera { private static final int defaultMaxRowsInMemory = TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY; private static final int defaultMaxRowsPerSegment = 5_000_000; - private static final long defaultMaxBytesInMemory = TuningConfig.DEFAULT_MAX_BYTES_IN_MEMORY; private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); private static final int defaultMaxPendingPersists = 0; private static final ShardSpec defaultShardSpec = NoneShardSpec.instance(); @@ -94,6 +93,8 @@ public RealtimeAppenderatorTuningConfig( { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; this.maxRowsPerSegment = maxRowsPerSegment == null ? defaultMaxRowsPerSegment : maxRowsPerSegment; + // initializing this to 0, it will be lazily intialized to a value + // @see server.src.main.java.io.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaultIntermediatePersistPeriod diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 9c652e5c7a72..711f400136b7 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -1346,6 +1346,8 @@ private IndexTuningConfig( this.targetPartitionSize = initializeTargetPartitionSize(numShards, targetPartitionSize); this.maxRowsInMemory = maxRowsInMemory == null ? TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; + // initializing this to 0, it will be lazily intialized to a value + // @see server.src.main.java.io.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.maxTotalRows = initializeMaxTotalRows(numShards, maxTotalRows); this.numShards = numShards == null || numShards.equals(-1) ? null : numShards; diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java index 1b5d0ac82889..df28aa43f2e8 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -273,7 +273,9 @@ public int[] processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boole @Override public long estimateEncodedKeyComponentSize(int[] key) { - // string length is being accounted for each time they are referenced, even though they are stored just once + // string length is being accounted for each time they are referenced, based on dimension handler interface, + // even though they are stored just once. It may overestimate the size by a bit, but we wanted to leave + // more buffer to be safe long estimatedSize = key.length * Integer.BYTES; estimatedSize += Arrays.stream(key) .filter(element -> dimLookup.getValue(element) != null) diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 39556bbd31ba..f496d53a7dec 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -402,6 +402,7 @@ public Builder setMaxRowCount(final int maxRowCount) return this; } + //maxBytesInMemory only applies to OnHeapIncrementalIndex public Builder setMaxBytesInMemory(final long maxBytesInMemory) { this.maxBytesInMemory = maxBytesInMemory; diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRow.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRow.java index c8fcc1ecb68d..dd671267c4a6 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRow.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexRow.java @@ -113,12 +113,13 @@ void setRowIndex(int rowIndex) } /** - * bytesInMemory estimates the size of TimeAndDims key, it takes into account the timestamp(long), + * bytesInMemory estimates the size of IncrementalIndexRow key, it takes into account the timestamp(long), * dims(Object Array) and dimensionDescsList(List). Each of these are calculated as follows: *
        *
      • timestamp : Long.BYTES *
      • dims array : Integer.BYTES * array length + Long.BYTES (dims object) + dimsKeySize(passed via constructor) *
      • dimensionDescList : Long.BYTES (shared pointer) + *
      • dimsKeySize : this value is passed in based on the key type (int, long, double, String etc.) *
      * * @return long estimated bytesInMemory diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 9d6d297bdcd4..bc2f2ec37929 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -52,8 +52,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex { private static final Logger log = new Logger(OnheapIncrementalIndex.class); /** - * overhead per Map$Entry object - * KeyHash + next pointer + key pointer + value pointer + safe extra + * overhead per {@link ConcurrentHashMap.Node} object */ private static final int ROUGH_OVERHEAD_PER_MAP_ENTRY = Long.BYTES * 5 + Integer.BYTES; private final ConcurrentHashMap aggregators = new ConcurrentHashMap<>(); @@ -82,12 +81,12 @@ public class OnheapIncrementalIndex extends IncrementalIndex this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) : new PlainFactsHolder(sortFacts); maxBytesPerRowForAggregators = getMaxBytesPerRowForAggregators(incrementalIndexSchema); - } /** - * Gives estimated max size per aggregator. It is assumed that every aggregator will have enough overhead for its own object header - * and for a pointer to a selector. We are adding a overhead-factor for each object as additional 16 bytes. + * Gives estimated max size per aggregator. It is assumed that every aggregator will have enough overhead for its own + * object header and for a pointer to a selector. We are adding a overhead-factor for each object as additional 16 + * bytes. * These 16 bytes or 128 bits is the object metadata for 64-bit JVM process and consists of: *
        *
      • Class pointer which describes the object type: 64 bits @@ -107,8 +106,7 @@ private static long getMaxBytesPerRowForAggregators(IncrementalIndexSchema incre { long maxAggregatorIntermediateSize = Integer.BYTES * incrementalIndexSchema.getMetrics().length; maxAggregatorIntermediateSize += Arrays.stream(incrementalIndexSchema.getMetrics()) - .mapToLong(aggregator -> aggregator.getMaxIntermediateSize() - + Long.BYTES * 2) + .mapToLong(aggregator -> aggregator.getMaxIntermediateSize() + Long.BYTES * 2) .sum(); return maxAggregatorIntermediateSize; } @@ -172,10 +170,9 @@ protected AddToFactsResult addToFacts( concurrentSet(rowIndex, aggs); // Last ditch sanity checks - if (numEntries.get() >= maxRowCount - || (maxBytesInMemory != -1 && sizeInBytes.get() >= maxBytesInMemory) - && facts.getPriorIndex(key) == IncrementalIndexRow.EMPTY_ROW_INDEX - && !skipMaxRowsInMemoryCheck) { + if ((numEntries.get() >= maxRowCount || (maxBytesInMemory > 0 && sizeInBytes.get() >= maxBytesInMemory)) + && facts.getPriorIndex(key) == IncrementalIndexRow.EMPTY_ROW_INDEX + && !skipMaxRowsInMemoryCheck) { throw new IndexSizeExceededException( "Maximum number of rows [%d] or max size in bytes [%d] reached", maxRowCount, @@ -304,10 +301,8 @@ protected void concurrentRemove(int offset) public boolean canAppendRow() { final boolean countCheck = size() < maxRowCount; - boolean sizeCheck = true; - if (maxBytesInMemory > 0) { - sizeCheck = getBytesInMemory() < maxBytesInMemory; - } + // if maxBytesInMemory = -1, then ignore sizeCheck + final boolean sizeCheck = maxBytesInMemory <= 0 || getBytesInMemory() < maxBytesInMemory; final boolean canAdd = countCheck && sizeCheck; if (!countCheck && !sizeCheck) { outOfRowsReason = StringUtils.format( diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index fa89f4e06257..43264b959243 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -208,9 +208,8 @@ protected AddToFactsResult addToFacts( // Last ditch sanity checks - if (numEntries.get() >= maxRowCount - || sizeInBytes.get() >= maxBytesInMemory - && getFacts().getPriorIndex(key) == IncrementalIndexRow.EMPTY_ROW_INDEX) { + if ((numEntries.get() >= maxRowCount || sizeInBytes.get() >= maxBytesInMemory) + && getFacts().getPriorIndex(key) == IncrementalIndexRow.EMPTY_ROW_INDEX) { throw new IndexSizeExceededException("Maximum number of rows or max bytes reached"); } final int prev = getFacts().putIfAbsent(key, rowIndex); diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index 0edda59ee300..935e4edc1528 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -73,7 +73,7 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File { return new RealtimeTuningConfig( defaultMaxRowsInMemory, - defaultMaxBytesInMemory, + 0L, defaultIntermediatePersistPeriod, defaultWindowPeriod, basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory, @@ -133,7 +133,9 @@ public RealtimeTuningConfig( ) { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; - this.maxBytesInMemory = maxBytesInMemory == null ? defaultMaxBytesInMemory : maxBytesInMemory; + // initializing this to 0, it will be lazily intialized to a value + // @see server.src.main.java.io.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) + this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaultIntermediatePersistPeriod : intermediatePersistPeriod; diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 52b47b648af8..0248ade8bb68 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -85,6 +85,7 @@ import java.nio.channels.FileChannel; import java.nio.channels.FileLock; import java.nio.file.StandardOpenOption; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Comparator; @@ -250,7 +251,7 @@ public AppenderatorAddResult add( boolean isPersistRequired = false; boolean persist = false; - List persistReasons = Lists.newArrayList(); + List persistReasons = new ArrayList(); if (!sink.canAppendRow()) { persist = true; @@ -272,7 +273,7 @@ public AppenderatorAddResult add( tuningConfig.getMaxRowsInMemory() )); } - if (tuningConfig.getMaxBytesInMemory() != -1 + if (tuningConfig.getMaxBytesInMemory() > 0 && bytesCurrentlyInMemory.get() >= TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory())) { persist = true; @@ -622,13 +623,6 @@ private ListenableFuture pushBarrier() /** * Merge segment, push to deep storage. Should only be used on segments that have been fully persisted. Must only * be run in the single-threaded pushExecutor. -<<<<<<< HEAD - *

        - * Note that this calls DataSegmentPusher.push() with replaceExisting == true which is appropriate for the indexing - * tasks it is currently being used for (local indexing and Kafka indexing). If this is going to be used by an - * indexing task type that requires replaceExisting == false, this setting will need to be pushed to the caller. -======= ->>>>>>> 8ec2d2fe1855109014c2246a7d67b0e32c09d3f7 * * @param identifier sink identifier * @param sink sink to push diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index 39f03fcf629c..50d0cf2a1281 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -54,7 +54,6 @@ public class Sink implements Iterable { - private static final IncrementalIndexAddResult ADD_FAILED = new IncrementalIndexAddResult(-1, -1, null); private final Object hydrantLock = new Object(); @@ -256,7 +255,6 @@ public int getNumRowsInMemory() } } - public long getBytesInMemory() { synchronized (hydrantLock) { From 5288da3ab1a5b2f2bcc270b7bbfb1a8f8402d651 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Wed, 2 May 2018 13:51:22 -0700 Subject: [PATCH 30/31] Address PR comments --- docs/content/development/extensions-core/kafka-ingestion.md | 4 ++-- docs/content/ingestion/batch-ingestion.md | 4 ++-- docs/content/ingestion/stream-pull.md | 4 ++-- .../io/druid/segment/incremental/OnheapIncrementalIndex.java | 4 ++-- .../java/io/druid/segment/indexing/RealtimeTuningConfig.java | 1 - 5 files changed, 8 insertions(+), 9 deletions(-) diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index 9ad029c50f3d..ed2e594d425b 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -115,8 +115,8 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |Field|Type|Description|Required| |-----|----|-----------|--------| |`type`|String|The indexing task type, this should always be `kafka`.|yes| -|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 1000000)| -|`maxBytesInMemory`|Long|The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists). |no (default == One-sixth of max JVM memory)| +|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists). Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|no (default == 1000000)| +|`maxBytesInMemory`|Long|The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally this is computed internally and user does not need to set it. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists). |no (default == One-sixth of max JVM memory)| |`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == 5000000)| |`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)| |`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)| diff --git a/docs/content/ingestion/batch-ingestion.md b/docs/content/ingestion/batch-ingestion.md index 2ea7e8014c8a..f0f3040cb989 100644 --- a/docs/content/ingestion/batch-ingestion.md +++ b/docs/content/ingestion/batch-ingestion.md @@ -154,8 +154,8 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |workingPath|String|The working path to use for intermediate results (results between Hadoop jobs).|no (default == '/tmp/druid-indexing')| |version|String|The version of created segments. Ignored for HadoopIndexTask unless useExplicitVersion is set to true|no (default == datetime that indexing starts at)| |partitionsSpec|Object|A specification of how to partition each time bucket into segments. Absence of this property means no partitioning will occur. See 'Partitioning specification' below.|no (default == 'hashed')| -|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. Note that this is the number of post-aggregation rows which may not be equal to the number of input events due to roll-up. This is used to manage the required JVM heap size.|no (default == 1000000)| -|maxBytesInMemory|Long|The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists).|no (default == One-sixth of max JVM memory)| +|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. Note that this is the number of post-aggregation rows which may not be equal to the number of input events due to roll-up. This is used to manage the required JVM heap size. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|no (default == 1000000)| +|maxBytesInMemory|Long|The number of bytes to aggregate in heap memory before persisting. Normally this is computed internally and user does not need to set it. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists).|no (default == One-sixth of max JVM memory)| |leaveIntermediate|Boolean|Leave behind intermediate files (for debugging) in the workingPath when a job completes, whether it passes or fails.|no (default == false)| |cleanupOnFailure|Boolean|Clean up intermediate files when a job fails (unless leaveIntermediate is on).|no (default == true)| |overwriteFiles|Boolean|Override existing files found during indexing.|no (default == false)| diff --git a/docs/content/ingestion/stream-pull.md b/docs/content/ingestion/stream-pull.md index 6f899f01bfa8..1bd33caad821 100644 --- a/docs/content/ingestion/stream-pull.md +++ b/docs/content/ingestion/stream-pull.md @@ -288,8 +288,8 @@ The following table summarizes constraints between settings in the spec file for |segmentGranularity| Time granularity (minute, hour, day, week, month) for loading data at query time | equal to indexGranularity| more than queryGranularity| |queryGranularity| Time granularity (minute, hour, day, week, month) for rollup | less than segmentGranularity| minute, hour, day, week, month | |intermediatePersistPeriod| The max time (ISO8601 Period) between flushes of ingested rows from memory to disk | avoid excessive flushing | number of un-persisted rows in memory also constrained by maxRowsInMemory | -|maxRowsInMemory| The max number of ingested rows to hold in memory before a flush to disk | number of un-persisted post-aggregation rows in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod | -|maxBytesInMemory| The number of bytes to keep in memory before a flush to disk. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)| number of un-persisted post-aggregation bytes in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod | +|maxRowsInMemory| The max number of ingested rows to hold in memory before a flush to disk. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set| number of un-persisted post-aggregation rows in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod | +|maxBytesInMemory| The number of bytes to keep in memory before a flush to disk. Normally this is computed internally and user does not need to set it. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)| number of un-persisted post-aggregation bytes in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod | The normal, expected use cases have the following overall constraints: `intermediatePersistPeriod ≤ windowPeriod < segmentGranularity` and `queryGranularity ≤ segmentGranularity` diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index bc2f2ec37929..cbe9ba48bc9b 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -52,13 +52,13 @@ public class OnheapIncrementalIndex extends IncrementalIndex { private static final Logger log = new Logger(OnheapIncrementalIndex.class); /** - * overhead per {@link ConcurrentHashMap.Node} object + * overhead per {@link ConcurrentHashMap.Node} or {@link java.util.concurrent.ConcurrentSkipListMap.Node} object */ private static final int ROUGH_OVERHEAD_PER_MAP_ENTRY = Long.BYTES * 5 + Integer.BYTES; private final ConcurrentHashMap aggregators = new ConcurrentHashMap<>(); private final FactsHolder facts; private final AtomicInteger indexIncrement = new AtomicInteger(0); - private long maxBytesPerRowForAggregators = 0; + private final long maxBytesPerRowForAggregators; protected final int maxRowCount; protected final long maxBytesInMemory; private volatile Map selectors; diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index 935e4edc1528..5aaf13daef43 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -43,7 +43,6 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig { private static final int defaultMaxRowsInMemory = TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY; - private static final long defaultMaxBytesInMemory = TuningConfig.DEFAULT_MAX_BYTES_IN_MEMORY; private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); private static final Period defaultWindowPeriod = new Period("PT10M"); private static final VersioningPolicy defaultVersioningPolicy = new IntervalStartVersioningPolicy(); From c405b533788544dd62a1f022f5eb3ea926397c59 Mon Sep 17 00:00:00 2001 From: Surekha Saharan Date: Thu, 3 May 2018 13:57:19 -0700 Subject: [PATCH 31/31] Fix spelling typo --- .../main/java/io/druid/indexing/kafka/KafkaTuningConfig.java | 2 +- .../src/main/java/io/druid/indexer/HadoopTuningConfig.java | 2 +- .../src/main/java/io/druid/indexing/common/task/IndexTask.java | 2 +- .../java/io/druid/segment/indexing/RealtimeTuningConfig.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index 91b0027b07ac..a31fb1a92b31 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -82,7 +82,7 @@ public KafkaTuningConfig( this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; this.maxRowsPerSegment = maxRowsPerSegment == null ? DEFAULT_MAX_ROWS_PER_SEGMENT : maxRowsPerSegment; - // initializing this to 0, it will be lazily intialized to a value + // initializing this to 0, it will be lazily initialized to a value // @see server.src.main.java.io.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java index 3664d4edefcd..babc1bc515ce 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java @@ -132,7 +132,7 @@ public HadoopTuningConfig( this.rowFlushBoundary = maxRowsInMemory == null ? maxRowsInMemoryCOMPAT == null ? DEFAULT_ROW_FLUSH_BOUNDARY : maxRowsInMemoryCOMPAT : maxRowsInMemory; - // initializing this to 0, it will be lazily intialized to a value + // initializing this to 0, it will be lazily initialized to a value // @see server.src.main.java.io.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.leaveIntermediate = leaveIntermediate; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 711f400136b7..554969ca993c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -1346,7 +1346,7 @@ private IndexTuningConfig( this.targetPartitionSize = initializeTargetPartitionSize(numShards, targetPartitionSize); this.maxRowsInMemory = maxRowsInMemory == null ? TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - // initializing this to 0, it will be lazily intialized to a value + // initializing this to 0, it will be lazily initialized to a value // @see server.src.main.java.io.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.maxTotalRows = initializeMaxTotalRows(numShards, maxTotalRows); diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index 5aaf13daef43..f3f3ae6a1d97 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -132,7 +132,7 @@ public RealtimeTuningConfig( ) { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; - // initializing this to 0, it will be lazily intialized to a value + // initializing this to 0, it will be lazily initialized to a value // @see server.src.main.java.io.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null