From 9e786e89b5fb58758247f486f9d5b5170a4a9cca Mon Sep 17 00:00:00 2001 From: Jonathan Wei Date: Thu, 5 Apr 2018 21:38:57 -0700 Subject: [PATCH] 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);