diff --git a/.travis.yml b/.travis.yml index 4fb64a7bf9af..4ec30507b4f4 100644 --- a/.travis.yml +++ b/.travis.yml @@ -47,6 +47,7 @@ matrix: - sudo: false env: - NAME="other modules test" + - AWS_REGION=us-east-1 # set a aws region for unit tests install: echo "MAVEN_OPTS='-Xmx3000m'" > ~/.mavenrc && mvn install -q -ff -DskipTests -B before_script: - unset _JAVA_OPTIONS diff --git a/api/src/main/java/io/druid/data/input/Rows.java b/api/src/main/java/io/druid/data/input/Rows.java index 0ef09e9fa246..a963c605944e 100644 --- a/api/src/main/java/io/druid/data/input/Rows.java +++ b/api/src/main/java/io/druid/data/input/Rows.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableSortedSet; import com.google.common.collect.Maps; import com.google.common.primitives.Longs; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.parsers.ParseException; @@ -116,6 +117,8 @@ public static Number objectToNumber(final String name, final Object inputValue) catch (Exception e) { throw new ParseException(e, "Unable to parse value[%s] for field[%s]", inputValue, name); } + } else if (inputValue instanceof Pair) { + return objectToNumber(name, ((Pair) inputValue).rhs); } else { throw new ParseException("Unknown type[%s] for field", inputValue.getClass(), inputValue); } 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/data/input/impl/StringInputRowParser.java b/api/src/main/java/io/druid/data/input/impl/StringInputRowParser.java index cefe4706e2f4..2a7fcdca80af 100644 --- a/api/src/main/java/io/druid/data/input/impl/StringInputRowParser.java +++ b/api/src/main/java/io/druid/data/input/impl/StringInputRowParser.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; import io.druid.data.input.ByteBufferInputRowParser; @@ -36,6 +35,7 @@ import java.nio.charset.Charset; import java.nio.charset.CoderResult; import java.nio.charset.CodingErrorAction; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -43,7 +43,7 @@ */ public class StringInputRowParser implements ByteBufferInputRowParser { - private static final Charset DEFAULT_CHARSET = Charsets.UTF_8; + private static final Charset DEFAULT_CHARSET = StandardCharsets.UTF_8; private final ParseSpec parseSpec; private final MapInputRowParser mapParser; diff --git a/api/src/main/java/io/druid/guice/JsonConfigProvider.java b/api/src/main/java/io/druid/guice/JsonConfigProvider.java index c3a9cfd64d80..609567b2dd7d 100644 --- a/api/src/main/java/io/druid/guice/JsonConfigProvider.java +++ b/api/src/main/java/io/druid/guice/JsonConfigProvider.java @@ -130,8 +130,8 @@ public static void bind( Key> supplierKey ) { - binder.bind(supplierKey).toProvider((Provider) of(propertyBase, clazz)).in(LazySingleton.class); - binder.bind(instanceKey).toProvider(new SupplierProvider(supplierKey)); + binder.bind(supplierKey).toProvider(of(propertyBase, clazz)).in(LazySingleton.class); + binder.bind(instanceKey).toProvider(new SupplierProvider<>(supplierKey)); } @SuppressWarnings("unchecked") 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..af60cd95e2ad 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,20 +50,22 @@ public TaskStatusPlus( @JsonProperty("statusCode") @Nullable TaskState state, @JsonProperty("duration") @Nullable Long duration, @JsonProperty("location") TaskLocation location, - @JsonProperty("dataSource") String dataSource + @JsonProperty("dataSource") @Nullable String dataSource, // nullable for backward compatibility + @JsonProperty("errorMsg") @Nullable String errorMsg ) { if (state != null && state.isComplete()) { Preconditions.checkNotNull(duration, "duration"); } this.id = Preconditions.checkNotNull(id, "id"); - this.type = Preconditions.checkNotNull(type, "type"); + this.type = type; this.createdTime = Preconditions.checkNotNull(createdTime, "createdTime"); this.queueInsertionTime = Preconditions.checkNotNull(queueInsertionTime, "queueInsertionTime"); this.state = state; 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/tasklogs/NoopTaskLogs.java b/api/src/main/java/io/druid/tasklogs/NoopTaskLogs.java index 6fb0f309cfe2..d54c63cce182 100644 --- a/api/src/main/java/io/druid/tasklogs/NoopTaskLogs.java +++ b/api/src/main/java/io/druid/tasklogs/NoopTaskLogs.java @@ -24,6 +24,7 @@ import io.druid.java.util.common.logger.Logger; import java.io.File; +import java.io.IOException; public class NoopTaskLogs implements TaskLogs { @@ -41,6 +42,12 @@ public void pushTaskLog(String taskid, File logFile) log.info("Not pushing logs for task: %s", taskid); } + @Override + public void pushTaskReports(String taskid, File reportFile) throws IOException + { + log.info("Not pushing reports for task: %s", taskid); + } + @Override public void killAll() { diff --git a/api/src/main/java/io/druid/tasklogs/TaskLogPusher.java b/api/src/main/java/io/druid/tasklogs/TaskLogPusher.java index a904a16f5d1e..6329aac866d8 100644 --- a/api/src/main/java/io/druid/tasklogs/TaskLogPusher.java +++ b/api/src/main/java/io/druid/tasklogs/TaskLogPusher.java @@ -31,4 +31,8 @@ public interface TaskLogPusher { void pushTaskLog(String taskid, File logFile) throws IOException; + + default void pushTaskReports(String taskid, File reportFile) throws IOException + { + } } diff --git a/api/src/main/java/io/druid/tasklogs/TaskLogStreamer.java b/api/src/main/java/io/druid/tasklogs/TaskLogStreamer.java index b685c7b7659b..7569cdd145ba 100644 --- a/api/src/main/java/io/druid/tasklogs/TaskLogStreamer.java +++ b/api/src/main/java/io/druid/tasklogs/TaskLogStreamer.java @@ -40,4 +40,9 @@ public interface TaskLogStreamer * @return input supplier for this log, if available from this provider */ Optional streamTaskLog(String taskid, long offset) throws IOException; + + default Optional streamTaskReports(final String taskid) throws IOException + { + return Optional.absent(); + } } diff --git a/api/src/main/java/io/druid/timeline/partition/NoneShardSpec.java b/api/src/main/java/io/druid/timeline/partition/NoneShardSpec.java index 1fcadfb42d23..d20c3b5458c2 100644 --- a/api/src/main/java/io/druid/timeline/partition/NoneShardSpec.java +++ b/api/src/main/java/io/druid/timeline/partition/NoneShardSpec.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; import io.druid.data.input.InputRow; import java.util.List; @@ -71,19 +71,11 @@ public int getPartitionNum() @Override public ShardSpecLookup getLookup(final List shardSpecs) { - - return new ShardSpecLookup() - { - @Override - public ShardSpec getShardSpec(long timestamp, InputRow row) - { - return shardSpecs.get(0); - } - }; + return (long timestamp, InputRow row) -> shardSpecs.get(0); } @Override - public Map> getDomain() + public Map> getDomain() { return ImmutableMap.of(); } diff --git a/api/src/main/java/io/druid/timeline/partition/ShardSpec.java b/api/src/main/java/io/druid/timeline/partition/ShardSpec.java index 5461544c609c..c691c3959dbb 100644 --- a/api/src/main/java/io/druid/timeline/partition/ShardSpec.java +++ b/api/src/main/java/io/druid/timeline/partition/ShardSpec.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; import io.druid.data.input.InputRow; import java.util.List; @@ -50,5 +50,5 @@ public interface ShardSpec * * @return map of dimensions to its possible range. Dimensions with unknown possible range are not mapped */ - Map> getDomain(); + Map> getDomain(); } diff --git a/api/src/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/data/input/MapBasedRowTest.java b/api/src/test/java/io/druid/data/input/MapBasedRowTest.java index b6b4d5a9840f..aacf66c1a836 100644 --- a/api/src/test/java/io/druid/data/input/MapBasedRowTest.java +++ b/api/src/test/java/io/druid/data/input/MapBasedRowTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableMap; import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.Pair; import org.junit.Assert; import org.junit.Test; @@ -39,6 +40,7 @@ public void testGetLongMetricFromString() .put("k4", "9223372036854775806") .put("k5", "-9223372036854775807") .put("k6", "+9223372036854775802") + .put("k7", new Pair<>(1234L, "1.24")) .build() ); @@ -49,5 +51,6 @@ public void testGetLongMetricFromString() Assert.assertEquals(9223372036854775806L, row.getMetric("k4")); Assert.assertEquals(-9223372036854775807L, row.getMetric("k5")); Assert.assertEquals(9223372036854775802L, row.getMetric("k6")); + Assert.assertEquals(1.24, row.getMetric("k7")); } } diff --git a/api/src/test/java/io/druid/data/input/impl/InputRowParserSerdeTest.java b/api/src/test/java/io/druid/data/input/impl/InputRowParserSerdeTest.java index e81696283447..d9da1907a0ec 100644 --- a/api/src/test/java/io/druid/data/input/impl/InputRowParserSerdeTest.java +++ b/api/src/test/java/io/druid/data/input/impl/InputRowParserSerdeTest.java @@ -20,7 +20,6 @@ package io.druid.data.input.impl; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Charsets; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -37,6 +36,7 @@ import java.nio.ByteBuffer; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -73,8 +73,8 @@ public void testStringInputRowParserSerde() throws Exception public void testStringInputRowParserSerdeMultiCharset() throws Exception { Charset[] testCharsets = { - Charsets.US_ASCII, Charsets.ISO_8859_1, Charsets.UTF_8, - Charsets.UTF_16BE, Charsets.UTF_16LE, Charsets.UTF_16 + StandardCharsets.US_ASCII, StandardCharsets.ISO_8859_1, StandardCharsets.UTF_8, + StandardCharsets.UTF_16BE, StandardCharsets.UTF_16LE, StandardCharsets.UTF_16 }; for (Charset testCharset : testCharsets) { @@ -101,9 +101,9 @@ public void testMapInputRowParserSerde() throws Exception null ) ); - final MapInputRowParser parser2 = jsonMapper.readValue( + final MapInputRowParser parser2 = (MapInputRowParser) jsonMapper.readValue( jsonMapper.writeValueAsBytes(parser), - MapInputRowParser.class + InputRowParser.class ); final InputRow parsed = parser2.parseBatch( ImmutableMap.of( @@ -134,9 +134,9 @@ public void testMapInputRowParserNumbersSerde() throws Exception null ) ); - final MapInputRowParser parser2 = jsonMapper.readValue( + final MapInputRowParser parser2 = (MapInputRowParser) jsonMapper.readValue( jsonMapper.writeValueAsBytes(parser), - MapInputRowParser.class + InputRowParser.class ); final InputRow parsed = parser2.parseBatch( ImmutableMap.of( diff --git a/api/src/test/java/io/druid/data/input/impl/JSONParseSpecTest.java b/api/src/test/java/io/druid/data/input/impl/JSONParseSpecTest.java index de2814eda9a7..c7c73802f73d 100644 --- a/api/src/test/java/io/druid/data/input/impl/JSONParseSpecTest.java +++ b/api/src/test/java/io/druid/data/input/impl/JSONParseSpecTest.java @@ -91,9 +91,9 @@ public void testSerde() throws IOException feature ); - final JSONParseSpec serde = jsonMapper.readValue( + final JSONParseSpec serde = (JSONParseSpec) jsonMapper.readValue( jsonMapper.writeValueAsString(spec), - JSONParseSpec.class + ParseSpec.class ); Assert.assertEquals("timestamp", serde.getTimestampSpec().getTimestampColumn()); Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat()); diff --git a/api/src/test/java/io/druid/data/input/impl/JavaScriptParseSpecTest.java b/api/src/test/java/io/druid/data/input/impl/JavaScriptParseSpecTest.java index 805e019b1b08..b63caf43ff33 100644 --- a/api/src/test/java/io/druid/data/input/impl/JavaScriptParseSpecTest.java +++ b/api/src/test/java/io/druid/data/input/impl/JavaScriptParseSpecTest.java @@ -58,9 +58,9 @@ public void testSerde() throws IOException "abc", JavaScriptConfig.getEnabledInstance() ); - final JavaScriptParseSpec serde = jsonMapper.readValue( + final JavaScriptParseSpec serde = (JavaScriptParseSpec) jsonMapper.readValue( jsonMapper.writeValueAsString(spec), - JavaScriptParseSpec.class + ParseSpec.class ); Assert.assertEquals("abc", serde.getTimestampSpec().getTimestampColumn()); Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat()); diff --git a/api/src/test/java/io/druid/data/input/impl/RegexParseSpecTest.java b/api/src/test/java/io/druid/data/input/impl/RegexParseSpecTest.java index 68930ea6269d..5468ae0302f9 100644 --- a/api/src/test/java/io/druid/data/input/impl/RegexParseSpecTest.java +++ b/api/src/test/java/io/druid/data/input/impl/RegexParseSpecTest.java @@ -43,9 +43,9 @@ public void testSerde() throws IOException Collections.singletonList("abc"), "abc" ); - final RegexParseSpec serde = jsonMapper.readValue( + final RegexParseSpec serde = (RegexParseSpec) jsonMapper.readValue( jsonMapper.writeValueAsString(spec), - RegexParseSpec.class + ParseSpec.class ); Assert.assertEquals("abc", serde.getTimestampSpec().getTimestampColumn()); Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat()); diff --git a/api/src/test/java/io/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java index 340da4d49349..2d43fc3179a5 100644 --- a/api/src/test/java/io/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/api/src/test/java/io/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java @@ -19,7 +19,6 @@ package io.druid.data.input.impl.prefetch; -import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.Lists; @@ -82,7 +81,7 @@ public class PrefetchableTextFilesFirehoseFactoryTest false, 0 ), - Charsets.UTF_8.name() + StandardCharsets.UTF_8.name() ); @Rule 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/api/src/test/java/io/druid/timeline/DataSegmentTest.java b/api/src/test/java/io/druid/timeline/DataSegmentTest.java index bda7947dd2b2..f2743d24b0a2 100644 --- a/api/src/test/java/io/druid/timeline/DataSegmentTest.java +++ b/api/src/test/java/io/druid/timeline/DataSegmentTest.java @@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; import com.google.common.collect.Sets; import io.druid.TestObjectMapper; import io.druid.data.input.InputRow; @@ -82,7 +82,7 @@ public ShardSpecLookup getLookup(List shardSpecs) } @Override - public Map> getDomain() + public Map> getDomain() { return ImmutableMap.of(); } diff --git a/aws-common/pom.xml b/aws-common/pom.xml index 9ce40e3951ea..c6e69b099f66 100644 --- a/aws-common/pom.xml +++ b/aws-common/pom.xml @@ -37,7 +37,7 @@ com.amazonaws - aws-java-sdk-ec2 + aws-java-sdk-bundle diff --git a/aws-common/src/main/java/io/druid/common/aws/AWSEndpointConfig.java b/aws-common/src/main/java/io/druid/common/aws/AWSEndpointConfig.java new file mode 100644 index 000000000000..773a2ab15013 --- /dev/null +++ b/aws-common/src/main/java/io/druid/common/aws/AWSEndpointConfig.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.common.aws; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class AWSEndpointConfig +{ + @JsonProperty + private String url; + + @JsonProperty + private String serviceName; + + @JsonProperty + private String signingRegion; + + @JsonProperty + public String getUrl() + { + return url; + } + + @JsonProperty + public String getServiceName() + { + return serviceName; + } + + @JsonProperty + public String getSigningRegion() + { + return signingRegion; + } +} diff --git a/aws-common/src/main/java/io/druid/common/aws/AWSProxyConfig.java b/aws-common/src/main/java/io/druid/common/aws/AWSProxyConfig.java new file mode 100644 index 000000000000..eda04bb37152 --- /dev/null +++ b/aws-common/src/main/java/io/druid/common/aws/AWSProxyConfig.java @@ -0,0 +1,61 @@ +/* + * 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.common.aws; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class AWSProxyConfig +{ + @JsonProperty + private String host; + + @JsonProperty + private int port = -1; // AWS's default proxy port is -1 + + @JsonProperty + private String username; + + @JsonProperty + private String password; + + @JsonProperty + public String getHost() + { + return host; + } + + @JsonProperty + public int getPort() + { + return port; + } + + @JsonProperty + public String getUsername() + { + return username; + } + + @JsonProperty + public String getPassword() + { + return password; + } +} 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/LoadStatusBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/LoadStatusBenchmark.java new file mode 100644 index 000000000000..e1b877d481d8 --- /dev/null +++ b/benchmarks/src/main/java/io/druid/benchmark/LoadStatusBenchmark.java @@ -0,0 +1,121 @@ +/* + * 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.benchmark; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.StringUtils; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.NoneShardSpec; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 15) +@Measurement(iterations = 30) +public class LoadStatusBenchmark +{ + // Number of total data segments + @Param({"10000"}) + int totalSegmentsCount; + + @Param({"true", "false"}) + private boolean serverHasAllSegments; + + private Set datasourceSegments; + private Collection serverSegments; + + @Setup(Level.Invocation) + public void setup() + { + Map immutableDatasourceSegmentsMap; + ConcurrentHashMap serverSegmentsMap; + + HashMap datasourceSegmentsMap = Maps.newHashMap(); + serverSegmentsMap = new ConcurrentHashMap<>(); + + for (int i = 0; i < totalSegmentsCount; i++) { + DataSegment segment = new DataSegment( + "benchmarkDatasource", + Intervals.of(StringUtils.format("%s-01-01/%s-12-31", i + 1970, i + 1970)), + "1", + null, + null, + null, + NoneShardSpec.instance(), + 1, + 1 + ); + + datasourceSegmentsMap.put(segment.getIdentifier(), segment); + + if (serverHasAllSegments || i % 2 == 0) { + serverSegmentsMap.put(segment.getIdentifier(), segment); + } + } + + immutableDatasourceSegmentsMap = ImmutableMap.copyOf(datasourceSegmentsMap); + + datasourceSegments = Sets.newHashSet(immutableDatasourceSegmentsMap.values()); + serverSegments = Collections.unmodifiableCollection(serverSegmentsMap.values()); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void oldVersion(Blackhole blackhole) + { + datasourceSegments.removeAll(serverSegments); + blackhole.consume(datasourceSegments); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void newVersion(Blackhole blackhole) + { + for (DataSegment segment : serverSegments) { + datasourceSegments.remove(segment); + } + blackhole.consume(datasourceSegments); + } +} 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/codestyle/checkstyle.xml b/codestyle/checkstyle.xml index 9b8a9ed3eab7..32711f957595 100644 --- a/codestyle/checkstyle.xml +++ b/codestyle/checkstyle.xml @@ -94,6 +94,7 @@ + diff --git a/codestyle/druid-forbidden-apis.txt b/codestyle/druid-forbidden-apis.txt index 96db08826006..c50619d0cf4f 100644 --- a/codestyle/druid-forbidden-apis.txt +++ b/codestyle/druid-forbidden-apis.txt @@ -1,4 +1,5 @@ com.google.common.collect.MapMaker @ Create java.util.concurrent.ConcurrentHashMap directly com.google.common.collect.Maps#newConcurrentMap() @ Create java.util.concurrent.ConcurrentHashMap directly com.google.common.util.concurrent.Futures#transform(com.google.common.util.concurrent.ListenableFuture, com.google.common.util.concurrent.AsyncFunction) @ Use io.druid.java.util.common.concurrent.ListenableFutures#transformAsync -com.google.common.collect.Iterators#emptyIterator() @ Use java.util.Collections#emptyIterator() \ No newline at end of file +com.google.common.collect.Iterators#emptyIterator() @ Use java.util.Collections#emptyIterator() +com.google.common.base.Charsets @ Use java.nio.charset.StandardCharsets instead \ No newline at end of file diff --git a/common/pom.xml b/common/pom.xml index 234f21af5bb2..0f07a4105ab8 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -157,21 +157,7 @@ com.lmax disruptor - - - net.java.dev.jets3t - jets3t - 0.9.4 - + org.antlr antlr4-runtime diff --git a/common/src/main/java/io/druid/concurrent/LifecycleLock.java b/common/src/main/java/io/druid/concurrent/LifecycleLock.java index a94e3cd31a3e..7cc8e64a113f 100644 --- a/common/src/main/java/io/druid/concurrent/LifecycleLock.java +++ b/common/src/main/java/io/druid/concurrent/LifecycleLock.java @@ -175,9 +175,9 @@ void exitStop() } } - void reset() + void exitStopAndReset() { - if (!compareAndSetState(STOPPED, NOT_STARTED)) { + if (!compareAndSetState(STOPPING, NOT_STARTED)) { throw new IllegalMonitorStateException("Not called exitStop() before reset()"); } } @@ -187,7 +187,7 @@ void reset() /** * Start latch, only one canStart() call in any thread on this LifecycleLock object could return true, if {@link - * #reset()} is not called in between. + * #exitStopAndReset()} is not called in between. */ public boolean canStart() { @@ -257,8 +257,8 @@ public boolean canStop() } /** - * If this LifecycleLock is used in a restartable object, which uses {@link #reset()}, exitStop() must be called - * before exit from stop() on this object, usually in a finally block. + * Finalizes stopping the the LifecycleLock. This method must be called before exit from stop() on this object, + * usually in a finally block. If you're using a restartable object, use {@link #exitStopAndReset()} instead. * * @throws IllegalMonitorStateException if {@link #canStop()} is not yet called on this LifecycleLock */ @@ -268,12 +268,14 @@ public void exitStop() } /** - * Resets the LifecycleLock after {@link #exitStop()}, so that {@link #canStart()} could be called again. + * Finalizes stopping the LifecycleLock and resets it, so that {@link #canStart()} could be called again. If this + * LifecycleLock is used in a restartable object, this method must be called before exit from stop() on this object, + * usually in a finally block. * - * @throws IllegalMonitorStateException if {@link #exitStop()} is not yet called on this LifecycleLock + * @throws IllegalMonitorStateException if {@link #canStop()} is not yet called on this LifecycleLock */ - public void reset() + public void exitStopAndReset() { - sync.reset(); + sync.exitStopAndReset(); } } 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/common/src/test/java/io/druid/concurrent/LifecycleLockTest.java b/common/src/test/java/io/druid/concurrent/LifecycleLockTest.java index e1d9f8adbbf7..afb55bf2c2c2 100644 --- a/common/src/test/java/io/druid/concurrent/LifecycleLockTest.java +++ b/common/src/test/java/io/druid/concurrent/LifecycleLockTest.java @@ -138,8 +138,7 @@ public void testRestart() lifecycleLock.started(); lifecycleLock.exitStart(); Assert.assertTrue(lifecycleLock.canStop()); - lifecycleLock.exitStop(); - lifecycleLock.reset(); + lifecycleLock.exitStopAndReset(); Assert.assertTrue(lifecycleLock.canStart()); } diff --git a/docs/_redirects.json b/docs/_redirects.json index 0e6ef6ff9868..26921990df95 100644 --- a/docs/_redirects.json +++ b/docs/_redirects.json @@ -57,7 +57,6 @@ {"source": "Performance-FAQ.html", "target": "operations/performance-faq.html"}, {"source": "Plumber.html", "target": "design/plumber.html"}, {"source": "Post-aggregations.html", "target": "querying/post-aggregations.html"}, - {"source": "Production-Cluster-Configuration.html", "target": "configuration/production-cluster.html"}, {"source": "Query-Context.html", "target": "querying/query-context.html"}, {"source": "Querying.html", "target": "querying/querying.html"}, {"source": "Realtime-Config.html", "target": "configuration/realtime.html"}, @@ -94,7 +93,8 @@ {"source": "configuration/simple-cluster.html", "target": "../tutorials/cluster.html"}, {"source": "design/concepts-and-terminology.html", "target": "index.html"}, {"source": "development/approximate-histograms.html", "target": "extensions-core/approximate-histograms.html"}, - {"source": "development/datasketches-aggregators.html", "target": "extensions-core/datasketches-aggregators.html"}, + {"source": "development/datasketches-aggregators.html", "target": "extensions-core/datasketches-extension.html"}, + {"source": "development/extensions-core/datasketches-aggregators.html", "target": "datasketches-extension.html"}, {"source": "development/libraries.html", "target": "/libraries.html"}, {"source": "development/kafka-simple-consumer-firehose.html", "target": "extensions-contrib/kafka-simple.html"}, {"source": "development/select-query.html", "target": "../querying/select-query.html"}, diff --git a/docs/content/configuration/auth.md b/docs/content/configuration/auth.md index ae17ebf14d0d..4044fadd6dd5 100644 --- a/docs/content/configuration/auth.md +++ b/docs/content/configuration/auth.md @@ -9,6 +9,8 @@ layout: doc_page |`druid.auth.authenticationChain`|JSON List of Strings|List of Authenticator type names|["allowAll"]|no| |`druid.escalator.type`|String|Type of the Escalator that should be used for internal Druid communications. This Escalator must use an authentication scheme that is supported by an Authenticator in `druid.auth.authenticationChain`.|"noop"|no| |`druid.auth.authorizers`|JSON List of Strings|List of Authorizer type names |["allowAll"]|no| +|`druid.auth.unsecuredPaths`| List of Strings|List of paths for which security checks will not be performed. All requests to these paths will be allowed.|[]|no| +|`druid.auth.allowUnauthenticatedHttpOptions`|Boolean|If true, skip authentication checks for HTTP OPTIONS requests. This is needed for certain use cases, such as supporting CORS pre-flight requests. Note that disabling authentication checks for OPTIONS requests will allow unauthenticated users to determine what Druid endpoints are valid (by checking if the OPTIONS request returns a 200 instead of 404), so enabling this option may reveal information about server configuration, including information about what extensions are loaded (if those extensions add endpoints).|false|no| ## Enabling Authentication/Authorization diff --git a/docs/content/configuration/broker.md b/docs/content/configuration/broker.md index edddca824db3..cd074353201f 100644 --- a/docs/content/configuration/broker.md +++ b/docs/content/configuration/broker.md @@ -41,11 +41,13 @@ Druid uses Jetty to serve HTTP requests. |`druid.server.http.enableRequestLimit`|If enabled, no requests would be queued in jetty queue and "HTTP 429 Too Many Requests" error response would be sent. |false| |`druid.server.http.defaultQueryTimeout`|Query timeout in millis, beyond which unfinished queries will be cancelled|300000| |`druid.server.http.maxScatterGatherBytes`|Maximum number of bytes gathered from data nodes such as historicals and realtime processes to execute a query. This is an advance configuration that allows to protect in case broker is under heavy load and not utilizing the data gathered in memory fast enough and leading to OOMs. This limit can be further reduced at query time using `maxScatterGatherBytes` in the context. Note that having large limit is not necessarily bad if broker is never under heavy concurrent load in which case data gathered is processed quickly and freeing up the memory used.|Long.MAX_VALUE| +|`druid.server.http.gracefulShutdownTimeout`|The maximum amount of time Jetty waits after receiving shutdown signal. After this timeout the threads will be forcefully shutdown. This allows any queries that are executing to complete.|`PT0s` (do not wait)| +|`druid.server.http.unannouncePropogationDelay`|How long to wait for zookeeper unannouncements to propgate before shutting down Jetty. This is a minimum and `druid.server.http.gracefulShutdownTimeout` does not start counting down until after this period elapses.|`PT0s` (do not wait)| |`druid.broker.http.numConnections`|Size of connection pool for the Broker to connect to historical and real-time processes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|20| |`druid.broker.http.compressionCodec`|Compression codec the Broker uses to communicate with historical and real-time processes. May be "gzip" or "identity".|gzip| |`druid.broker.http.readTimeout`|The timeout for data reads from historical and real-time processes.|PT15M| |`druid.broker.http.unusedConnectionTimeout`|The timeout for idle connections in connection pool. This timeout should be less than `druid.broker.http.readTimeout`. Set this timeout = ~90% of `druid.broker.http.readTimeout`|`PT4M`| -|`druid.server.http.maxQueryTimeout`|Maximum allowed value (in milliseconds) for `timeout` parameter. See [query-context](query-context.html) to know more about `timeout`. Query is rejected if the query context `timeout` is greater than this value. |Long.MAX_VALUE| +|`druid.server.http.maxQueryTimeout`|Maximum allowed value (in milliseconds) for `timeout` parameter. See [query-context](../querying/query-context.html) to know more about `timeout`. Query is rejected if the query context `timeout` is greater than this value. |Long.MAX_VALUE| |`druid.server.http.maxRequestHeaderSize`|Maximum size of a request header in bytes. Larger headers consume more memory and can make a server more vulnerable to denial of service attacks. |8 * 1024| @@ -108,6 +110,9 @@ You can optionally only configure caching to be enabled on the broker by setting |--------|---------------|-----------|-------| |`druid.broker.cache.useCache`|true, false|Enable the cache on the broker.|false| |`druid.broker.cache.populateCache`|true, false|Populate the cache on the broker.|false| +|`druid.broker.cache.useResultLevelCache`|true, false|Enable result level caching on the broker.|false| +|`druid.broker.cache.populateResultLevelCache`|true, false|Populate the result level cache on the broker.|false| +|`druid.broker.cache.resultLevelCacheLimit`|positive integer|Maximum size of query response that can be cached.|`Integer.MAX_VALUE`| |`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`| |`druid.broker.cache.cacheBulkMergeLimit`|positive integer or 0|Queries with more segments than this number will not attempt to fetch from cache at the broker level, leaving potential caching fetches (and cache result merging) to the historicals|`Integer.MAX_VALUE`| diff --git a/docs/content/configuration/coordinator.md b/docs/content/configuration/coordinator.md index d571dadcd3b0..9e4f38cf44e9 100644 --- a/docs/content/configuration/coordinator.md +++ b/docs/content/configuration/coordinator.md @@ -29,6 +29,7 @@ The coordinator node uses several of the global configs in [Configuration](../co |`druid.coordinator.merge.on`|Boolean flag for whether or not the coordinator should try and merge small segments into a more optimal segment size.|false| |`druid.coordinator.conversion.on`|Boolean flag for converting old segment indexing versions to the latest segment indexing version.|false| |`druid.coordinator.load.timeout`|The timeout duration for when the coordinator assigns a segment to a historical node.|PT15M| +|`druid.coordinator.kill.pendingSegments.on`|Boolean flag for whether or not the coordinator clean up old entries in the `pendingSegments` table of metadata store. If set to true, coordinator will check the created time of most recently complete task. If it doesn't exist, it finds the created time of the earlist running/pending/waiting tasks. Once the created time is found, then for all dataSources not in the `killPendingSegmentsSkipList` (see [Dynamic configuration](#dynamic-configuration)), coordinator will ask the overlord to clean up the entries 1 day or more older than the found created time in the `pendingSegments` table. This will be done periodically based on `druid.coordinator.period` specified.|false| |`druid.coordinator.kill.on`|Boolean flag for whether or not the coordinator should submit kill task for unused segments, that is, hard delete them from metadata store and deep storage. If set to true, then for all whitelisted dataSources (or optionally all), coordinator will submit tasks periodically based on `period` specified. These kill tasks will delete all segments except for the last `durationToRetain` period. Whitelist or All can be set via dynamic configuration `killAllDataSources` and `killDataSourceWhitelist` described later.|false| |`druid.coordinator.kill.period`|How often to send kill tasks to the indexing service. Value must be greater than `druid.coordinator.period.indexingPeriod`. Only applies if kill is turned on.|P1D (1 Day)| |`druid.coordinator.kill.durationToRetain`| Do not kill segments in last `durationToRetain`, must be greater or equal to 0. Only applies and MUST be specified if kill is turned on. Note that default value is invalid.|PT-1S (-1 seconds)| @@ -103,8 +104,9 @@ Issuing a GET request at the same URL will return the spec that is currently in |`replicantLifetime`|The maximum number of coordinator runs for a segment to be replicated before we start alerting.|15| |`replicationThrottleLimit`|The maximum number of segments that can be replicated at one time.|10| |`emitBalancingStats`|Boolean flag for whether or not we should emit balancing stats. This is an expensive operation.|false| -|`killDataSourceWhitelist`|List of dataSources for which kill tasks are sent if property `druid.coordinator.kill.on` is true.|none| +|`killDataSourceWhitelist`|List of dataSources for which kill tasks are sent if property `druid.coordinator.kill.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none| |`killAllDataSources`|Send kill tasks for ALL dataSources if property `druid.coordinator.kill.on` is true. If this is set to true then `killDataSourceWhitelist` must not be specified or be empty list.|false| +|`killPendingSegmentsSkipList`|List of dataSources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none| |`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of nodes. Value 1000 could be a start point for a rather big cluster. Default value is 0 (loading queue is unbounded) |0| To view the audit history of coordinator dynamic config issue a GET request to the URL - diff --git a/docs/content/configuration/historical.md b/docs/content/configuration/historical.md index 9290e265d27e..7d2d1ff49873 100644 --- a/docs/content/configuration/historical.md +++ b/docs/content/configuration/historical.md @@ -54,6 +54,8 @@ Druid uses Jetty to serve HTTP requests. |`druid.server.http.maxIdleTime`|The Jetty max idle time for a connection.|PT5m| |`druid.server.http.enableRequestLimit`|If enabled, no requests would be queued in jetty queue and "HTTP 429 Too Many Requests" error response would be sent. |false| |`druid.server.http.defaultQueryTimeout`|Query timeout in millis, beyond which unfinished queries will be cancelled|300000| +|`druid.server.http.gracefulShutdownTimeout`|The maximum amount of time Jetty waits after receiving shutdown signal. After this timeout the threads will be forcefully shutdown. This allows any queries that are executing to complete.|`PT0s` (do not wait)| +|`druid.server.http.unannouncePropogationDelay`|How long to wait for zookeeper unannouncements to propgate before shutting down Jetty. This is a minimum and `druid.server.http.gracefulShutdownTimeout` does not start counting down until after this period elapses.|`PT0s` (do not wait)| |`druid.server.http.maxQueryTimeout`|Maximum allowed value (in milliseconds) for `timeout` parameter. See [query-context](query-context.html) to know more about `timeout`. Query is rejected if the query context `timeout` is greater than this value. |Long.MAX_VALUE| |`druid.server.http.maxRequestHeaderSize`|Maximum size of a request header in bytes. Larger headers consume more memory and can make a server more vulnerable to denial of service attacks.|8 * 1024| diff --git a/docs/content/development/extensions-contrib/influx.md b/docs/content/development/extensions-contrib/influx.md new file mode 100644 index 000000000000..a8d29718a1f0 --- /dev/null +++ b/docs/content/development/extensions-contrib/influx.md @@ -0,0 +1,46 @@ +--- +layout: doc_page +--- + +# InfluxDB Line Protocol Parser + +To use this extension, make sure to [include](../../operations/including-extensions.html) `druid-influx-extensions`. + +This extension enables Druid to parse the [InfluxDB Line Protocol](https://docs.influxdata.com/influxdb/v1.5/write_protocols/line_protocol_tutorial/), a popular text-based timeseries metric serialization format. + +## Line Protocol + +A typical line looks like this: + +```cpu,application=dbhost=prdb123,region=us-east-1 usage_idle=99.24,usage_user=0.55 1520722030000000000``` + +which contains four parts: + - measurement: A string indicating the name of the measurement represented (e.g. cpu, network, web_requests) + - tags: zero or more key-value pairs (i.e. dimensions) + - measurements: one or more key-value pairs; values can be numeric, boolean, or string + - timestamp: nanoseconds since Unix epoch (the parser truncates it to milliseconds) + +The parser extracts these fields into a map, giving the measurement the key `measurement` and the timestamp the key `_ts`. The tag and measurement keys are copied verbatim, so users should take care to avoid name collisions. It is up to the ingestion spec to decide which fields should be treated as dimensions and which should be treated as metrics (typically tags correspond to dimensions and measurements correspond to metrics). + +The parser is configured like so: +```json +"parser": { + "type": "string", + "parseSpec": { + "format": "influx", + "timestampSpec": { + "column": "__ts", + "format": "millis" + }, + "dimensionsSpec": { + "dimensionExclusions": [ + "__ts" + ] + }, + "whitelistMeasurements": [ + "cpu" + ] + } +``` + +The `whitelistMeasurements` field is an optional list of strings. If present, measurements that do not match one of the strings in the list will be ignored. diff --git a/docs/content/development/extensions-core/avro.md b/docs/content/development/extensions-core/avro.md index a8a4b820e221..57d6355b6e32 100644 --- a/docs/content/development/extensions-core/avro.md +++ b/docs/content/development/extensions-core/avro.md @@ -16,7 +16,7 @@ This is for streaming/realtime ingestion. | avroBytesDecoder | JSON Object | Specifies how to decode bytes to Avro record. | yes | | parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be an "avro" parseSpec. | yes | -An Avro parseSpec can contain a [flattenSpec](../../ingestion/flatten-spec.html) using either the "root" or "path" +An Avro parseSpec can contain a [flattenSpec](../../ingestion/flatten-json.html) using either the "root" or "path" field types, which can be used to read nested Avro records. The "jq" field type is not currently supported for Avro. For example, using Avro stream parser with schema repo Avro bytes decoder: diff --git a/docs/content/development/extensions-core/datasketches-extension.md b/docs/content/development/extensions-core/datasketches-extension.md new file mode 100644 index 000000000000..a8a1d8738b82 --- /dev/null +++ b/docs/content/development/extensions-core/datasketches-extension.md @@ -0,0 +1,19 @@ +--- +layout: doc_page +--- + +## DataSketches extension + +Druid aggregators based on [datasketches](http://datasketches.github.io/) library. Sketches are data structures implementing approximate streaming mergeable algorithms. Sketches can be ingested from the outside of Druid or built from raw data at ingestion time. Sketches can be stored in Druid segments as additive metrics. + +To use the datasketch aggregators, make sure you [include](../../operations/including-extensions.html) the extension in your config file: + +``` +druid.extensions.loadList=["druid-datasketches"] +``` + +The following aggregators are available: + +1. [Theta sketch](datasketches-theta.html), useful for approximate set counting, and supporting union, intersection, and difference operations. +2. [Quantiles sketch](datasketches-quantiles.html). +3. [Tuple sketch](datasketches-tuple.html). diff --git a/docs/content/development/extensions-core/datasketches-quantiles.md b/docs/content/development/extensions-core/datasketches-quantiles.md new file mode 100644 index 000000000000..c9eeb84c1777 --- /dev/null +++ b/docs/content/development/extensions-core/datasketches-quantiles.md @@ -0,0 +1,92 @@ +--- +layout: doc_page +--- + +## DataSketches Quantiles Sketch module + +This module provides Druid aggregators based on numeric quantiles DoublesSketch from [datasketches](http://datasketches.github.io/) library. Quantiles sketch is a mergeable streaming algorithm to estimate the distribution of values, and approximately answer queries about the rank of a value, probability mass function of the distribution (PMF) or histogram, cummulative distribution function (CDF), and quantiles (median, min, max, 95th percentile and such). See [Quantiles Sketch Overview](https://datasketches.github.io/docs/Quantiles/QuantilesOverview.html). + +There are three major modes of operation: + +1. Ingesting sketches built outside of Druid (say, with Pig or Hive) +2. Building sketches from raw data during ingestion +3. Building sketches from raw data at query time + +To use this aggregator, make sure you [include](../../operations/including-extensions.html) the extension in your config file: + +``` +druid.extensions.loadList=["druid-datasketches"] +``` + +### Aggregator + +The result of the aggregation is a DoublesSketch that is the union of all sketches either built from raw data or read from the segments. + +```json +{ + "type" : "quantilesDoublesSketch", + "name" : , + "fieldName" : , + "k": + } +``` + +|property|description|required?| +|--------|-----------|---------| +|type|This String should always be "quantilesDoublesSketch"|yes| +|name|A String for the output (result) name of the calculation.|yes| +|fieldName|A String for the name of the input field (can contain sketches or raw numeric values).|yes| +|k|Parameter that determines the accuracy and size of the sketch. Higher k means higher accuracy but more space to store sketches. Must be a power of 2 from 2 to 32768. See the [Quantiles Accuracy](https://datasketches.github.io/docs/Quantiles/QuantilesAccuracy.html) for details. |no, defaults to 128| + +### Post Aggregators + +#### Quantile + +This returns an approximation to the value that would be preceded by a given fraction of a hypothetical sorted version of the input stream. + +```json +{ + "type" : "quantilesDoublesSketchToQuantile", + "name": , + "field" : , + "fraction" : +} +``` + +#### Quantiles + +This returns an array of quantiles corresponding to a given array of fractions + +```json +{ + "type" : "quantilesDoublesSketchToQuantiles", + "name": , + "field" : , + "fractions" : +} +``` + +#### Histogram + +This returns an approximation to the histogram given an array of split points that define the histogram bins. An array of m unique, monotonically increasing split points divide the real number line into m+1 consecutive disjoint intervals. The definition of an interval is inclusive of the left split point and exclusive of the right split point. + +```json +{ + "type" : "quantilesDoublesSketchToHistogram", + "name": , + "field" : , + "splitPoints" : +} +``` + +#### Sketch Summary + +This returns a summary of the sketch that can be used for debugging. This is the result of calling toString() method. + +```json +{ + "type" : "quantilesDoublesSketchToString", + "name": , + "field" : +} +``` diff --git a/docs/content/development/extensions-core/datasketches-aggregators.md b/docs/content/development/extensions-core/datasketches-theta.md similarity index 91% rename from docs/content/development/extensions-core/datasketches-aggregators.md rename to docs/content/development/extensions-core/datasketches-theta.md index bf7e4a601f39..07d179555bef 100644 --- a/docs/content/development/extensions-core/datasketches-aggregators.md +++ b/docs/content/development/extensions-core/datasketches-theta.md @@ -2,13 +2,13 @@ layout: doc_page --- -## DataSketches aggregator +## DataSketches Theta Sketch module -Druid aggregators based on [datasketches](http://datasketches.github.io/) library. Note that sketch algorithms are approximate; see details in the "Accuracy" section of the datasketches doc. -At ingestion time, this aggregator creates the theta sketch objects which get stored in Druid segments. Logically speaking, a theta sketch object can be thought of as a Set data structure. At query time, sketches are read and aggregated (set unioned) together. In the end, by default, you receive the estimate of the number of unique entries in the sketch object. Also, you can use post aggregators to do union, intersection or difference on sketch columns in the same row. -Note that you can use `thetaSketch` aggregator on columns which were not ingested using same, it will return estimated cardinality of the column. It is recommended to use it at ingestion time as well to make querying faster. +This module provides Druid aggregators based on Theta sketch from [datasketches](http://datasketches.github.io/) library. Note that sketch algorithms are approximate; see details in the "Accuracy" section of the datasketches doc. +At ingestion time, this aggregator creates the Theta sketch objects which get stored in Druid segments. Logically speaking, a Theta sketch object can be thought of as a Set data structure. At query time, sketches are read and aggregated (set unioned) together. In the end, by default, you receive the estimate of the number of unique entries in the sketch object. Also, you can use post aggregators to do union, intersection or difference on sketch columns in the same row. +Note that you can use `thetaSketch` aggregator on columns which were not ingested using the same. It will return estimated cardinality of the column. It is recommended to use it at ingestion time as well to make querying faster. -To use the datasketch aggregators, make sure you [include](../../operations/including-extensions.html) the extension in your config file: +To use this aggregator, make sure you [include](../../operations/including-extensions.html) the extension in your config file: ``` druid.extensions.loadList=["druid-datasketches"] diff --git a/docs/content/development/extensions-core/datasketches-tuple.md b/docs/content/development/extensions-core/datasketches-tuple.md new file mode 100644 index 000000000000..37ed8ead4004 --- /dev/null +++ b/docs/content/development/extensions-core/datasketches-tuple.md @@ -0,0 +1,155 @@ +--- +layout: doc_page +--- + +## DataSketches Tuple Sketch module + +This module provides Druid aggregators based on Tuple sketch from [datasketches](http://datasketches.github.io/) library. ArrayOfDoublesSketch sketches extend the functionality of the count-distinct Theta sketches by adding arrays of double values associated with unique keys. + +To use this aggregator, make sure you [include](../../operations/including-extensions.html) the extension in your config file: + +``` +druid.extensions.loadList=["druid-datasketches"] +``` + +### Aggregators + +```json +{ + "type" : "arrayOfDoublesSketch", + "name" : , + "fieldName" : , + "nominalEntries": , + "numberOfValues" : , + "metricColumns" : + } +``` + +|property|description|required?| +|--------|-----------|---------| +|type|This String should always be "arrayOfDoublesSketch"|yes| +|name|A String for the output (result) name of the calculation.|yes| +|fieldName|A String for the name of the input field.|yes| +|nominalEntries|Parameter that determines the accuracy and size of the sketch. Higher k means higher accuracy but more space to store sketches. Must be a power of 2. See the [Theta sketch accuracy](https://datasketches.github.io/docs/Theta/ThetaErrorTable.html) for details. |no, defaults to 16384| +|numberOfValues|Number of values associated with each distinct key. |no, defaults to 1| +|metricCoulumns|If building sketches from raw data, an array of names of the input columns containing numeric vaues to be associated with each distinct key.|no, defaults to empty array| + +### Post Aggregators + +#### Estimate of the number of distinct keys + +Returns a distinct count estimate from a given ArrayOfDoublesSketch. + +```json +{ + "type" : "arrayOfDoublesSketchToEstimate", + "name": , + "field" : +} +``` + +#### Estimate of the number of distinct keys with error bounds + +Returns a distinct count estimate and error bounds from a given ArrayOfDoublesSketch. The result will be three double values: estimate of the number of distinct keys, lower bound and upper bound. The bounds are provided at the given number of standard deviations (optional, defaults to 1). This must be an integer value of 1, 2 or 3 corresponding to approximately 68.3%, 95.4% and 99.7% confidence intervals. + +```json +{ + "type" : "arrayOfDoublesSketchToEstimateAndBounds", + "name": , + "field" : , + "numStdDevs", +} +``` + +#### Number of retained entries + +Returns the number of retained entries from a given ArrayOfDoublesSketch. + +```json +{ + "type" : "arrayOfDoublesSketchToNumEntries", + "name": , + "field" : +} +``` + +#### Mean values for each column + +Returns a list of mean values from a given ArrayOfDoublesSketch. The result will be N double values, where N is the number of double values kept in the sketch per key. + +```json +{ + "type" : "arrayOfDoublesSketchToMeans", + "name": , + "field" : +} +``` + +#### Variance values for each column + +Returns a list of variance values from a given ArrayOfDoublesSketch. The result will be N double values, where N is the number of double values kept in the sketch per key. + +```json +{ + "type" : "arrayOfDoublesSketchToVariances", + "name": , + "field" : +} +``` + +#### Quantiles sketch from a column + +Returns a quanitles DoublesSketch constructed from a given column of values from a given ArrayOfDoublesSketch using optional parameter k that determines the accuracy and size of the quantiles sketch. See [Quantiles Sketch Module](datasketches-quantiles.html) + +* The column number is 1-based and is optional (the default is 1). +* The parameter k is optional (the default is defined in the sketch library). +* The result is a quantiles sketch. + +```json +{ + "type" : "arrayOfDoublesSketchToQuantilesSketch", + "name": , + "field" : , + "column" : , + "k" : +} +``` + +#### Set Operations + +Returns a result of a specified set operation on the given array of sketches. Supported operations are: union, intersection and set difference (UNION, INTERSECT, NOT). + +```json +{ + "type" : "arrayOfDoublesSketchSetOp", + "name": , + "func": <"UNION"|"INTERSECT"|"NOT">, + "fields" : , + "nominalEntries" : , + "numberOfValues" : +} +``` + +#### Student's t-test + +Performs Student's t-test and returns a list of p-values given two instances of ArrayOfDoublesSketch. The result will be N double values, where N is the number of double values kept in the sketch per key. See [t-test documentation](http://commons.apache.org/proper/commons-math/javadocs/api-3.4/org/apache/commons/math3/stat/inference/TTest.html). + +```json +{ + "type" : "arrayOfDoublesSketchTTest", + "name": , + "fields" : , +} +``` + +#### Sketch summary + +Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a string returned by toString() method of the sketch. This can be useful for debugging. + +```json +{ + "type" : "arrayOfDoublesSketchToString", + "name": , + "field" : +} +``` diff --git a/docs/content/development/extensions-core/druid-kerberos.md b/docs/content/development/extensions-core/druid-kerberos.md index 1ded0e0f5a54..dc0d38a7b5af 100644 --- a/docs/content/development/extensions-core/druid-kerberos.md +++ b/docs/content/development/extensions-core/druid-kerberos.md @@ -31,8 +31,6 @@ The configuration examples in the rest of this document will use "kerberos" as t ### Properties |Property|Possible Values|Description|Default|required| |--------|---------------|-----------|-------|--------| -|`druid.auth.authenticator.kerberos.internalClientPrincipal`|`druid@EXAMPLE.COM`| Principal user name, used for internal node communication|empty|Yes| -|`druid.auth.authenticator.kerberos.internalClientKeytab`|`/etc/security/keytabs/druid.keytab`|Path to keytab file used for internal node communication|empty|Yes| |`druid.auth.authenticator.kerberos.serverPrincipal`|`HTTP/_HOST@EXAMPLE.COM`| SPNego service principal used by druid nodes|empty|Yes| |`druid.auth.authenticator.kerberos.serverKeytab`|`/etc/security/keytabs/spnego.service.keytab`|SPNego service keytab used by druid nodes|empty|Yes| |`druid.auth.authenticator.kerberos.authToLocal`|`RULE:[1:$1@$0](druid@EXAMPLE.COM)s/.*/druid DEFAULT`|It allows you to set a general rule for mapping principal names to local user names. It will be used if there is not an explicit mapping for the principal name that is being translated.|DEFAULT|No| @@ -54,6 +52,17 @@ In Active Directory environment, SPNEGO token in the Authorization header includ which includes all security groups for the user. In some cases when the user belongs to many security groups the header to grow beyond what druid can handle by default. In such cases, max request header size that druid can handle can be increased by setting `druid.server.http.maxRequestHeaderSize` (default 8Kb) and `druid.router.http.maxRequestBufferSize` (default 8Kb). +## Configuring Kerberos Escalated Client + +Druid internal nodes communicate with each other using an escalated http Client. A Kerberos enabled escalated HTTP Client can be configured by following properties - + + +|Property|Example Values|Description|Default|required| +|--------|---------------|-----------|-------|--------| +|`druid.escalator.type`|`kerberos`| Type of Escalator client used for internal node communication.|n/a|Yes| +|`druid.escalator.internalClientPrincipal`|`druid@EXAMPLE.COM`| Principal user name, used for internal node communication|n/a|Yes| +|`druid.escalator.internalClientKeytab`|`/etc/security/keytabs/druid.keytab`|Path to keytab file used for internal node communication|n/a|Yes| +|`druid.escalator.authorizerName`|`MyBasicAuthorizer`|Authorizer that requests should be directed to.|n/a|Yes| ## Accessing Druid HTTP end points when kerberos security is enabled 1. To access druid HTTP endpoints via curl user will need to first login using `kinit` command as follows - diff --git a/docs/content/development/extensions-core/kafka-extraction-namespace.md b/docs/content/development/extensions-core/kafka-extraction-namespace.md index fbf8915f4248..e1343296d62b 100644 --- a/docs/content/development/extensions-core/kafka-extraction-namespace.md +++ b/docs/content/development/extensions-core/kafka-extraction-namespace.md @@ -25,7 +25,7 @@ If you need updates to populate as promptly as possible, it is possible to plug |`kafkaTopic`|The kafka topic to read the data from|Yes|| |`kafkaProperties`|Kafka consumer properties. At least"zookeeper.connect" must be specified. Only the zookeeper connector is supported|Yes|| |`connectTimeout`|How long to wait for an initial connection|No|`0` (do not wait)| -|`isOneToOne`|The map is a one-to-one (see[Lookup DimensionSpecs](../querying/dimensionspecs.html))|No|`false`| +|`isOneToOne`|The map is a one-to-one (see [Lookup DimensionSpecs](../../querying/dimensionspecs.html))|No|`false`| The extension `kafka-extraction-namespace` enables reading from a kafka feed which has name/key pairs to allow renaming of dimension values. An example use case would be to rename an ID to a human readable format. diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index e0d7f73c689d..2c09b64e15f0 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -164,7 +164,7 @@ For Roaring bitmaps: |`consumerProperties`|Map|A map of properties to be passed to the Kafka consumer. This must contain a property `bootstrap.servers` with a list of Kafka brokers in the form: `:,:,...`.|yes| |`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against node failure.|no (default == 1)| |`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See 'Capacity Planning' below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKafkaPartitions}`.|no (default == 1)| -|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment. Note that segments are only pushed to deep storage and loadable by historical nodes when the indexing task completes.|no (default == PT1H)| +|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment.|no (default == PT1H)| |`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)| |`period`|ISO8601 Period|How often the supervisor will execute its management logic. Note that the supervisor will also run in response to certain events (such as tasks succeeding, failing, and reaching their taskDuration) so this value specifies the maximum time between iterations.|no (default == PT30S)| |`useEarliestOffset`|Boolean|If a supervisor is managing a dataSource for the first time, it will obtain a set of starting offsets from Kafka. This flag determines whether it retrieves the earliest or latest offsets in Kafka. Under normal circumstances, subsequent tasks will start from where the previous segments ended so this flag will only be used on first run.|no (default == false)| diff --git a/docs/content/development/extensions-core/lookups-cached-global.md b/docs/content/development/extensions-core/lookups-cached-global.md index a5e7d5eb04ff..7199e76ded23 100644 --- a/docs/content/development/extensions-core/lookups-cached-global.md +++ b/docs/content/development/extensions-core/lookups-cached-global.md @@ -86,23 +86,26 @@ In a simple case where only one [tier](../../querying/lookups.html#dynamic-confi { "realtime_customer2": { "country_code": { - "type": "cachedNamespace", - "extractionNamespace": { - "type": "jdbc", - "connectorConfig": { - "createTables": true, - "connectURI": "jdbc:mysql:\/\/localhost:3306\/druid", - "user": "druid", - "password": "diurd" - }, - "table": "lookupValues", - "keyColumn": "value_id", - "valueColumn": "value_text", - "filter": "value_type='country'", - "tsColumn": "timeColumn" - }, - "firstCacheTimeout": 120000, - "injective":true + "version": "v0", + "lookupExtractorFactory": { + "type": "cachedNamespace", + "extractionNamespace": { + "type": "jdbc", + "connectorConfig": { + "createTables": true, + "connectURI": "jdbc:mysql:\/\/localhost:3306\/druid", + "user": "druid", + "password": "diurd" + }, + "table": "lookupValues", + "keyColumn": "value_id", + "valueColumn": "value_text", + "filter": "value_type='country'", + "tsColumn": "timeColumn" + }, + "firstCacheTimeout": 120000, + "injective": true + } } } } @@ -112,22 +115,26 @@ Where the coordinator endpoint `/druid/coordinator/v1/lookups/realtime_customer2 ```json { - "type": "cachedNamespace", - "extractionNamespace": { - "type": "jdbc", - "connectorConfig": { - "createTables": true, - "connectURI": "jdbc:mysql:\/\/localhost:3306\/druid", - "user": "druid", - "password": "diurd" + "version": "v0", + "lookupExtractorFactory": { + "type": "cachedNamespace", + "extractionNamespace": { + "type": "jdbc", + "connectorConfig": { + "createTables": true, + "connectURI": "jdbc:mysql://localhost:3306/druid", + "user": "druid", + "password": "diurd" + }, + "table": "lookupValues", + "keyColumn": "value_id", + "valueColumn": "value_text", + "filter": "value_type='country'", + "tsColumn": "timeColumn" }, - "table": "lookupTable", - "keyColumn": "country_id", - "valueColumn": "country_name", - "tsColumn": "timeColumn" - }, - "firstCacheTimeout": 120000, - "injective":true + "firstCacheTimeout": 120000, + "injective": true + } } ``` diff --git a/docs/content/development/extensions-core/mysql.md b/docs/content/development/extensions-core/mysql.md index 3276811eaed2..a51b84ffa9da 100644 --- a/docs/content/development/extensions-core/mysql.md +++ b/docs/content/development/extensions-core/mysql.md @@ -65,11 +65,11 @@ Make sure to [include](../../operations/including-extensions.html) `mysql-metada |`druid.metadata.mysql.ssl.useSSL`|Enable SSL|`false`|no| |`druid.metadata.mysql.ssl.clientCertificateKeyStoreUrl`|The file path URL to the client certificate key store.|none|no| |`druid.metadata.mysql.ssl.clientCertificateKeyStoreType`|The type of the key store where the client certificate is stored.|none|no| -|`druid.metadata.mysql.ssl.clientCertificateKeyStorePassword`|The [Password Provider](../operations/password-provider.html) or String password for the client key store.|none|no| +|`druid.metadata.mysql.ssl.clientCertificateKeyStorePassword`|The [Password Provider](../../operations/password-provider.html) or String password for the client key store.|none|no| |`druid.metadata.mysql.ssl.verifyServerCertificate`|Enables server certificate verification.|false|no| |`druid.metadata.mysql.ssl.trustCertificateKeyStoreUrl`|The file path to the trusted root certificate key store.|Default trust store provided by MySQL|yes if `verifyServerCertificate` is set to true and a custom trust store is used| |`druid.metadata.mysql.ssl.trustCertificateKeyStoreType`|The type of the key store where trusted root certificates are stored.|JKS|yes if `verifyServerCertificate` is set to true and keystore type is not JKS| -|`druid.metadata.mysql.ssl.trustCertificateKeyStorePassword`|The [Password Provider](../operations/password-provider.html) or String password for the trust store.|none|yes if `verifyServerCertificate` is set to true and password is not null| +|`druid.metadata.mysql.ssl.trustCertificateKeyStorePassword`|The [Password Provider](../../operations/password-provider.html) or String password for the trust store.|none|yes if `verifyServerCertificate` is set to true and password is not null| |`druid.metadata.mysql.ssl.enabledSSLCipherSuites`|Overrides the existing cipher suites with these cipher suites.|none|no| |`druid.metadata.mysql.ssl.enabledTLSProtocols`|Overrides the TLS protocols with these protocols.|none|no| diff --git a/docs/content/development/extensions-core/protobuf.md b/docs/content/development/extensions-core/protobuf.md index c45e47c6ecd5..1a066c4d4b9f 100644 --- a/docs/content/development/extensions-core/protobuf.md +++ b/docs/content/development/extensions-core/protobuf.md @@ -18,7 +18,7 @@ This extension enables Druid to ingest and understand the Protobuf data format. ## Example: Load Protobuf messages from Kafka -This example demonstrates how to load Protobuf messages from Kafka. Please read the [Load from Kafka tutorial](../../tutorial/tutorial-kafka.html) first. This example will use the same "metrics" dataset. +This example demonstrates how to load Protobuf messages from Kafka. Please read the [Load from Kafka tutorial](../../tutorials/tutorial-kafka.html) first. This example will use the same "metrics" dataset. Files used in this example are found at `./examples/quickstart/protobuf` in your Druid directory. diff --git a/docs/content/development/extensions-core/s3.md b/docs/content/development/extensions-core/s3.md index f24a406946b6..3d32c9de8953 100644 --- a/docs/content/development/extensions-core/s3.md +++ b/docs/content/development/extensions-core/s3.md @@ -12,12 +12,18 @@ S3-compatible deep storage is basically either S3 or something like Google Stora ### Configuration -|Property|Possible Values|Description|Default| -|--------|---------------|-----------|-------| -|`druid.s3.accessKey`||S3 access key.|Must be set.| -|`druid.s3.secretKey`||S3 secret key.|Must be set.| -|`druid.storage.bucket`||Bucket to store in.|Must be set.| -|`druid.storage.baseKey`||Base key prefix to use, i.e. what directory.|Must be set.| +|Property|Description|Default| +|--------|-----------|-------| +|`druid.s3.accessKey`|S3 access key.|Must be set.| +|`druid.s3.secretKey`|S3 secret key.|Must be set.| +|`druid.storage.bucket`|Bucket to store in.|Must be set.| +|`druid.storage.baseKey`|Base key prefix to use, i.e. what directory.|Must be set.| +|`druid.s3.endpoint.url`|Service endpoint either with or without the protocol.|None| +|`druid.s3.endpoint.signingRegion`|Region to use for SigV4 signing of requests (e.g. us-west-1).|None| +|`druid.s3.proxy.host`|Proxy host to connect through.|None| +|`druid.s3.proxy.port`|Port on the proxy host to connect through.|None| +|`druid.s3.proxy.username`|User name to use when connecting through a proxy.|None| +|`druid.s3.proxy.password`|Password to use when connecting through a proxy.|None| ## StaticS3Firehose diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md index 4e2bea613fad..41c725ec91b4 100644 --- a/docs/content/development/extensions.md +++ b/docs/content/development/extensions.md @@ -24,7 +24,7 @@ Core extensions are maintained by Druid committers. |druid-avro-extensions|Support for data in Apache Avro data format.|[link](../development/extensions-core/avro.html)| |druid-basic-security|Support for Basic HTTP authentication and role-based access control.|[link](../development/extensions-core/druid-basic-security.html)| |druid-caffeine-cache|A local cache implementation backed by Caffeine.|[link](../development/extensions-core/caffeine-cache.html)| -|druid-datasketches|Support for approximate counts and set operations with [DataSketches](http://datasketches.github.io/).|[link](../development/extensions-core/datasketches-aggregators.html)| +|druid-datasketches|Support for approximate counts and set operations with [DataSketches](http://datasketches.github.io/).|[link](../development/extensions-core/datasketches-extension.html)| |druid-hdfs-storage|HDFS deep storage.|[link](../development/extensions-core/hdfs.html)| |druid-histogram|Approximate histograms and quantiles aggregator.|[link](../development/extensions-core/approximate-histograms.html)| |druid-kafka-eight|Kafka ingest firehose (high level consumer) for realtime nodes.|[link](../development/extensions-core/kafka-eight-firehose.html)| diff --git a/docs/content/ingestion/index.md b/docs/content/ingestion/index.md index b533db622a09..3672a1060c49 100644 --- a/docs/content/ingestion/index.md +++ b/docs/content/ingestion/index.md @@ -258,7 +258,7 @@ This spec is used to generated segments with uniform intervals. | segmentGranularity | string | The granularity to create segments at. | no (default == 'DAY') | | queryGranularity | string | The minimum granularity to be able to query results at and the granularity of the data inside the segment. E.g. a value of "minute" will mean that data is aggregated at minutely granularity. That is, if there are collisions in the tuple (minute(timestamp), dimensions), then it will aggregate values together using the aggregators instead of storing individual rows. | no (default == 'NONE') | | rollup | boolean | rollup or not | no (default == true) | -| intervals | string | A list of intervals for the raw data being ingested. Ignored for real-time ingestion. | yes for batch, no for real-time | +| intervals | string | A list of intervals for the raw data being ingested. Ignored for real-time ingestion. | no. If specified, batch ingestion tasks may skip determining partitions phase which results in faster ingestion. | ### Arbitrary Granularity Spec @@ -268,7 +268,7 @@ This spec is used to generate segments with arbitrary intervals (it tries to cre |-------|------|-------------|----------| | queryGranularity | string | The minimum granularity to be able to query results at and the granularity of the data inside the segment. E.g. a value of "minute" will mean that data is aggregated at minutely granularity. That is, if there are collisions in the tuple (minute(timestamp), dimensions), then it will aggregate values together using the aggregators instead of storing individual rows. | no (default == 'NONE') | | rollup | boolean | rollup or not | no (default == true) | -| intervals | string | A list of intervals for the raw data being ingested. Ignored for real-time ingestion. | yes for batch, no for real-time | +| intervals | string | A list of intervals for the raw data being ingested. Ignored for real-time ingestion. | no. If specified, batch ingestion tasks may skip determining partitions phase which results in faster ingestion. | # IO Config diff --git a/docs/content/querying/aggregations.md b/docs/content/querying/aggregations.md index 420daf97afbb..b0ce5cc24c9d 100644 --- a/docs/content/querying/aggregations.md +++ b/docs/content/querying/aggregations.md @@ -332,7 +332,7 @@ The HyperLogLog algorithm generates decimal estimates with some error. "round" c values to whole numbers. Note that even with rounding, the cardinality is still an estimate. The "round" field only affects query-time behavior, and is ignored at ingestion-time. -For more approximate aggregators, please see [theta sketches](../development/extensions-core/datasketches-aggregators.html). +For more approximate aggregators, check out the [DataSketches extension](../development/extensions-core/datasketches-extension.html). ## Miscellaneous Aggregations diff --git a/docs/content/querying/caching.md b/docs/content/querying/caching.md index 1d6f60af441e..2dd2c88d6b62 100644 --- a/docs/content/querying/caching.md +++ b/docs/content/querying/caching.md @@ -3,9 +3,10 @@ layout: doc_page --- # Query Caching -Druid supports query result caching through an LRU cache. Results are stored on a per segment basis, along with the -parameters of a given query. This allows Druid to return final results based partially on segment results in the cache and partially -on segment results from scanning historical/real-time segments. +Druid supports query result caching through an LRU cache. Results are stored as a whole or either on a per segment basis along with the +parameters of a given query. Segment level caching allows Druid to return final results based partially on segment results in the cache +and partially on segment results from scanning historical/real-time segments. Result level caching enables Druid to cache the entire +result set, so that query results can be completely retrieved from the cache for identical queries. Segment results can be stored in a local heap cache or in an external distributed key/value store. Segment query caches can be enabled at either the Historical and Broker level (it is not recommended to enable caching on both). @@ -15,6 +16,7 @@ can be enabled at either the Historical and Broker level (it is not recommended Enabling caching on the broker can yield faster results than if query caches were enabled on Historicals for small clusters. This is the recommended setup for smaller production clusters (< 20 servers). Take note that when caching is enabled on the Broker, results from Historicals are returned on a per segment basis, and Historicals will not be able to do any local result merging. +Result level caching is enabled only on the Broker side. ## Query caching on Historicals diff --git a/docs/content/querying/datasourcemetadataquery.md b/docs/content/querying/datasourcemetadataquery.md index a812c15633ba..daee0dfdf301 100644 --- a/docs/content/querying/datasourcemetadataquery.md +++ b/docs/content/querying/datasourcemetadataquery.md @@ -29,7 +29,7 @@ The format of the result is: [ { "timestamp" : "2013-05-09T18:24:00.000Z", "result" : { - "maxIngestedEventTime" : "2013-05-09T18:24:09.007Z", + "maxIngestedEventTime" : "2013-05-09T18:24:09.007Z" } } ] ``` diff --git a/docs/content/querying/filters.md b/docs/content/querying/filters.md index d5e30caa8ac1..d158b893d36f 100644 --- a/docs/content/querying/filters.md +++ b/docs/content/querying/filters.md @@ -487,3 +487,11 @@ Filtering on a set of ISO 8601 intervals: ] } ``` + + +### Noop Filter +The noop filter is a filter which applies no conditions to your query. Useful if you need to disable other filters when queries are generated programatically. + +```json +{ "type" : "noop" } +``` \ No newline at end of file diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index d67671fdd188..52dfddfd24d9 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -235,10 +235,8 @@ strategy perform the outer query on the broker in a single-threaded fashion. #### Configurations -This section describes the configurations for groupBy queries. You can set system-wide configurations by adding them to runtime properties or query-specific configurations by adding them to query contexts. All runtime properties are prefixed by `druid.query.groupBy`. - -#### Commonly tuned configurations - +This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on broker, historical, and MiddleManager nodes. You can set the query context parameters through the [query context](query-context.html). + ##### Configurations for groupBy v2 Supported runtime properties: diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index 771c9adfb743..b6b83977a637 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -364,9 +364,53 @@ It is possible to save the configuration across restarts such that a node will n ## Introspect a Lookup -Lookup implementations can provide some introspection capabilities by implementing `LookupIntrospectHandler`. User will send request to `/druid/lookups/v1/introspect/{lookupId}` to enable introspection on a given lookup. +The broker provides an API for lookup introspection if the lookup type implements a `LookupIntrospectHandler`. -For instance you can list all the keys/values of a map based lookup by issuing a `GET` request to `/druid/lookups/v1/introspect/{lookupId}/keys"` or `/druid/lookups/v1/introspect/{lookupId}/values"` +A `GET` request to `/druid/v1/lookups/introspect/{lookupId}` will return the map of complete values. + +ex: `GET /druid/v1/lookups/introspect/nato-phonetic` +``` +{ + "A": "Alfa", + "B": "Bravo", + "C": "Charlie", + ... + "Y": "Yankee", + "Z": "Zulu", + "-": "Dash" +} + +``` + +The list of keys can be retrieved via `GET` to `/druid/v1/lookups/introspect/{lookupId}/keys"` + +ex: `GET /druid/v1/lookups/introspect/nato-phonetic/keys` +``` +[ + "A", + "B", + "C", + ... + "Y", + "Z", + "-" +] +``` + +A `GET` request to `/druid/v1/lookups/introspect/{lookupId}/values"` will return the list of values. + +ex: `GET /druid/v1/lookups/introspect/nato-phonetic/values` +``` +[ + "Alfa", + "Bravo", + "Charlie", + ... + "Yankee", + "Zulu", + "Dash" +] +``` ## Druid version 0.10.0 to 0.10.1 upgrade/downgrade Overall druid cluster lookups configuration is persisted in metadata store and also individual lookup nodes optionally persist a snapshot of loaded lookups on disk. diff --git a/docs/content/querying/query-context.md b/docs/content/querying/query-context.md index b0effe81cf9d..d4e2be28f123 100644 --- a/docs/content/querying/query-context.md +++ b/docs/content/querying/query-context.md @@ -15,6 +15,8 @@ The query context is used for various query configuration parameters. The follow |queryId | auto-generated | Unique identifier given to this query. If a query ID is set or known, this can be used to cancel the query | |useCache | `true` | Flag indicating whether to leverage the query cache for this query. When set to false, it disables reading from the query cache for this query. When set to true, Druid uses druid.broker.cache.useCache or druid.historical.cache.useCache to determine whether or not to read from the query cache | |populateCache | `true` | Flag indicating whether to save the results of the query to the query cache. Primarily used for debugging. When set to false, it disables saving the results of this query to the query cache. When set to true, Druid uses druid.broker.cache.populateCache or druid.historical.cache.populateCache to determine whether or not to save the results of this query to the query cache | +|useResultLevelCache | `false` | Flag indicating whether to leverage the result level cache for this query. When set to false, it disables reading from the query cache for this query. When set to true, Druid uses druid.broker.cache.useResultLevelCache to determine whether or not to read from the query cache | +|populateResultLevelCache | `false` | Flag indicating whether to save the results of the query to the result level cache. Primarily used for debugging. When set to false, it disables saving the results of this query to the query cache. When set to true, Druid uses druid.broker.cache.populateCache to determine whether or not to save the results of this query to the query cache | |bySegment | `false` | Return "by segment" results. Primarily used for debugging, setting it to `true` returns results associated with the data segment they came from | |finalize | `true` | Flag indicating whether to "finalize" aggregation results. Primarily used for debugging. For instance, the `hyperUnique` aggregator will return the full HyperLogLog sketch instead of the estimated cardinality when this flag is set to `false` | |chunkPeriod | `P0D` (off) | At the broker node level, long interval queries (of any type) may be broken into shorter interval queries to parallelize merging more than normal. Broken up queries will use a larger share of cluster resources, but may be able to complete faster as a result. Use ISO 8601 periods. For example, if this property is set to `P1M` (one month), then a query covering a year would be broken into 12 smaller queries. The broker uses its query processing executor service to initiate processing for query chunks, so make sure "druid.processing.numThreads" is configured appropriately on the broker. [groupBy queries](groupbyquery.html) do not support chunkPeriod by default, although they do if using the legacy "v1" engine. | diff --git a/docs/content/querying/sql.md b/docs/content/querying/sql.md index 5796a66701c9..09778119f2c5 100644 --- a/docs/content/querying/sql.md +++ b/docs/content/querying/sql.md @@ -220,7 +220,7 @@ Druid does not support all SQL features, including: Additionally, some Druid features are not supported by the SQL language. Some unsupported Druid features include: - [Multi-value dimensions](multi-value-dimensions.html). -- [DataSketches aggregators](../development/extensions-core/datasketches-aggregators.html). +- [DataSketches aggregators](../development/extensions-core/datasketches-extension.html). - [Spatial filters](../development/geo.html). - [Query cancellation](querying.html#query-cancellation). @@ -382,7 +382,7 @@ Properties connectionProperties = new Properties(); try (Connection connection = DriverManager.getConnection(url, connectionProperties)) { try ( - final Statement statement = client.createStatement(); + final Statement statement = connection.createStatement(); final ResultSet resultSet = statement.executeQuery(query) ) { while (resultSet.next()) { diff --git a/docs/content/querying/timeseriesquery.md b/docs/content/querying/timeseriesquery.md index 08175c7f2649..9a42bfa87264 100644 --- a/docs/content/querying/timeseriesquery.md +++ b/docs/content/querying/timeseriesquery.md @@ -56,7 +56,7 @@ There are 7 main parts to a timeseries query: |filter|See [Filters](../querying/filters.html)|no| |aggregations|See [Aggregations](../querying/aggregations.html)|no| |postAggregations|See [Post Aggregations](../querying/post-aggregations.html)|no| -|context|See [Context](../querying/query-context.html)|no| +|context|Can be used to modify query behavior, including [grand totals](#grand-totals) and [zero-filling](#zero-filling). See also [Context](../querying/query-context.html) for parameters that apply to all query types.|no| To pull it all together, the above query would return 2 data points, one for each day between 2012-01-01 and 2012-01-03, from the "sample\_datasource" table. Each data point would be the (long) sum of sample\_fieldName1, the (double) sum of sample\_fieldName2 and the (double) result of sample\_fieldName1 divided by sample\_fieldName2 for the filter set. The output looks like this: @@ -73,6 +73,31 @@ To pull it all together, the above query would return 2 data points, one for eac ] ``` +#### Grand totals + +Druid can include an extra "grand totals" row as the last row of a timeseries result set. To enable this, add +`"grandTotal" : true` to your query context. For example: + +```json +{ + "queryType": "timeseries", + "dataSource": "sample_datasource", + "intervals": [ "2012-01-01T00:00:00.000/2012-01-03T00:00:00.000" ], + "granularity": "day", + "aggregations": [ + { "type": "longSum", "name": "sample_name1", "fieldName": "sample_fieldName1" }, + { "type": "doubleSum", "name": "sample_name2", "fieldName": "sample_fieldName2" } + ], + "context": { + "grandTotal": true + } +} +``` + +The grand totals row will appear as the last row in the result array, and will have no timestamp. It will be the last +row even if the query is run in "descending" mode. Post-aggregations in the grand totals row will be computed based +upon the grand total aggregations. + #### Zero-filling Timeseries queries normally fill empty interior time buckets with zeroes. For example, if you issue a "day" granularity diff --git a/docs/content/toc.md b/docs/content/toc.md index e51f95c05d46..a9d91c8b4fd4 100644 --- a/docs/content/toc.md +++ b/docs/content/toc.md @@ -13,7 +13,7 @@ layout: toc ## Data Ingestion * [Data Formats](/docs/VERSION/ingestion/data-formats.html) - * [Data Schema](/docs/VERSION/ingestion/index.html) + * [Ingestion Spec](/docs/VERSION/ingestion/index.html) * [Schema Design](/docs/VERSION/ingestion/schema-design.html) * [Schema Changes](/docs/VERSION/ingestion/schema-changes.html) * [Batch File Ingestion](/docs/VERSION/ingestion/batch-ingestion.html) @@ -103,7 +103,7 @@ layout: toc * Experimental Features * [Overview](/docs/VERSION/development/experimental.html) * [Approximate Histograms and Quantiles](/docs/VERSION/development/extensions-core/approximate-histograms.html) - * [Datasketches](/docs/VERSION/development/extensions-core/datasketches-aggregators.html) + * [Datasketches](/docs/VERSION/development/extensions-core/datasketches-extension.html) * [Geographic Queries](/docs/VERSION/development/geo.html) * [Router](/docs/VERSION/development/router.html) * [Kafka Indexing Service](/docs/VERSION/development/extensions-core/kafka-ingestion.html) diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java index e3ad8b432aa7..345cffd512cf 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java @@ -91,7 +91,7 @@ protected InputStream openObjectStream(AzureBlob object, long start) throws IOEx @Override protected InputStream wrapObjectStream(AzureBlob object, InputStream stream) throws IOException { - return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + return CompressionUtils.decompress(stream, object.getPath()); } private static AzureByteSource makeByteSource(AzureStorage azureStorage, AzureBlob object) diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureTaskLogs.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureTaskLogs.java index 2cd17c1c41c1..8fe5f3b39266 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureTaskLogs.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureTaskLogs.java @@ -54,7 +54,19 @@ public void pushTaskLog(final String taskid, final File logFile) { final String taskKey = getTaskLogKey(taskid); log.info("Pushing task log %s to: %s", logFile, taskKey); + pushTaskFile(taskid, logFile, taskKey); + } + + @Override + public void pushTaskReports(String taskid, File reportFile) throws IOException + { + final String taskKey = getTaskReportsKey(taskid); + log.info("Pushing task reports %s to: %s", reportFile, taskKey); + pushTaskFile(taskid, reportFile, taskKey); + } + private void pushTaskFile(final String taskId, final File logFile, String taskKey) + { try { AzureUtils.retryAzureOperation( () -> { @@ -71,9 +83,19 @@ public void pushTaskLog(final String taskid, final File logFile) @Override public Optional streamTaskLog(final String taskid, final long offset) throws IOException + { + return streamTaskFile(taskid, offset, getTaskLogKey(taskid)); + } + + @Override + public Optional streamTaskReports(String taskid) throws IOException + { + return streamTaskFile(taskid, 0, getTaskReportsKey(taskid)); + } + + private Optional streamTaskFile(final String taskid, final long offset, String taskKey) throws IOException { final String container = config.getContainer(); - final String taskKey = getTaskLogKey(taskid); try { if (!azureStorage.getBlobExists(container, taskKey)) { @@ -116,12 +138,16 @@ public InputStream openStream() throws IOException } } - private String getTaskLogKey(String taskid) { return StringUtils.format("%s/%s/log", config.getPrefix(), taskid); } + private String getTaskReportsKey(String taskid) + { + return StringUtils.format("%s/%s/report.json", config.getPrefix(), taskid); + } + @Override public void killAll() { diff --git a/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureTaskLogsTest.java b/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureTaskLogsTest.java index 5f8394a735a4..456d80b530e8 100644 --- a/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureTaskLogsTest.java +++ b/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureTaskLogsTest.java @@ -19,7 +19,6 @@ package io.druid.storage.azure; -import com.google.common.base.Charsets; import com.google.common.base.Optional; import com.google.common.io.ByteSource; import com.google.common.io.Files; @@ -34,6 +33,7 @@ import java.io.ByteArrayInputStream; import java.io.File; import java.io.StringWriter; +import java.nio.charset.StandardCharsets; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expectLastCall; @@ -88,7 +88,7 @@ public void testStreamTaskLogWithoutOffset() throws Exception expect(azureStorage.getBlobExists(container, blobPath)).andReturn(true); expect(azureStorage.getBlobLength(container, blobPath)).andReturn((long) testLog.length()); expect(azureStorage.getBlobInputStream(container, blobPath)).andReturn( - new ByteArrayInputStream(testLog.getBytes(Charsets.UTF_8))); + new ByteArrayInputStream(testLog.getBytes(StandardCharsets.UTF_8))); replayAll(); @@ -111,7 +111,7 @@ public void testStreamTaskLogWithPositiveOffset() throws Exception expect(azureStorage.getBlobExists(container, blobPath)).andReturn(true); expect(azureStorage.getBlobLength(container, blobPath)).andReturn((long) testLog.length()); expect(azureStorage.getBlobInputStream(container, blobPath)).andReturn( - new ByteArrayInputStream(testLog.getBytes(Charsets.UTF_8))); + new ByteArrayInputStream(testLog.getBytes(StandardCharsets.UTF_8))); replayAll(); diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java index 5f39e7e5a446..343635c4680e 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java @@ -101,7 +101,7 @@ private CloudFilesByteSource createCloudFilesByteSource(CloudFilesBlob object) @Override protected InputStream wrapObjectStream(CloudFilesBlob object, InputStream stream) throws IOException { - return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + return CompressionUtils.decompress(stream, object.getPath()); } @Override diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java index 0d5d99959609..38fb83870883 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java @@ -93,7 +93,7 @@ private GoogleByteSource createGoogleByteSource(GoogleBlob object) @Override protected InputStream wrapObjectStream(GoogleBlob object, InputStream stream) throws IOException { - return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + return CompressionUtils.decompress(stream, object.getPath()); } @Override diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentKiller.java b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentKiller.java index 5a476a1170b8..942f5931a30f 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentKiller.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentKiller.java @@ -19,6 +19,7 @@ package io.druid.storage.google; +import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.inject.Inject; import io.druid.java.util.common.MapUtils; import io.druid.java.util.common.logger.Logger; @@ -52,14 +53,27 @@ public void kill(DataSegment segment) throws SegmentLoadingException final String descriptorPath = indexPath.substring(0, indexPath.lastIndexOf("/")) + "/descriptor.json"; try { - storage.delete(bucket, indexPath); - storage.delete(bucket, descriptorPath); + deleteIfPresent(bucket, indexPath); + deleteIfPresent(bucket, descriptorPath); } catch (IOException e) { throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getIdentifier(), e.getMessage()); } } + private void deleteIfPresent(String bucket, String path) throws IOException + { + try { + storage.delete(bucket, path); + } + catch (GoogleJsonResponseException e) { + if (e.getStatusCode() != 404) { + throw e; + } + LOG.debug("Already deleted: [%s] [%s]", bucket, path); + } + } + @Override public void killAll() { diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleTaskLogs.java b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleTaskLogs.java index 6ed64576266f..d379f3445abb 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleTaskLogs.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleTaskLogs.java @@ -51,7 +51,19 @@ public void pushTaskLog(final String taskid, final File logFile) throws IOExcept { final String taskKey = getTaskLogKey(taskid); LOG.info("Pushing task log %s to: %s", logFile, taskKey); + pushTaskFile(taskid, logFile, taskKey); + } + + @Override + public void pushTaskReports(String taskid, File reportFile) throws IOException + { + final String taskKey = getTaskReportKey(taskid); + LOG.info("Pushing task reports %s to: %s", reportFile, taskKey); + pushTaskFile(taskid, reportFile, taskKey); + } + private void pushTaskFile(final String taskid, final File logFile, final String taskKey) throws IOException + { FileInputStream fileSteam = new FileInputStream(logFile); InputStreamContent mediaContent = new InputStreamContent("text/plain", fileSteam); @@ -64,7 +76,18 @@ public void pushTaskLog(final String taskid, final File logFile) throws IOExcept public Optional streamTaskLog(final String taskid, final long offset) throws IOException { final String taskKey = getTaskLogKey(taskid); + return streamTaskFile(taskid, offset, taskKey); + } + @Override + public Optional streamTaskReports(String taskid) throws IOException + { + final String taskKey = getTaskReportKey(taskid); + return streamTaskFile(taskid, 0, taskKey); + } + + private Optional streamTaskFile(final String taskid, final long offset, String taskKey) throws IOException + { try { if (!storage.exists(config.getBucket(), taskKey)) { return Optional.absent(); @@ -111,6 +134,11 @@ private String getTaskLogKey(String taskid) return config.getPrefix() + "/" + taskid.replaceAll(":", "_"); } + private String getTaskReportKey(String taskid) + { + return config.getPrefix() + "/" + taskid.replaceAll(":", "_") + ".report.json"; + } + @Override public void killAll() { diff --git a/extensions-contrib/influx-extensions/pom.xml b/extensions-contrib/influx-extensions/pom.xml new file mode 100644 index 000000000000..0673a7957adf --- /dev/null +++ b/extensions-contrib/influx-extensions/pom.xml @@ -0,0 +1,112 @@ + + + + + 4.0.0 + + io.druid.extensions + druid-influx-extensions + druid-influx-extensions + druid-influx-extensions + + + io.druid + druid + 0.13.0-SNAPSHOT + ../../pom.xml + + + + + + + + + + + io.druid + druid-common + ${project.parent.version} + provided + + + org.antlr + antlr4-runtime + + + + + junit + junit + test + + + org.mockito + mockito-core + 2.2.10 + test + + + org.hamcrest + hamcrest-all + 1.3 + test + + + pl.pragmatists + JUnitParams + 1.0.4 + test + + + io.druid + druid-processing + ${project.parent.version} + test + + + + + + + org.antlr + antlr4-maven-plugin + + + + antlr4 + + + + + + + + + + strict + + + + + + + diff --git a/extensions-contrib/influx-extensions/src/main/antlr4/io/druid/data/input/influx/InfluxLineProtocol.g4 b/extensions-contrib/influx-extensions/src/main/antlr4/io/druid/data/input/influx/InfluxLineProtocol.g4 new file mode 100644 index 000000000000..89b419f09a27 --- /dev/null +++ b/extensions-contrib/influx-extensions/src/main/antlr4/io/druid/data/input/influx/InfluxLineProtocol.g4 @@ -0,0 +1,77 @@ +/** Based on v1.4 from their docs + at https://docs.influxdata.com/influxdb/v1.4/write_protocols/line_protocol_tutorial/ + **/ + +grammar InfluxLineProtocol; + +lines + : line ('\n' line)* '\n'? EOF +; + +line + : identifier (',' tag_set)? ' ' field_set (' ' timestamp)? +; + +timestamp + : NUMBER +; + +field_set + : field_pair (',' field_pair)* +; + +tag_set + : tag_pair (',' tag_pair)* +; + +tag_pair + : identifier '=' identifier +; + +field_pair + : identifier '=' field_value +; + +identifier + : IDENTIFIER_STRING | NUMBER | BOOLEAN +; + +field_value + : QUOTED_STRING | NUMBER | BOOLEAN +; + +eol + : NEWLINE | EOF +; + +NEWLINE + : '\n' +; + +NUMBER + : '-'? INT ('.' [0-9] +) ? 'i'? +; + +BOOLEAN + : 'TRUE' | 'true' | 'True' | 't' | 'T' | 'FALSE' | 'False' | 'false' | 'F' | 'f' +; + +QUOTED_STRING + : '"' (StringFieldEscapeSequence | ~(["\\]) )* '"' +; + +IDENTIFIER_STRING + : (IdentifierEscapeSequence | ~([,= \n\\]) )+ +; + +fragment IdentifierEscapeSequence + : '\\' [,= \\] +; + +fragment StringFieldEscapeSequence + : '\\' ["\\] +; + +fragment INT + : '0' | [1-9] [0-9]* +; diff --git a/extensions-contrib/influx-extensions/src/main/java/io/druid/data/input/influx/InfluxExtensionsModule.java b/extensions-contrib/influx-extensions/src/main/java/io/druid/data/input/influx/InfluxExtensionsModule.java new file mode 100644 index 000000000000..964c05564e75 --- /dev/null +++ b/extensions-contrib/influx-extensions/src/main/java/io/druid/data/input/influx/InfluxExtensionsModule.java @@ -0,0 +1,51 @@ +/* + * 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.data.input.influx; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.inject.Binder; +import io.druid.initialization.DruidModule; + +import java.util.Collections; +import java.util.List; + +public class InfluxExtensionsModule implements DruidModule +{ + public InfluxExtensionsModule() + { + } + + @Override + public List getJacksonModules() + { + return Collections.singletonList( + new SimpleModule("InfluxInputRowParserModule") + .registerSubtypes( + new NamedType(InfluxParseSpec.class, "influx") + ) + ); + } + + @Override + public void configure(Binder binder) + { + } +} diff --git a/extensions-contrib/influx-extensions/src/main/java/io/druid/data/input/influx/InfluxParseSpec.java b/extensions-contrib/influx-extensions/src/main/java/io/druid/data/input/influx/InfluxParseSpec.java new file mode 100644 index 000000000000..eb1a6f64a432 --- /dev/null +++ b/extensions-contrib/influx-extensions/src/main/java/io/druid/data/input/influx/InfluxParseSpec.java @@ -0,0 +1,63 @@ +/* + * 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.data.input.influx; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Sets; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.ParseSpec; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.java.util.common.parsers.Parser; + +import java.util.List; + +public class InfluxParseSpec extends ParseSpec +{ + private List measurementWhitelist; + + @JsonCreator + public InfluxParseSpec( + @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec, + @JsonProperty("whitelistMeasurements") List measurementWhitelist + ) + { + super( + new TimestampSpec(InfluxParser.TIMESTAMP_KEY, "millis", null), + dimensionsSpec != null ? dimensionsSpec : new DimensionsSpec(null, null, null) + ); + this.measurementWhitelist = measurementWhitelist; + } + + @Override + public Parser makeParser() + { + if (measurementWhitelist != null && measurementWhitelist.size() > 0) { + return new InfluxParser(Sets.newHashSet(measurementWhitelist)); + } else { + return new InfluxParser(null); + } + } + + @Override + public ParseSpec withDimensionsSpec(DimensionsSpec spec) + { + return new InfluxParseSpec(spec, measurementWhitelist); + } +} diff --git a/extensions-contrib/influx-extensions/src/main/java/io/druid/data/input/influx/InfluxParser.java b/extensions-contrib/influx-extensions/src/main/java/io/druid/data/input/influx/InfluxParser.java new file mode 100644 index 000000000000..2c3e0ec40d5c --- /dev/null +++ b/extensions-contrib/influx-extensions/src/main/java/io/druid/data/input/influx/InfluxParser.java @@ -0,0 +1,173 @@ +/* + * 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.data.input.influx; + +import com.google.common.collect.ImmutableList; +import io.druid.java.util.common.parsers.ParseException; +import io.druid.java.util.common.parsers.Parser; +import org.antlr.v4.runtime.ANTLRInputStream; +import org.antlr.v4.runtime.CharStream; +import org.antlr.v4.runtime.CommonTokenStream; +import org.antlr.v4.runtime.TokenStream; + +import javax.annotation.Nullable; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class InfluxParser implements Parser +{ + public static final String TIMESTAMP_KEY = "__ts"; + private static final String MEASUREMENT_KEY = "measurement"; + private final Set measurementWhitelist; + + public InfluxParser(Set measurementWhitelist) + { + this.measurementWhitelist = measurementWhitelist; + } + + @Override + public void startFileFromBeginning() + { + } + + @Nullable + @Override + public Map parseToMap(String input) + { + CharStream charStream = new ANTLRInputStream(input); + InfluxLineProtocolLexer lexer = new InfluxLineProtocolLexer(charStream); + TokenStream tokenStream = new CommonTokenStream(lexer); + InfluxLineProtocolParser parser = new InfluxLineProtocolParser(tokenStream); + + List lines = parser.lines().line(); + if (parser.getNumberOfSyntaxErrors() != 0) { + throw new ParseException("Unable to parse line."); + } + if (lines.size() != 1) { + throw new ParseException("Multiple lines present; unable to parse more than one per record."); + } + + Map out = new LinkedHashMap<>(); + + InfluxLineProtocolParser.LineContext line = lines.get(0); + String measurement = parseIdentifier(line.identifier()); + + if (!checkWhitelist(measurement)) { + throw new ParseException("Metric not whitelisted."); + } + + out.put(MEASUREMENT_KEY, measurement); + if (line.tag_set() != null) { + line.tag_set().tag_pair().forEach(t -> parseTag(t, out)); + } + + line.field_set().field_pair().forEach(t -> parseField(t, out)); + + if (line.timestamp() != null) { + String timestamp = line.timestamp().getText(); + parseTimestamp(timestamp, out); + } + return out; + } + + private void parseTag(InfluxLineProtocolParser.Tag_pairContext tag, Map out) + { + String key = parseIdentifier(tag.identifier(0)); + String value = parseIdentifier(tag.identifier(1)); + out.put(key, value); + } + + private void parseField(InfluxLineProtocolParser.Field_pairContext field, Map out) + { + String key = parseIdentifier(field.identifier()); + InfluxLineProtocolParser.Field_valueContext valueContext = field.field_value(); + Object value; + if (valueContext.NUMBER() != null) { + value = parseNumber(valueContext.NUMBER().getText()); + } else if (valueContext.BOOLEAN() != null) { + value = parseBool(valueContext.BOOLEAN().getText()); + } else { + value = parseQuotedString(valueContext.QUOTED_STRING().getText()); + } + out.put(key, value); + } + + private Object parseQuotedString(String text) + { + return text.substring(1, text.length() - 1).replaceAll("\\\\\"", "\""); + } + + private Object parseNumber(String raw) + { + if (raw.endsWith("i")) { + return new Long(raw.substring(0, raw.length() - 1)); + } + + return new Double(raw); + } + + private Object parseBool(String raw) + { + char first = raw.charAt(0); + if (first == 't' || first == 'T') { + return "true"; + } else { + return "false"; + } + } + + private String parseIdentifier(InfluxLineProtocolParser.IdentifierContext ctx) + { + if (ctx.BOOLEAN() != null || ctx.NUMBER() != null) { + return ctx.getText(); + } + + return ctx.IDENTIFIER_STRING().getText().replaceAll("\\\\([,= ])", "$1"); + } + + private boolean checkWhitelist(String m) + { + return (measurementWhitelist == null) || measurementWhitelist.contains(m); + } + + private void parseTimestamp(String timestamp, Map dest) + { + // Influx timestamps come in nanoseconds; treat anything less than 1 ms as 0 + if (timestamp.length() < 7) { + dest.put(TIMESTAMP_KEY, 0L); + } else { + timestamp = timestamp.substring(0, timestamp.length() - 6); + long timestampMillis = new Long(timestamp); + dest.put(TIMESTAMP_KEY, timestampMillis); + } + } + + @Override + public List getFieldNames() + { + return ImmutableList.of(); + } + + @Override + public void setFieldNames(Iterable fieldNames) + { + } +} diff --git a/extensions-contrib/influx-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-contrib/influx-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100644 index 000000000000..3ff740f01a24 --- /dev/null +++ b/extensions-contrib/influx-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1 @@ +io.druid.data.input.influx.InfluxExtensionsModule diff --git a/extensions-contrib/influx-extensions/src/test/java/io/druid/data/input/influx/InfluxParserTest.java b/extensions-contrib/influx-extensions/src/test/java/io/druid/data/input/influx/InfluxParserTest.java new file mode 100644 index 000000000000..0977bf5c3622 --- /dev/null +++ b/extensions-contrib/influx-extensions/src/test/java/io/druid/data/input/influx/InfluxParserTest.java @@ -0,0 +1,225 @@ +/* + * 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.data.input.influx; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import io.druid.java.util.common.Pair; +import io.druid.java.util.common.parsers.ParseException; +import io.druid.java.util.common.parsers.Parser; +import junitparams.JUnitParamsRunner; +import junitparams.Parameters; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.isA; + +@RunWith(JUnitParamsRunner.class) +public class InfluxParserTest +{ + private String name; + private String input; + private Map expected; + + private static Object[] testCase(String name, String input, Parsed expected) + { + return Lists.newArrayList(name, input, expected).toArray(); + } + + + public Object[] testData() + { + return Lists.newArrayList( + testCase( + "real sample", + "cpu,host=foo.bar.baz,region=us-east-1,application=echo pct_idle=99.3,pct_user=88.8,m1_load=2i 1465839830100400200", + Parsed.row("cpu", 1465839830100L) + .with("host", "foo.bar.baz") + .with("region", "us-east-1") + .with("application", "echo") + .with("pct_idle", 99.3) + .with("pct_user", 88.8) + .with("m1_load", 2L) + ), + testCase( + "negative timestamp", + "foo,region=us-east-1,host=127.0.0.1 m=1.0,n=3.0,o=500i -123456789", + Parsed.row("foo", -123L) + .with("region", "us-east-1") + .with("host", "127.0.0.1") + .with("m", 1.0) + .with("n", 3.0) + .with("o", 500L) + ), + testCase( + "truncated timestamp", + "foo,region=us-east-1,host=127.0.0.1 m=1.0,n=3.0,o=500i 123", + Parsed.row("foo", 0L) + .with("region", "us-east-1") + .with("host", "127.0.0.1") + .with("m", 1.0) + .with("n", 3.0) + .with("o", 500L) + ), + testCase( + "special characters", + "!@#$%^&*()_-\\=+,+++\\ +++=--\\ --- __**__=\"ü\" 123456789", + Parsed.row("!@#$%^&*()_-=+", 123L) + .with("+++ +++", "-- ---") + .with("__**__", "127.0.0.1") + .with("__**__", "ü") + ), + testCase( + "unicode characters", + "\uD83D\uDE00,\uD83D\uDE05=\uD83D\uDE06 \uD83D\uDE0B=100i,b=\"\uD83D\uDE42\" 123456789", + Parsed.row("\uD83D\uDE00", 123L) + .with("\uD83D\uDE05", "\uD83D\uDE06") + .with("\uD83D\uDE0B", 100L) + .with("b", "\uD83D\uDE42") + ), + testCase( + "quoted string measurement value", + "foo,region=us-east-1,host=127.0.0.1 m=1.0,n=3.0,o=\"something \\\"cool\\\" \" 123456789", + Parsed.row("foo", 123L) + .with("region", "us-east-1") + .with("host", "127.0.0.1") + .with("m", 1.0) + .with("n", 3.0) + .with("o", "something \"cool\" ") + ), + testCase( + "no tags", + "foo m=1.0,n=3.0 123456789", + Parsed.row("foo", 123L) + .with("m", 1.0) + .with("n", 3.0) + ), + testCase( + "Escaped characters in identifiers", + "f\\,oo\\ \\=,bar=baz m=1.0,n=3.0 123456789", + Parsed.row("f,oo =", 123L) + .with("bar", "baz") + .with("m", 1.0) + .with("n", 3.0) + ), + testCase( + "Escaped characters in identifiers", + "foo\\ \\=,bar=baz m=1.0,n=3.0 123456789", + Parsed.row("foo =", 123L) + .with("bar", "baz") + .with("m", 1.0) + .with("n", 3.0) + ) + ).toArray(); + } + + @Test + @Parameters(method = "testData") + public void testParse(String name, String input, Parsed expected) + { + Parser parser = new InfluxParser(null); + Map parsed = parser.parseToMap(input); + assertThat("correct measurement name", parsed.get("measurement"), equalTo(expected.measurement)); + assertThat("correct timestamp", parsed.get(InfluxParser.TIMESTAMP_KEY), equalTo(expected.timestamp)); + expected.kv.forEach((k, v) -> { + assertThat("correct field " + k, parsed.get(k), equalTo(v)); + }); + parsed.remove("measurement"); + parsed.remove(InfluxParser.TIMESTAMP_KEY); + assertThat("No extra keys in parsed data", parsed.keySet(), equalTo(expected.kv.keySet())); + } + + @Test + public void testParseWhitelistPass() + { + Parser parser = new InfluxParser(Sets.newHashSet("cpu")); + String input = "cpu,host=foo.bar.baz,region=us-east,application=echo pct_idle=99.3,pct_user=88.8,m1_load=2 1465839830100400200"; + Map parsed = parser.parseToMap(input); + assertThat(parsed.get("measurement"), equalTo("cpu")); + } + + @Test + public void testParseWhitelistFail() + { + Parser parser = new InfluxParser(Sets.newHashSet("mem")); + String input = "cpu,host=foo.bar.baz,region=us-east,application=echo pct_idle=99.3,pct_user=88.8,m1_load=2 1465839830100400200"; + try { + parser.parseToMap(input); + } + catch (ParseException t) { + assertThat(t, isA(ParseException.class)); + return; + } + + Assert.fail("Exception not thrown"); + } + + public Object[] failureTestData() + { + return Lists.newArrayList( + Pair.of("Empty line", ""), + Pair.of("Invalid measurement", "invalid measurement"), + Pair.of("Invalid timestamp", "foo i=123 123x") + ).toArray(); + } + + @Test + @Parameters(method = "failureTestData") + public void testParseFailures(Pair testCase) + { + Parser parser = new InfluxParser(null); + try { + Map res = parser.parseToMap(testCase.rhs); + } + catch (ParseException t) { + assertThat(t, isA(ParseException.class)); + return; + } + + Assert.fail(testCase.rhs + ": exception not thrown"); + } + + private static class Parsed + { + private String measurement; + private Long timestamp; + private Map kv = new HashMap<>(); + + public static Parsed row(String measurement, Long timestamp) + { + Parsed e = new Parsed(); + e.measurement = measurement; + e.timestamp = timestamp; + return e; + } + + public Parsed with(String k, Object v) + { + kv.put(k, v); + return this; + } + } +} 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-contrib/time-min-max/src/test/java/io/druid/query/aggregation/TimestampAggregationSelectTest.java b/extensions-contrib/time-min-max/src/test/java/io/druid/query/aggregation/TimestampAggregationSelectTest.java index 58fbdc700622..e862d3fcec08 100644 --- a/extensions-contrib/time-min-max/src/test/java/io/druid/query/aggregation/TimestampAggregationSelectTest.java +++ b/extensions-contrib/time-min-max/src/test/java/io/druid/query/aggregation/TimestampAggregationSelectTest.java @@ -19,7 +19,6 @@ package io.druid.query.aggregation; -import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; @@ -41,6 +40,7 @@ import javax.annotation.Nullable; import java.io.File; +import java.nio.charset.StandardCharsets; import java.sql.Timestamp; import java.util.List; import java.util.zip.ZipFile; @@ -147,7 +147,7 @@ public void testSimpleDataIngestionAndSelectTest() throws Exception 0, Granularities.MONTH, 100, - Resources.toString(Resources.getResource("select.json"), Charsets.UTF_8) + Resources.toString(Resources.getResource("select.json"), StandardCharsets.UTF_8) ); Result result = (Result) Iterables.getOnlyElement(seq.toList()); diff --git a/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/DruidVirtualColumnsModule.java b/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/DruidVirtualColumnsModule.java index 184225d02453..9db5be737a22 100644 --- a/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/DruidVirtualColumnsModule.java +++ b/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/DruidVirtualColumnsModule.java @@ -20,6 +20,7 @@ package io.druid.segment; import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; @@ -34,7 +35,12 @@ public class DruidVirtualColumnsModule implements DruidModule @Override public List getJacksonModules() { - return ImmutableList.of(new SimpleModule().registerSubtypes(MapVirtualColumn.class)); + return ImmutableList.of( + new SimpleModule(getClass().getSimpleName()) + .registerSubtypes( + new NamedType(MapVirtualColumn.class, "map") + ) + ); } @Override diff --git a/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java b/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java index 63155edd2ea8..5f25336f2ab4 100644 --- a/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java @@ -19,6 +19,7 @@ package io.druid.segment; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableMap; @@ -140,6 +141,18 @@ private Druids.SelectQueryBuilder testBuilder() .pagingSpec(new PagingSpec(null, 3)); } + @Test + public void testSerde() throws IOException + { + final ObjectMapper mapper = new DefaultObjectMapper(); + new DruidVirtualColumnsModule().getJacksonModules().forEach(mapper::registerModule); + + final MapVirtualColumn column = new MapVirtualColumn("keys", "values", "params"); + final String json = mapper.writeValueAsString(column); + final VirtualColumn fromJson = mapper.readValue(json, VirtualColumn.class); + Assert.assertEquals(column, fromJson); + } + @Test public void testBasic() { diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java index bfbaaac86f7d..199ec271a9a3 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java @@ -182,7 +182,7 @@ public int getMaxIntermediateSize() @Override public List getRequiredColumns() { - return Collections. singletonList( + return Collections.singletonList( new DoublesSketchAggregatorFactory( fieldName, fieldName, diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java index 0e3081965b03..8080eb62f323 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java @@ -19,22 +19,22 @@ package io.druid.query.aggregation.datasketches.quantiles; -import java.util.Arrays; -import java.util.List; - import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; import com.yahoo.sketches.quantiles.DoublesSketch; - import io.druid.initialization.DruidModule; import io.druid.segment.serde.ComplexMetrics; +import java.util.Collections; +import java.util.List; + public class DoublesSketchModule implements DruidModule { public static final String DOUBLES_SKETCH = "quantilesDoublesSketch"; + public static final String DOUBLES_SKETCH_MERGE = "quantilesDoublesSketchMerge"; public static final String DOUBLES_SKETCH_HISTOGRAM_POST_AGG = "quantilesDoublesSketchToHistogram"; public static final String DOUBLES_SKETCH_QUANTILE_POST_AGG = "quantilesDoublesSketchToQuantile"; @@ -52,14 +52,17 @@ public void configure(final Binder binder) @Override public List getJacksonModules() { - return Arrays. asList( - new SimpleModule("DoublesQuantilesSketchModule").registerSubtypes( - new NamedType(DoublesSketchAggregatorFactory.class, DOUBLES_SKETCH), - new NamedType(DoublesSketchToHistogramPostAggregator.class, DOUBLES_SKETCH_HISTOGRAM_POST_AGG), - new NamedType(DoublesSketchToQuantilePostAggregator.class, DOUBLES_SKETCH_QUANTILE_POST_AGG), - new NamedType(DoublesSketchToQuantilesPostAggregator.class, DOUBLES_SKETCH_QUANTILES_POST_AGG), - new NamedType(DoublesSketchToStringPostAggregator.class, DOUBLES_SKETCH_TO_STRING_POST_AGG)) - .addSerializer(DoublesSketch.class, new DoublesSketchJsonSerializer())); + return Collections.singletonList( + new SimpleModule("DoublesQuantilesSketchModule") + .registerSubtypes( + new NamedType(DoublesSketchAggregatorFactory.class, DOUBLES_SKETCH), + new NamedType(DoublesSketchMergeAggregatorFactory.class, DOUBLES_SKETCH_MERGE), + new NamedType(DoublesSketchToHistogramPostAggregator.class, DOUBLES_SKETCH_HISTOGRAM_POST_AGG), + new NamedType(DoublesSketchToQuantilePostAggregator.class, DOUBLES_SKETCH_QUANTILE_POST_AGG), + new NamedType(DoublesSketchToQuantilesPostAggregator.class, DOUBLES_SKETCH_QUANTILES_POST_AGG), + new NamedType(DoublesSketchToStringPostAggregator.class, DOUBLES_SKETCH_TO_STRING_POST_AGG) + ).addSerializer(DoublesSketch.class, new DoublesSketchJsonSerializer()) + ); } } diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperations.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperations.java index 5a0f70e6b76a..48e3f3c11872 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperations.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperations.java @@ -19,13 +19,12 @@ package io.druid.query.aggregation.datasketches.quantiles; -import org.apache.commons.codec.binary.Base64; - -import com.google.common.base.Charsets; import com.yahoo.memory.Memory; import com.yahoo.sketches.quantiles.DoublesSketch; - import io.druid.java.util.common.ISE; +import org.apache.commons.codec.binary.Base64; + +import java.nio.charset.StandardCharsets; public class DoublesSketchOperations { @@ -48,7 +47,7 @@ public static DoublesSketch deserialize(final Object serializedSketch) public static DoublesSketch deserializeFromBase64EncodedString(final String str) { - return deserializeFromByteArray(Base64.decodeBase64(str.getBytes(Charsets.UTF_8))); + return deserializeFromByteArray(Base64.decodeBase64(str.getBytes(StandardCharsets.UTF_8))); } public static DoublesSketch deserializeFromByteArray(final byte[] data) diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java new file mode 100644 index 000000000000..510cb78b80bd --- /dev/null +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java @@ -0,0 +1,133 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation.datasketches.theta; + +import java.util.Collections; +import java.util.Comparator; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.codec.digest.DigestUtils; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.post.PostAggregatorIds; +import io.druid.query.cache.CacheKeyBuilder; + +/** + */ +public class SketchConstantPostAggregator implements PostAggregator +{ + + private final String name; + private final String value; + private final SketchHolder sketchValue; + + @JsonCreator + public SketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value) + { + this.name = name; + Preconditions.checkArgument(value != null && !value.isEmpty(), + "Constant value cannot be null or empty, expecting base64 encoded sketch string"); + this.value = value; + this.sketchValue = SketchHolder.deserialize(value); + } + + @Override + public Set getDependentFields() + { + return Collections.emptySet(); + } + + @Override + public Comparator getComparator() + { + return SketchHolder.COMPARATOR; + } + + @Override + public Object compute(Map combinedAggregators) + { + return sketchValue; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + public SketchConstantPostAggregator decorate(Map aggregators) + { + return this; + } + + @JsonProperty("value") + public SketchHolder getSketchValue() + { + return sketchValue; + } + + @Override + public String toString() + { + return "SketchConstantPostAggregator{name='" + name + "', value='" + value + "'}"; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SketchConstantPostAggregator that = (SketchConstantPostAggregator) o; + if (!this.sketchValue.equals(that.sketchValue)) { + return false; + } + if (name != null ? !name.equals(that.name) : that.name != null) { + return false; + } + return true; + } + + @Override + public int hashCode() + { + int result = name != null ? name.hashCode() : 0; + result = 37 * result + sketchValue.hashCode(); + return result; + } + + @Override + public byte[] getCacheKey() + { + return new CacheKeyBuilder(PostAggregatorIds.THETA_SKETCH_CONSTANT) + .appendString(DigestUtils.sha1Hex(value)).build(); + } +} diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchHolder.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchHolder.java index cf5f1a3fb130..c273bd4a76aa 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchHolder.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchHolder.java @@ -36,6 +36,7 @@ import io.druid.java.util.common.StringUtils; import org.apache.commons.codec.binary.Base64; +import java.util.Arrays; import java.util.Comparator; /** @@ -286,6 +287,11 @@ public static SketchHolder sketchSetOperation(Func func, int sketchSize, Object. } } + /** + * Ideally make use of Sketch's equals and hashCode methods but which are not value based implementations. + * And yet need value based equals and hashCode implementations for SketchHolder. + * Hence using Arrays.equals() and Arrays.hashCode(). + */ @Override public boolean equals(Object o) { @@ -295,6 +301,12 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - return this.getSketch().equals(((SketchHolder) o).getSketch()); + return Arrays.equals(this.getSketch().toByteArray(), ((SketchHolder) o).getSketch().toByteArray()); + } + + @Override + public int hashCode() + { + return 31 * Arrays.hashCode(this.getSketch().toByteArray()); } } diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchModule.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchModule.java index 361dd4a2fbf9..18d699ec6c52 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchModule.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchModule.java @@ -38,6 +38,8 @@ public class SketchModule implements DruidModule public static final String THETA_SKETCH_ESTIMATE_POST_AGG = "thetaSketchEstimate"; public static final String THETA_SKETCH_SET_OP_POST_AGG = "thetaSketchSetOp"; + + public static final String THETA_SKETCH_CONSTANT_POST_AGG = "thetaSketchConstant"; @Override public void configure(Binder binder) @@ -63,7 +65,8 @@ public List getJacksonModules() .registerSubtypes( new NamedType(SketchMergeAggregatorFactory.class, THETA_SKETCH), new NamedType(SketchEstimatePostAggregator.class, THETA_SKETCH_ESTIMATE_POST_AGG), - new NamedType(SketchSetPostAggregator.class, THETA_SKETCH_SET_OP_POST_AGG) + new NamedType(SketchSetPostAggregator.class, THETA_SKETCH_SET_OP_POST_AGG), + new NamedType(SketchConstantPostAggregator.class, THETA_SKETCH_CONSTANT_POST_AGG) ) .addSerializer( SketchHolder.class, new SketchHolderJsonSerializer() diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java index 5da8e787ac1d..0c8f5f2e257b 100644 --- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java @@ -27,6 +27,7 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequence; import io.druid.query.aggregation.AggregationTestHelper; +import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.groupby.GroupByQueryConfig; import io.druid.query.groupby.GroupByQueryRunnerTest; import org.junit.Assert; @@ -58,7 +59,8 @@ public DoublesSketchAggregatorTest(final GroupByQueryConfig config) module.getJacksonModules(), config, tempFolder); timeSeriesHelper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper( module.getJacksonModules(), - tempFolder); + tempFolder + ); } @Parameterized.Parameters(name = "{0}") @@ -66,7 +68,7 @@ public static Collection constructorFeeder() { final List constructors = Lists.newArrayList(); for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { - constructors.add(new Object[] {config}); + constructors.add(new Object[]{config}); } return constructors; } @@ -76,11 +78,29 @@ public static Collection constructorFeeder() public void serializeDeserializeFactoryWithFieldName() throws Exception { ObjectMapper objectMapper = new DefaultObjectMapper(); + new DoublesSketchModule().getJacksonModules().forEach(objectMapper::registerModule); DoublesSketchAggregatorFactory factory = new DoublesSketchAggregatorFactory("name", "filedName", 128); - DoublesSketchAggregatorFactory other = objectMapper.readValue( + AggregatorFactory other = objectMapper.readValue( objectMapper.writeValueAsString(factory), - DoublesSketchAggregatorFactory.class); + AggregatorFactory.class + ); + + Assert.assertEquals(factory, other); + } + + // this is to test Json properties and equals for the combining factory + @Test + public void serializeDeserializeCombiningFactoryWithFieldName() throws Exception + { + ObjectMapper objectMapper = new DefaultObjectMapper(); + new DoublesSketchModule().getJacksonModules().forEach(objectMapper::registerModule); + DoublesSketchAggregatorFactory factory = new DoublesSketchMergeAggregatorFactory("name", 128); + + AggregatorFactory other = objectMapper.readValue( + objectMapper.writeValueAsString(factory), + AggregatorFactory.class + ); Assert.assertEquals(factory, other); } @@ -90,7 +110,8 @@ public void ingestingSketches() throws Exception { Sequence seq = helper.createIndexAndRunQueryOnSegment( new File(this.getClass().getClassLoader().getResource("quantiles/doubles_sketch_data.tsv").getFile()), - String.join("\n", + String.join( + "\n", "{", " \"type\": \"string\",", " \"parseSpec\": {", @@ -103,16 +124,20 @@ public void ingestingSketches() throws Exception " },", " \"columns\": [\"timestamp\", \"product\", \"sketch\"]", " }", - "}"), - String.join("\n", + "}" + ), + String.join( + "\n", "[", " {\"type\": \"quantilesDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"k\": 128},", " {\"type\": \"quantilesDoublesSketch\", \"name\": \"non_existent_sketch\", \"fieldName\": \"non_existent_sketch\", \"k\": 128}", - "]"), + "]" + ), 0, // minTimestamp Granularities.NONE, 10, // maxRowCount - String.join("\n", + String.join( + "\n", "{", " \"queryType\": \"groupBy\",", " \"dataSource\": \"test_datasource\",", @@ -127,7 +152,9 @@ public void ingestingSketches() throws Exception " {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", " ],", " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", - "}")); + "}" + ) + ); List results = seq.toList(); Assert.assertEquals(1, results.size()); Row row = results.get(0); @@ -155,8 +182,8 @@ public void ingestingSketches() throws Exception Assert.assertTrue(histogramObject instanceof double[]); double[] histogram = (double[]) histogramObject; for (final double bin : histogram) { - Assert.assertEquals(100, bin, 100 * 0.2); // 400 items uniformly - // distributed into 4 bins + // 400 items uniformly distributed into 4 bins + Assert.assertEquals(100, bin, 100 * 0.2); } } @@ -165,7 +192,8 @@ public void buildingSketchesAtIngestionTime() throws Exception { Sequence seq = helper.createIndexAndRunQueryOnSegment( new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()), - String.join("\n", + String.join( + "\n", "{", " \"type\": \"string\",", " \"parseSpec\": {", @@ -178,12 +206,14 @@ public void buildingSketchesAtIngestionTime() throws Exception " },", " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", " }", - "}"), + "}" + ), "[{\"type\": \"quantilesDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 128}]", 0, // minTimestamp Granularities.NONE, 10, // maxRowCount - String.join("\n", + String.join( + "\n", "{", " \"queryType\": \"groupBy\",", " \"dataSource\": \"test_datasource\",", @@ -198,7 +228,9 @@ public void buildingSketchesAtIngestionTime() throws Exception " {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", " ],", " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", - "}")); + "}" + ) + ); List results = seq.toList(); Assert.assertEquals(1, results.size()); Row row = results.get(0); @@ -223,7 +255,7 @@ public void buildingSketchesAtIngestionTime() throws Exception Assert.assertEquals(4, histogram.length); for (final double bin : histogram) { Assert.assertEquals(100, bin, 100 * 0.2); // 400 items uniformly - // distributed into 4 bins + // distributed into 4 bins } } @@ -232,7 +264,8 @@ public void buildingSketchesAtQueryTime() throws Exception { Sequence seq = helper.createIndexAndRunQueryOnSegment( new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()), - String.join("\n", + String.join( + "\n", "{", " \"type\": \"string\",", " \"parseSpec\": {", @@ -245,12 +278,14 @@ public void buildingSketchesAtQueryTime() throws Exception " },", " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", " }", - "}"), + "}" + ), "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]", 0, // minTimestamp Granularities.NONE, 10, // maxRowCount - String.join("\n", + String.join( + "\n", "{", " \"queryType\": \"groupBy\",", " \"dataSource\": \"test_datasource\",", @@ -265,7 +300,9 @@ public void buildingSketchesAtQueryTime() throws Exception " {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", " ],", " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", - "}")); + "}" + ) + ); List results = seq.toList(); Assert.assertEquals(1, results.size()); Row row = results.get(0); @@ -294,7 +331,7 @@ public void buildingSketchesAtQueryTime() throws Exception double[] histogram = (double[]) histogramObject; for (final double bin : histogram) { Assert.assertEquals(100, bin, 100 * 0.2); // 400 items uniformly - // distributed into 4 bins + // distributed into 4 bins } } @@ -317,7 +354,8 @@ public void QueryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Except " },", " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", " }", - "}"), + "}" + ), "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]", 0, // minTimestamp Granularities.NONE, @@ -338,7 +376,9 @@ public void QueryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Except " {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", " ],", " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", - "}")); + "}" + ) + ); List results = seq.toList(); Assert.assertEquals(1, results.size()); Row row = results.get(0); @@ -367,7 +407,7 @@ public void QueryingDataWithFieldNameValueAsFloatInsteadOfSketch() throws Except double[] histogram = (double[]) histogramObject; for (final double bin : histogram) { Assert.assertEquals(100, bin, 100 * 0.2); // 400 items uniformly - // distributed into 4 bins + // distributed into 4 bins } } @@ -390,7 +430,8 @@ public void TimeSeriesQueryInputAsFloat() throws Exception " },", " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", " }", - "}"), + "}" + ), "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]", 0, // minTimestamp Granularities.NONE, @@ -410,7 +451,9 @@ public void TimeSeriesQueryInputAsFloat() throws Exception " {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram1\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", " ],", " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", - "}")); + "}" + ) + ); List results = seq.toList(); Assert.assertEquals(1, results.size()); } diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java index 260dc12b3507..b5eb3cb0f6c7 100644 --- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java +++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java @@ -277,6 +277,14 @@ public void testSketchEstimatePostAggregatorSerde() throws Exception 2 ) ); + + assertPostAggregatorSerde( + new SketchEstimatePostAggregator( + "name", + new SketchConstantPostAggregator("name", "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI="), + null + ) + ); } @Test @@ -293,6 +301,18 @@ public void testSketchSetPostAggregatorSerde() throws Exception ) ) ); + + assertPostAggregatorSerde( + new SketchSetPostAggregator( + "name", + "INTERSECT", + null, + Lists.newArrayList( + new FieldAccessPostAggregator("name1", "fieldName1"), + new SketchConstantPostAggregator("name2", "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=") + ) + ) + ); } @Test diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java index 870fa9794b0c..d38166b4b915 100644 --- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java +++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java @@ -276,6 +276,51 @@ public void testSimpleDataIngestAndSelectQuery() throws Exception Assert.assertEquals(100, result.getValue().getEvents().size()); Assert.assertEquals("AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=", result.getValue().getEvents().get(0).getEvent().get("pty_country")); } + + @Test + public void testTopNQueryWithSketchConstant() throws Exception + { + AggregationTestHelper topNQueryAggregationTestHelper = AggregationTestHelper.createTopNQueryAggregationTestHelper( + sm.getJacksonModules(), + tempFolder + ); + + Sequence seq = topNQueryAggregationTestHelper.runQueryOnSegments( + ImmutableList.of(s1, s2), + readFileFromClasspathAsString("topn_query_sketch_const.json") + ); + + Result result = (Result) Iterables.getOnlyElement(seq.toList()); + + Assert.assertEquals(DateTimes.of("2014-10-20T00:00:00.000Z"), result.getTimestamp()); + + DimensionAndMetricValueExtractor value1 = Iterables.get(result.getValue().getValue(), 0); + Assert.assertEquals(38.0, value1.getDoubleMetric("sketch_count"), 0.01); + Assert.assertEquals(38.0, value1.getDoubleMetric("sketchEstimatePostAgg"), 0.01); + Assert.assertEquals(2.0, value1.getDoubleMetric("sketchEstimatePostAggForSketchConstant"), 0.01); + Assert.assertEquals(39.0, value1.getDoubleMetric("sketchUnionPostAggEstimate"), 0.01); + Assert.assertEquals(1.0, value1.getDoubleMetric("sketchIntersectionPostAggEstimate"), 0.01); + Assert.assertEquals(37.0, value1.getDoubleMetric("sketchAnotBPostAggEstimate"), 0.01); + Assert.assertEquals("product_3", value1.getDimensionValue("product")); + + DimensionAndMetricValueExtractor value2 = Iterables.get(result.getValue().getValue(), 1); + Assert.assertEquals(42.0, value2.getDoubleMetric("sketch_count"), 0.01); + Assert.assertEquals(42.0, value2.getDoubleMetric("sketchEstimatePostAgg"), 0.01); + Assert.assertEquals(2.0, value2.getDoubleMetric("sketchEstimatePostAggForSketchConstant"), 0.01); + Assert.assertEquals(42.0, value2.getDoubleMetric("sketchUnionPostAggEstimate"), 0.01); + Assert.assertEquals(2.0, value2.getDoubleMetric("sketchIntersectionPostAggEstimate"), 0.01); + Assert.assertEquals(40.0, value2.getDoubleMetric("sketchAnotBPostAggEstimate"), 0.01); + Assert.assertEquals("product_1", value2.getDimensionValue("product")); + + DimensionAndMetricValueExtractor value3 = Iterables.get(result.getValue().getValue(), 2); + Assert.assertEquals(42.0, value3.getDoubleMetric("sketch_count"), 0.01); + Assert.assertEquals(42.0, value3.getDoubleMetric("sketchEstimatePostAgg"), 0.01); + Assert.assertEquals(2.0, value3.getDoubleMetric("sketchEstimatePostAggForSketchConstant"), 0.01); + Assert.assertEquals(42.0, value3.getDoubleMetric("sketchUnionPostAggEstimate"), 0.01); + Assert.assertEquals(2.0, value3.getDoubleMetric("sketchIntersectionPostAggEstimate"), 0.01); + Assert.assertEquals(40.0, value3.getDoubleMetric("sketchAnotBPostAggEstimate"), 0.01); + Assert.assertEquals("product_2", value3.getDimensionValue("product")); + } public static final String readFileFromClasspathAsString(String fileName) throws IOException { diff --git a/extensions-core/datasketches/src/test/resources/topn_query_sketch_const.json b/extensions-core/datasketches/src/test/resources/topn_query_sketch_const.json new file mode 100644 index 000000000000..3dc47dca86c3 --- /dev/null +++ b/extensions-core/datasketches/src/test/resources/topn_query_sketch_const.json @@ -0,0 +1,104 @@ +{ + "queryType": "topN", + "dataSource": "test_datasource", + "granularity":"ALL", + "metric": { + "type": "inverted", + "metric": "sketch_count" + }, + "dimension": "product", + "threshold": 3, + "aggregations": [ + { + "type": "thetaSketch", + "name": "sketch_count", + "fieldName": "pty_country", + "size": 16384 + } + ], + "postAggregations": [ + { + "type": "thetaSketchEstimate", + "name": "sketchEstimatePostAgg", + "field": { + "type": "fieldAccess", + "fieldName": "sketch_count" + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchEstimatePostAggForSketchConstant", + "field": { + "type": "thetaSketchConstant", + "name": "theta_sketch_count", + "value": "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=" + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchIntersectionPostAggEstimate", + "field": { + "type": "thetaSketchSetOp", + "name": "sketchIntersectionPostAgg", + "func": "INTERSECT", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sketch_count" + }, + { + "type": "thetaSketchConstant", + "name": "theta_sketch_count", + "value": "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=" + } + ] + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchAnotBPostAggEstimate", + "field": { + "type": "thetaSketchSetOp", + "name": "sketchAnotBUnionPostAgg", + "func": "NOT", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sketch_count" + }, + { + "type": "thetaSketchConstant", + "name": "theta_sketch_count", + "value": "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=" + } + ] + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchUnionPostAggEstimate", + "field": { + "type": "thetaSketchSetOp", + "name": "sketchUnionPostAgg", + "func": "UNION", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sketch_count" + }, + { + "type": "thetaSketchConstant", + "name": "theta_sketch_count", + "value": "AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=" + } + ] + } + } + ], + "intervals": [ + "2014-10-19T00:00:00.000Z/2014-10-22T00:00:00.000Z" + ] +} diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java index fbdfb066c8e4..4d601b01c561 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java @@ -99,8 +99,6 @@ public class KerberosAuthenticator implements Authenticator private final DruidNode node; private final String serverPrincipal; private final String serverKeytab; - private final String internalClientPrincipal; - private final String internalClientKeytab; private final String authToLocal; private final List excludedPaths; private final String cookieSignatureSecret; @@ -111,8 +109,6 @@ public class KerberosAuthenticator implements Authenticator public KerberosAuthenticator( @JsonProperty("serverPrincipal") String serverPrincipal, @JsonProperty("serverKeytab") String serverKeytab, - @JsonProperty("internalClientPrincipal") String internalClientPrincipal, - @JsonProperty("internalClientKeytab") String internalClientKeytab, @JsonProperty("authToLocal") String authToLocal, @JsonProperty("excludedPaths") List excludedPaths, @JsonProperty("cookieSignatureSecret") String cookieSignatureSecret, @@ -123,8 +119,6 @@ public KerberosAuthenticator( this.node = node; this.serverPrincipal = serverPrincipal; this.serverKeytab = serverKeytab; - this.internalClientPrincipal = internalClientPrincipal; - this.internalClientKeytab = internalClientKeytab; this.authToLocal = authToLocal == null ? "DEFAULT" : authToLocal; this.excludedPaths = excludedPaths == null ? DEFAULT_EXCLUDED_PATHS : excludedPaths; this.cookieSignatureSecret = cookieSignatureSecret; @@ -334,15 +328,21 @@ public Principal getUserPrincipal() }; if (newToken && !token.isExpired() && token != AuthenticationToken.ANONYMOUS) { String signedToken = mySigner.sign(token.toString()); - tokenToAuthCookie(httpResponse, - signedToken, - getCookieDomain(), - getCookiePath(), - token.getExpires(), - !token.isExpired() && token.getExpires() > 0, - isHttps + tokenToAuthCookie( + httpResponse, + signedToken, + getCookieDomain(), + getCookiePath(), + token.getExpires(), + !token.isExpired() && token.getExpires() > 0, + isHttps ); } + // Since this request is validated also set DRUID_AUTHENTICATION_RESULT + request.setAttribute( + AuthConfig.DRUID_AUTHENTICATION_RESULT, + new AuthenticationResult(token.getName(), authorizerName, null) + ); doFilter(filterChain, httpRequest, httpResponse); } } else { @@ -361,8 +361,14 @@ public Principal getUserPrincipal() } if (unauthorizedResponse) { if (!httpResponse.isCommitted()) { - tokenToAuthCookie(httpResponse, "", getCookieDomain(), - getCookiePath(), 0, false, isHttps + tokenToAuthCookie( + httpResponse, + "", + getCookieDomain(), + getCookiePath(), + 0, + false, + isHttps ); // If response code is 401. Then WWW-Authenticate Header should be // present.. reset to 403 if not found.. diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterConfig.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterConfig.java deleted file mode 100644 index 4dd4f9b01f97..000000000000 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterConfig.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * 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.security.kerberos; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.util.Collections; -import java.util.List; - -public class SpnegoFilterConfig -{ - - public static final List DEFAULT_EXCLUDED_PATHS = Collections.emptyList(); - - @JsonProperty - private final String principal; - - @JsonProperty - private final String keytab; - - @JsonProperty - private final String authToLocal; - - @JsonProperty - private final List excludedPaths; - - @JsonProperty - private final String cookieSignatureSecret; - - @JsonCreator - public SpnegoFilterConfig( - @JsonProperty("principal") String principal, - @JsonProperty("keytab") String keytab, - @JsonProperty("authToLocal") String authToLocal, - @JsonProperty("excludedPaths") List excludedPaths, - @JsonProperty("cookieSignatureSecret") String cookieSignatureSecret - ) - { - this.principal = principal; - this.keytab = keytab; - this.authToLocal = authToLocal == null ? "DEFAULT" : authToLocal; - this.excludedPaths = excludedPaths == null ? DEFAULT_EXCLUDED_PATHS : excludedPaths; - this.cookieSignatureSecret = cookieSignatureSecret; - } - - @JsonProperty - public String getPrincipal() - { - return principal; - } - - @JsonProperty - public String getKeytab() - { - return keytab; - } - - @JsonProperty - public String getAuthToLocal() - { - return authToLocal; - } - - @JsonProperty - public List getExcludedPaths() - { - return excludedPaths; - } - - @JsonProperty - public String getCookieSignatureSecret() - { - return cookieSignatureSecret; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - SpnegoFilterConfig that = (SpnegoFilterConfig) o; - - if (principal != null ? !principal.equals(that.principal) : that.principal != null) { - return false; - } - if (keytab != null ? !keytab.equals(that.keytab) : that.keytab != null) { - return false; - } - if (authToLocal != null ? !authToLocal.equals(that.authToLocal) : that.authToLocal != null) { - return false; - } - if (excludedPaths != null ? !excludedPaths.equals(that.excludedPaths) : that.excludedPaths != null) { - return false; - } - return cookieSignatureSecret != null - ? cookieSignatureSecret.equals(that.cookieSignatureSecret) - : that.cookieSignatureSecret == null; - - } - - @Override - public int hashCode() - { - int result = principal != null ? principal.hashCode() : 0; - result = 31 * result + (keytab != null ? keytab.hashCode() : 0); - result = 31 * result + (authToLocal != null ? authToLocal.hashCode() : 0); - result = 31 * result + (excludedPaths != null ? excludedPaths.hashCode() : 0); - result = 31 * result + (cookieSignatureSecret != null ? cookieSignatureSecret.hashCode() : 0); - return result; - } -} diff --git a/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/SpnegoFilterConfigTest.java b/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/SpnegoFilterConfigTest.java deleted file mode 100644 index e1ee98606324..000000000000 --- a/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/SpnegoFilterConfigTest.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.security.kerberos; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.inject.Binder; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.google.inject.Module; -import com.google.inject.Provides; -import io.druid.guice.ConfigModule; -import io.druid.guice.DruidGuiceExtensions; -import io.druid.guice.JsonConfigProvider; -import io.druid.guice.LazySingleton; -import io.druid.guice.PropertiesModule; -import io.druid.jackson.DefaultObjectMapper; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Properties; - -public class SpnegoFilterConfigTest -{ - @Test - public void testserde() - { - Injector injector = Guice.createInjector( - new Module() - { - @Override - public void configure(Binder binder) - { - binder.install(new PropertiesModule(Arrays.asList("test.runtime.properties"))); - binder.install(new ConfigModule()); - binder.install(new DruidGuiceExtensions()); - JsonConfigProvider.bind(binder, "druid.hadoop.security.spnego", SpnegoFilterConfig.class); - } - - @Provides - @LazySingleton - public ObjectMapper jsonMapper() - { - return new DefaultObjectMapper(); - } - } - ); - - Properties props = injector.getInstance(Properties.class); - SpnegoFilterConfig config = injector.getInstance(SpnegoFilterConfig.class); - - Assert.assertEquals(props.getProperty("druid.hadoop.security.spnego.principal"), config.getPrincipal()); - Assert.assertEquals(props.getProperty("druid.hadoop.security.spnego.keytab"), config.getKeytab()); - Assert.assertEquals(props.getProperty("druid.hadoop.security.spnego.authToLocal"), config.getAuthToLocal()); - - - } -} diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml index f737da39c3e1..4c1a853744fb 100644 --- a/extensions-core/hdfs-storage/pom.xml +++ b/extensions-core/hdfs-storage/pom.xml @@ -145,16 +145,7 @@ org.apache.hadoop hadoop-aws ${hadoop.compile.version} - - - com.amazonaws - aws-java-sdk - - - - - com.amazonaws - aws-java-sdk-s3 + provided commons-io diff --git a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentFinder.java b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentFinder.java index 6fba009cf865..c960541e634e 100644 --- a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentFinder.java +++ b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentFinder.java @@ -22,11 +22,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Sets; import com.google.inject.Inject; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.segment.loading.DataSegmentFinder; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; -import io.druid.java.util.common.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; diff --git a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java index 5851c2fa48a9..61da166187f0 100644 --- a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java +++ b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java @@ -61,6 +61,21 @@ public void pushTaskLog(String taskId, File logFile) throws IOException { final Path path = getTaskLogFileFromId(taskId); log.info("Writing task log to: %s", path); + pushTaskFile(path, logFile); + log.info("Wrote task log to: %s", path); + } + + @Override + public void pushTaskReports(String taskId, File reportFile) throws IOException + { + final Path path = getTaskReportsFileFromId(taskId); + log.info("Writing task reports to: %s", path); + pushTaskFile(path, reportFile); + log.info("Wrote task reports to: %s", path); + } + + private void pushTaskFile(Path path, File logFile) throws IOException + { final FileSystem fs = path.getFileSystem(hadoopConfig); try ( final InputStream in = new FileInputStream(logFile); @@ -68,14 +83,24 @@ public void pushTaskLog(String taskId, File logFile) throws IOException ) { ByteStreams.copy(in, out); } - - log.info("Wrote task log to: %s", path); } @Override public Optional streamTaskLog(final String taskId, final long offset) throws IOException { final Path path = getTaskLogFileFromId(taskId); + return streamTaskFile(path, offset); + } + + @Override + public Optional streamTaskReports(String taskId) throws IOException + { + final Path path = getTaskReportsFileFromId(taskId); + return streamTaskFile(path, 0); + } + + private Optional streamTaskFile(final Path path, final long offset) throws IOException + { final FileSystem fs = path.getFileSystem(hadoopConfig); if (fs.exists(path)) { return Optional.of( @@ -113,6 +138,15 @@ private Path getTaskLogFileFromId(String taskId) return new Path(mergePaths(config.getDirectory(), taskId.replaceAll(":", "_"))); } + /** + * Due to https://issues.apache.org/jira/browse/HDFS-13 ":" are not allowed in + * path names. So we format paths differently for HDFS. + */ + private Path getTaskReportsFileFromId(String taskId) + { + return new Path(mergePaths(config.getDirectory(), taskId.replaceAll(":", "_") + ".reports.json")); + } + // some hadoop version Path.mergePaths does not exist private static String mergePaths(String path1, String path2) { diff --git a/extensions-core/hdfs-storage/src/test/java/io/druid/indexing/common/tasklogs/HdfsTaskLogsTest.java b/extensions-core/hdfs-storage/src/test/java/io/druid/indexing/common/tasklogs/HdfsTaskLogsTest.java index 981fb51bf257..9e4843417a54 100644 --- a/extensions-core/hdfs-storage/src/test/java/io/druid/indexing/common/tasklogs/HdfsTaskLogsTest.java +++ b/extensions-core/hdfs-storage/src/test/java/io/druid/indexing/common/tasklogs/HdfsTaskLogsTest.java @@ -19,7 +19,6 @@ package io.druid.indexing.common.tasklogs; -import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; import com.google.common.io.Files; @@ -37,6 +36,7 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Map; public class HdfsTaskLogsTest @@ -50,7 +50,7 @@ public void testStream() throws Exception final File tmpDir = tempFolder.newFolder(); final File logDir = new File(tmpDir, "logs"); final File logFile = new File(tmpDir, "log"); - Files.write("blah", logFile, Charsets.UTF_8); + Files.write("blah", logFile, StandardCharsets.UTF_8); final TaskLogs taskLogs = new HdfsTaskLogs(new HdfsTaskLogsConfig(logDir.toString()), new Configuration()); taskLogs.pushTaskLog("foo", logFile); @@ -69,11 +69,11 @@ public void testOverwrite() throws Exception final File logFile = new File(tmpDir, "log"); final TaskLogs taskLogs = new HdfsTaskLogs(new HdfsTaskLogsConfig(logDir.toString()), new Configuration()); - Files.write("blah", logFile, Charsets.UTF_8); + Files.write("blah", logFile, StandardCharsets.UTF_8); taskLogs.pushTaskLog("foo", logFile); Assert.assertEquals("blah", readLog(taskLogs, "foo", 0)); - Files.write("blah blah", logFile, Charsets.UTF_8); + Files.write("blah blah", logFile, StandardCharsets.UTF_8); taskLogs.pushTaskLog("foo", logFile); Assert.assertEquals("blah blah", readLog(taskLogs, "foo", 0)); } @@ -90,7 +90,7 @@ public void testKill() throws Exception final TaskLogs taskLogs = new HdfsTaskLogs(new HdfsTaskLogsConfig(logDir.toString()), new Configuration()); - Files.write("log1content", logFile, Charsets.UTF_8); + Files.write("log1content", logFile, StandardCharsets.UTF_8); taskLogs.pushTaskLog("log1", logFile); Assert.assertEquals("log1content", readLog(taskLogs, "log1", 0)); @@ -101,7 +101,7 @@ public void testKill() throws Exception long time = (System.currentTimeMillis() / 1000) * 1000; Assert.assertTrue(fs.getFileStatus(new Path(logDirPath, "log1")).getModificationTime() < time); - Files.write("log2content", logFile, Charsets.UTF_8); + Files.write("log2content", logFile, StandardCharsets.UTF_8); taskLogs.pushTaskLog("log2", logFile); Assert.assertEquals("log2content", readLog(taskLogs, "log2", 0)); Assert.assertTrue(fs.getFileStatus(new Path(logDirPath, "log2")).getModificationTime() >= time); 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 a325948a6506..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,10 +929,11 @@ public void onFailure(Throwable t) toolbox.getDataSegmentServerAnnouncer().unannounce(); } + 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(); @@ -930,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), @@ -949,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); @@ -1126,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()); } @@ -1140,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); @@ -1165,6 +1189,7 @@ public void run() } } } + ingestionState = IngestionState.COMPLETED; } catch (Exception e) { log.error(e, "Encountered exception in runLegacy() before persisting."); @@ -1272,7 +1297,76 @@ public String apply(DataSegment input) toolbox.getDataSegmentServerAnnouncer().unannounce(); } - 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() @@ -1340,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 @@ -1491,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, @@ -2037,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/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java index c869ed4665bd..315cc0fda933 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -57,6 +56,7 @@ import java.io.IOException; import java.net.Socket; import java.net.URI; +import java.nio.charset.StandardCharsets; import java.util.Map; import java.util.TreeMap; import java.util.concurrent.Callable; @@ -551,7 +551,7 @@ private FullResponseHolder submitRequest( } log.debug("HTTP %s: %s", method.getName(), serviceUri.toString()); - response = httpClient.go(request, new FullResponseHandler(Charsets.UTF_8), httpTimeout).get(); + response = httpClient.go(request, new FullResponseHandler(StandardCharsets.UTF_8), httpTimeout).get(); } catch (Exception e) { Throwables.propagateIfInstanceOf(e.getCause(), IOException.class); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index 21ec1ed9f1ca..4c0277646368 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 @@ -51,6 +51,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, @@ -61,11 +65,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 @@ -92,6 +99,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) @@ -108,7 +126,10 @@ public static KafkaTuningConfig copyOf(KafkaTuningConfig config) config.handoffConditionTimeout, config.resetOffsetAutomatically, config.segmentWriteOutMediumFactory, - config.intermediateHandoffPeriod + config.intermediateHandoffPeriod, + config.logParseExceptions, + config.maxParseExceptions, + config.maxSavedParseExceptions ); } @@ -197,6 +218,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( @@ -211,7 +250,10 @@ public KafkaTuningConfig withBasePersistDirectory(File dir) handoffConditionTimeout, resetOffsetAutomatically, segmentWriteOutMediumFactory, - intermediateHandoffPeriod + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); } @@ -235,7 +277,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 @@ -252,7 +297,10 @@ public int hashCode() handoffConditionTimeout, resetOffsetAutomatically, segmentWriteOutMediumFactory, - intermediateHandoffPeriod + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); } @@ -271,6 +319,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 1db4580c23f9..c796b3eb9b6c 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -90,6 +90,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 c6e5a3fe8d65..4467a65d4e13 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 @@ -57,7 +57,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( @@ -72,7 +75,10 @@ public KafkaSupervisorTuningConfig( handoffConditionTimeout, resetOffsetAutomatically, segmentWriteOutMediumFactory, - intermediateHandoffPeriod + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); this.workerThreads = workerThreads; @@ -140,6 +146,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 13452ce42276..e0dd1be77184 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 @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Charsets; import com.google.common.base.Predicate; import com.google.common.base.Predicates; import com.google.common.base.Throwables; @@ -37,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; @@ -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; @@ -151,6 +159,8 @@ import java.io.IOException; 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; @@ -182,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; @@ -196,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 @@ -217,18 +231,27 @@ 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 ), new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of() ), - Charsets.UTF_8.name() + StandardCharsets.UTF_8.name() ), 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 @@ -237,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")) ); } @@ -296,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(); } @@ -312,7 +343,7 @@ public void tearDownTest() runningTasks.clear(); } - + reportsFile.delete(); destroyToolboxFactory(); } @@ -458,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( @@ -495,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); @@ -508,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()) ); @@ -923,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) { @@ -952,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) @@ -1050,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, @@ -1080,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); @@ -1119,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, @@ -1155,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); @@ -1547,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); @@ -1787,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) @@ -1826,7 +2017,10 @@ private KafkaIndexTask createTask( handoffConditionTimeout, resetOffsetAutomatically, null, - null + null, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); if (isIncrementalHandoffSupported) { context.put(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true); @@ -2032,7 +2226,8 @@ public List getLocations() EasyMock.createNiceMock(DruidNodeAnnouncer.class), EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), - new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0) + new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), + new TaskReportFileWriter(reportsFile) ); } @@ -2130,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) { @@ -2147,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 d18a34651de3..78916e7d27b7 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 a152fff6c685..c44425c17e9b 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 @@ -20,7 +20,6 @@ package io.druid.indexing.kafka.supervisor; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Charsets; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -91,6 +90,7 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -201,6 +201,9 @@ public void setupTest() TEST_HTTP_TIMEOUT, TEST_SHUTDOWN_TIMEOUT, null, + null, + null, + null, null ); @@ -2003,7 +2006,7 @@ private static DataSchema getDataSchema(String dataSource) new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of() ), - Charsets.UTF_8.name() + StandardCharsets.UTF_8.name() ), Map.class ), diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java index c0b768d3cb7c..fd6f122ad0da 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java @@ -19,13 +19,13 @@ package io.druid.data.input; -import com.google.common.base.Charsets; import com.google.common.io.ByteSource; import com.google.common.io.LineProcessor; import io.druid.java.util.common.ISE; import io.druid.java.util.common.parsers.Parser; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Map; /** @@ -79,7 +79,7 @@ public int getEntries() */ public PopulateResult populate(final ByteSource source, final Map map) throws IOException { - return source.asCharSource(Charsets.UTF_8).readLines( + return source.asCharSource(StandardCharsets.UTF_8).readLines( new LineProcessor() { private int lines = 0; diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/UriCacheGenerator.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/UriCacheGenerator.java index 27fd2eacbdfd..c2ca336c2eca 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/UriCacheGenerator.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/UriCacheGenerator.java @@ -134,28 +134,14 @@ public CacheScheduler.VersionedCache generateCache( catch (NumberFormatException ex) { log.debug(ex, "Failed to get last modified timestamp. Assuming no timestamp"); } - final ByteSource source; - if (CompressionUtils.isGz(uriPath)) { - // Simple gzip stream - log.debug("Loading gz"); - source = new ByteSource() + final ByteSource source = new ByteSource() + { + @Override + public InputStream openStream() throws IOException { - @Override - public InputStream openStream() throws IOException - { - return CompressionUtils.gzipInputStream(puller.getInputStream(uri)); - } - }; - } else { - source = new ByteSource() - { - @Override - public InputStream openStream() throws IOException - { - return puller.getInputStream(uri); - } - }; - } + return CompressionUtils.decompress(puller.getInputStream(uri), uri.getPath()); + } + }; final CacheScheduler.VersionedCache versionedCache = scheduler.createVersionedCache(entryId, version); try { diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheScheduler.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheScheduler.java index 7669168d1f4a..1f1d3e3f8135 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheScheduler.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheScheduler.java @@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Throwables; import com.google.inject.Inject; +import io.druid.concurrent.ConcurrentAwaitableCounter; import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.java.util.emitter.service.ServiceMetricEvent; import io.druid.guice.LazySingleton; @@ -51,11 +52,11 @@ * // cacheState could be either NoCache or VersionedCache. * if (cacheState instanceof NoCache) { * // the cache is not yet created, or already closed - * } else if (cacheState instanceof VersionedCache) { + * } else { * Map cache = ((VersionedCache) cacheState).getCache(); // use the cache * // Although VersionedCache implements AutoCloseable, versionedCache shouldn't be manually closed * // when obtained from entry.getCacheState(). If the namespace updates should be ceased completely, - * // entry.close() (see below) should be called, it will close the last VersionedCache itself. + * // entry.close() (see below) should be called, it will close the last VersionedCache as well. * // On scheduled updates, outdated VersionedCaches are also closed automatically. * } * ... @@ -105,14 +106,16 @@ Future getUpdaterFuture() return impl.updaterFuture; } + @VisibleForTesting public void awaitTotalUpdates(int totalUpdates) throws InterruptedException { - impl.updateCounter.awaitTotalUpdates(totalUpdates); + impl.updateCounter.awaitCount(totalUpdates); } + @VisibleForTesting void awaitNextUpdates(int nextUpdates) throws InterruptedException { - impl.updateCounter.awaitNextUpdates(nextUpdates); + impl.updateCounter.awaitNextIncrements(nextUpdates); } /** @@ -145,7 +148,7 @@ public class EntryImpl implements AutoCloseable private final Future updaterFuture; private final Cleaner entryCleaner; private final CacheGenerator cacheGenerator; - private final UpdateCounter updateCounter = new UpdateCounter(); + private final ConcurrentAwaitableCounter updateCounter = new ConcurrentAwaitableCounter(); private final CountDownLatch startLatch = new CountDownLatch(1); private EntryImpl(final T namespace, final Entry entry, final CacheGenerator cacheGenerator) @@ -276,7 +279,7 @@ private CacheState swapCacheState(VersionedCache newVersionedCache) return lastCacheState; } } while (!cacheStateHolder.compareAndSet(lastCacheState, newVersionedCache)); - updateCounter.update(); + updateCounter.increment(); return lastCacheState; } @@ -485,7 +488,7 @@ public Entry scheduleAndWait(ExtractionNamespace namespace, long waitForFirstRun log.debug("Scheduled new %s", entry); boolean success = false; try { - success = entry.impl.updateCounter.awaitFirstUpdate(waitForFirstRunMs, TimeUnit.MILLISECONDS); + success = entry.impl.updateCounter.awaitFirstIncrement(waitForFirstRunMs, TimeUnit.MILLISECONDS); if (success) { return entry; } else { diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/UpdateCounter.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/UpdateCounter.java deleted file mode 100644 index ed229b45f891..000000000000 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/UpdateCounter.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * 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.server.lookup.namespace.cache; - -import java.util.concurrent.Phaser; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -final class UpdateCounter -{ - /** - * Max {@link Phaser}'s phase, specified in it's javadoc. Then it wraps to zero. - */ - private static final int MAX_PHASE = Integer.MAX_VALUE; - - private final Phaser phaser = new Phaser(1); - - void update() - { - phaser.arrive(); - } - - void awaitTotalUpdates(int totalUpdates) throws InterruptedException - { - totalUpdates &= MAX_PHASE; - int currentUpdates = phaser.getPhase(); - checkNotTerminated(currentUpdates); - while (comparePhases(totalUpdates, currentUpdates) > 0) { - currentUpdates = phaser.awaitAdvanceInterruptibly(currentUpdates); - checkNotTerminated(currentUpdates); - } - } - - private static int comparePhases(int phase1, int phase2) - { - int diff = (phase1 - phase2) & MAX_PHASE; - if (diff == 0) { - return 0; - } - return diff < MAX_PHASE / 2 ? 1 : -1; - } - - private void checkNotTerminated(int phase) - { - if (phase < 0) { - throw new IllegalStateException("Phaser[" + phaser + "] unexpectedly terminated."); - } - } - - void awaitNextUpdates(int nextUpdates) throws InterruptedException - { - if (nextUpdates <= 0) { - throw new IllegalArgumentException("nextUpdates is not positive: " + nextUpdates); - } - if (nextUpdates > MAX_PHASE / 4) { - throw new UnsupportedOperationException("Couldn't wait for so many updates: " + nextUpdates); - } - awaitTotalUpdates(phaser.getPhase() + nextUpdates); - } - - boolean awaitFirstUpdate(long timeout, TimeUnit unit) throws InterruptedException - { - try { - phaser.awaitAdvanceInterruptibly(0, timeout, unit); - return true; - } - catch (TimeoutException e) { - return false; - } - } -} diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java index e636d66bdc59..c07c1ea43d9a 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java @@ -19,9 +19,7 @@ package io.druid.query.lookup.namespace; - import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -39,6 +37,7 @@ import org.junit.rules.TemporaryFolder; import java.io.File; +import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -68,7 +67,7 @@ public class JSONFlatDataParserTest public void setUp() throws Exception { tmpFile = temporaryFolder.newFile("lookup.json"); - final CharSink sink = Files.asByteSink(tmpFile).asCharSink(Charsets.UTF_8); + final CharSink sink = Files.asByteSink(tmpFile).asCharSink(StandardCharsets.UTF_8); sink.writeLines( Iterables.transform( MAPPINGS, diff --git a/extensions-core/s3-extensions/pom.xml b/extensions-core/s3-extensions/pom.xml index 3d389eab4d3f..487d33782542 100644 --- a/extensions-core/s3-extensions/pom.xml +++ b/extensions-core/s3-extensions/pom.xml @@ -18,80 +18,75 @@ - 4.0.0 + 4.0.0 - io.druid.extensions - druid-s3-extensions - druid-s3-extensions - druid-s3-extensions + io.druid.extensions + druid-s3-extensions + druid-s3-extensions + druid-s3-extensions - - io.druid - druid - 0.13.0-SNAPSHOT - ../../pom.xml - + + io.druid + druid + 0.13.0-SNAPSHOT + ../../pom.xml + - - - io.druid - druid-api - ${project.parent.version} - provided - - - io.druid - druid-aws-common - ${project.parent.version} - provided - - - io.druid - java-util - ${project.parent.version} - provided - - - net.java.dev.jets3t - jets3t - provided - - - commons-io - commons-io - provided - - - com.fasterxml.jackson.module - jackson-module-guice - ${jackson.version} - provided - + + + io.druid + druid-api + ${project.parent.version} + provided + + + io.druid + druid-aws-common + ${project.parent.version} + provided + + + io.druid + java-util + ${project.parent.version} + provided + + + commons-io + commons-io + provided + + + com.fasterxml.jackson.module + jackson-module-guice + ${jackson.version} + provided + - - - io.druid - druid-server - ${project.parent.version} - test - - - io.druid - druid-processing - ${project.parent.version} - test-jar - test - - - junit - junit - test - - - org.easymock - easymock - test - - + + + io.druid + druid-server + ${project.parent.version} + test + + + io.druid + druid-processing + ${project.parent.version} + test-jar + test + + + junit + junit + test + + + org.easymock + easymock + test + + diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index fa649dc495c7..8c41d6f1f129 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -19,28 +19,32 @@ package io.druid.firehose.s3; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.AmazonS3Exception; +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.S3Object; +import com.amazonaws.services.s3.model.S3ObjectSummary; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import com.google.common.collect.Lists; import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.IOE; +import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.storage.s3.S3Utils; -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.ServiceException; -import org.jets3t.service.StorageObjectsChunk; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; import java.io.IOException; import java.io.InputStream; import java.net.URI; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; +import java.util.Iterator; import java.util.List; import java.util.Objects; import java.util.stream.Collectors; @@ -48,18 +52,18 @@ /** * Builds firehoses that read from a predefined list of S3 objects and then dry up. */ -public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory +public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private static final Logger log = new Logger(StaticS3FirehoseFactory.class); - private static final long MAX_LISTING_LENGTH = 1024; + private static final int MAX_LISTING_LENGTH = 1024; - private final RestS3Service s3Client; + private final AmazonS3 s3Client; private final List uris; private final List prefixes; @JsonCreator public StaticS3FirehoseFactory( - @JacksonInject("s3Client") RestS3Service s3Client, + @JacksonInject("s3Client") AmazonS3 s3Client, @JsonProperty("uris") List uris, @JsonProperty("prefixes") List prefixes, @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, @@ -70,7 +74,7 @@ public StaticS3FirehoseFactory( ) { super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); - this.s3Client = Preconditions.checkNotNull(s3Client, "null s3Client"); + this.s3Client = Preconditions.checkNotNull(s3Client, "s3Client"); this.uris = uris == null ? new ArrayList<>() : uris; this.prefixes = prefixes == null ? new ArrayList<>() : prefixes; @@ -104,62 +108,58 @@ public List getPrefixes() } @Override - protected Collection initObjects() throws IOException + protected Collection initObjects() throws IOException { // Here, the returned s3 objects contain minimal information without data. // Getting data is deferred until openObjectStream() is called for each object. if (!uris.isEmpty()) { return uris.stream() - .map( - uri -> { - final String s3Bucket = uri.getAuthority(); - final S3Object s3Object = new S3Object(extractS3Key(uri)); - s3Object.setBucketName(s3Bucket); - return s3Object; - } - ) - .collect(Collectors.toList()); + .map( + uri -> { + final String s3Bucket = uri.getAuthority(); + final String key = S3Utils.extractS3Key(uri); + return S3Utils.getSingleObjectSummary(s3Client, s3Bucket, key); + } + ) + .collect(Collectors.toList()); } else { - final List objects = new ArrayList<>(); + final List objects = new ArrayList<>(); for (URI uri : prefixes) { final String bucket = uri.getAuthority(); - final String prefix = extractS3Key(uri); + final String prefix = S3Utils.extractS3Key(uri); + try { - String lastKey = null; - StorageObjectsChunk objectsChunk; - do { - objectsChunk = s3Client.listObjectsChunked( - bucket, - prefix, - null, - MAX_LISTING_LENGTH, - lastKey - ); - Arrays.stream(objectsChunk.getObjects()) - .filter(storageObject -> !storageObject.isDirectoryPlaceholder()) - .forEach(storageObject -> objects.add((S3Object) storageObject)); - lastKey = objectsChunk.getPriorLastKey(); - } while (!objectsChunk.isListingComplete()); + final Iterator objectSummaryIterator = S3Utils.objectSummaryIterator( + s3Client, + bucket, + prefix, + MAX_LISTING_LENGTH + ); + objects.addAll(Lists.newArrayList(objectSummaryIterator)); } - catch (ServiceException outerException) { + catch (AmazonS3Exception outerException) { log.error(outerException, "Exception while listing on %s", uri); - if (outerException.getResponseCode() == 403) { + if (outerException.getStatusCode() == 403) { // The "Access Denied" means users might not have a proper permission for listing on the given uri. // Usually this is not a problem, but the uris might be the full paths to input objects instead of prefixes. // In this case, users should be able to get objects if they have a proper permission for GetObject. log.warn("Access denied for %s. Try to get the object from the uri without listing", uri); try { - final S3Object s3Object = s3Client.getObject(bucket, prefix); - if (!s3Object.isDirectoryPlaceholder()) { - objects.add(s3Object); + final ObjectMetadata objectMetadata = s3Client.getObjectMetadata(bucket, prefix); + + if (!S3Utils.isDirectoryPlaceholder(prefix, objectMetadata)) { + objects.add(S3Utils.getSingleObjectSummary(s3Client, bucket, prefix)); } else { - throw new IOException(uri + " is a directory placeholder, " - + "but failed to get the object list under the directory due to permission"); + throw new IOE( + "[%s] is a directory placeholder, " + + "but failed to get the object list under the directory due to permission", + uri + ); } } - catch (S3ServiceException innerException) { + catch (AmazonS3Exception innerException) { throw new IOException(innerException); } } else { @@ -171,51 +171,48 @@ protected Collection initObjects() throws IOException } } - private static String extractS3Key(URI uri) - { - return uri.getPath().startsWith("/") - ? uri.getPath().substring(1) - : uri.getPath(); - } - @Override - protected InputStream openObjectStream(S3Object object) throws IOException + protected InputStream openObjectStream(S3ObjectSummary object) throws IOException { try { // Get data of the given object and open an input stream - return s3Client.getObject(object.getBucketName(), object.getKey()).getDataInputStream(); + final S3Object s3Object = s3Client.getObject(object.getBucketName(), object.getKey()); + if (s3Object == null) { + throw new ISE("Failed to get an s3 object for bucket[%s] and key[%s]", object.getBucketName(), object.getKey()); + } + return s3Object.getObjectContent(); } - catch (ServiceException e) { + catch (AmazonS3Exception e) { throw new IOException(e); } } @Override - protected InputStream openObjectStream(S3Object object, long start) throws IOException + protected InputStream openObjectStream(S3ObjectSummary object, long start) throws IOException { + final GetObjectRequest request = new GetObjectRequest(object.getBucketName(), object.getKey()); + request.setRange(start); try { - final S3Object result = s3Client.getObject( - object.getBucketName(), - object.getKey(), - null, - null, - null, - null, - start, - null - ); - - return result.getDataInputStream(); + final S3Object s3Object = s3Client.getObject(request); + if (s3Object == null) { + throw new ISE( + "Failed to get an s3 object for bucket[%s], key[%s], and start[%d]", + object.getBucketName(), + object.getKey(), + start + ); + } + return s3Object.getObjectContent(); } - catch (ServiceException e) { + catch (AmazonS3Exception e) { throw new IOException(e); } } @Override - protected InputStream wrapObjectStream(S3Object object, InputStream stream) throws IOException + protected InputStream wrapObjectStream(S3ObjectSummary object, InputStream stream) throws IOException { - return object.getKey().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + return CompressionUtils.decompress(stream, object.getKey()); } @Override diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java deleted file mode 100644 index 7a64a81e7c96..000000000000 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.storage.s3; - -import com.amazonaws.auth.AWSCredentialsProvider; -import org.jets3t.service.security.AWSSessionCredentials; - -public class AWSSessionCredentialsAdapter extends AWSSessionCredentials -{ - private final AWSCredentialsProvider provider; - - public AWSSessionCredentialsAdapter(AWSCredentialsProvider provider) - { - super(null, null, null); - if (provider.getCredentials() instanceof com.amazonaws.auth.AWSSessionCredentials) { - this.provider = provider; - } else { - throw new IllegalArgumentException("provider does not contain session credentials"); - } - } - - @Override - protected String getTypeName() - { - return "AWSSessionCredentialsAdapter"; - } - - @Override - public String getVersionPrefix() - { - return "AWSSessionCredentialsAdapter, version: "; - } - - @Override - public String getAccessKey() - { - return provider.getCredentials().getAWSAccessKeyId(); - } - - @Override - public String getSecretKey() - { - return provider.getCredentials().getAWSSecretKey(); - } - - @Override - public String getSessionToken() - { - com.amazonaws.auth.AWSSessionCredentials sessionCredentials = - (com.amazonaws.auth.AWSSessionCredentials) provider.getCredentials(); - return sessionCredentials.getSessionToken(); - } -} diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java index d7bc1b2d4912..42eef5ce819e 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java @@ -19,6 +19,7 @@ package io.druid.storage.s3; +import com.amazonaws.services.s3.AmazonS3; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Objects; import com.google.common.collect.ImmutableMap; @@ -28,7 +29,6 @@ import io.druid.segment.loading.LoadSpec; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; public class S3DataSegmentArchiver extends S3DataSegmentMover implements DataSegmentArchiver @@ -40,7 +40,7 @@ public class S3DataSegmentArchiver extends S3DataSegmentMover implements DataSeg @Inject public S3DataSegmentArchiver( @Json ObjectMapper mapper, - RestS3Service s3Client, + AmazonS3 s3Client, S3DataSegmentArchiverConfig archiveConfig, S3DataSegmentPusherConfig restoreConfig ) diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentFinder.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentFinder.java index d6d773640e8e..649554e7564b 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentFinder.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentFinder.java @@ -19,22 +19,24 @@ package io.druid.storage.s3; +import com.amazonaws.AmazonServiceException; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.S3Object; +import com.amazonaws.services.s3.model.S3ObjectInputStream; +import com.amazonaws.services.s3.model.S3ObjectSummary; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Sets; import com.google.inject.Inject; - +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.segment.loading.DataSegmentFinder; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; -import org.jets3t.service.ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; -import org.jets3t.service.model.StorageObject; +import java.io.ByteArrayInputStream; import java.io.IOException; -import java.io.InputStream; import java.util.Iterator; import java.util.Map; import java.util.Set; @@ -43,13 +45,13 @@ public class S3DataSegmentFinder implements DataSegmentFinder { private static final Logger log = new Logger(S3DataSegmentFinder.class); - private final RestS3Service s3Client; + private final AmazonS3 s3Client; private final ObjectMapper jsonMapper; private final S3DataSegmentPusherConfig config; @Inject public S3DataSegmentFinder( - RestS3Service s3Client, + AmazonS3 s3Client, S3DataSegmentPusherConfig config, ObjectMapper jsonMapper ) @@ -65,24 +67,24 @@ public Set findSegments(String workingDirPath, boolean updateDescri final Set segments = Sets.newHashSet(); try { - Iterator objectsIterator = S3Utils.storageObjectsIterator( + final Iterator objectSummaryIterator = S3Utils.objectSummaryIterator( s3Client, config.getBucket(), workingDirPath.length() == 0 ? config.getBaseKey() : workingDirPath, - config.getMaxListingLength()); + config.getMaxListingLength() + ); - while (objectsIterator.hasNext()) { - StorageObject storageObject = objectsIterator.next(); - storageObject.closeDataInputStream(); + while (objectSummaryIterator.hasNext()) { + final S3ObjectSummary objectSummary = objectSummaryIterator.next(); - if (S3Utils.toFilename(storageObject.getKey()).equals("descriptor.json")) { - final String descriptorJson = storageObject.getKey(); + if (S3Utils.toFilename(objectSummary.getKey()).equals("descriptor.json")) { + final String descriptorJson = objectSummary.getKey(); String indexZip = S3Utils.indexZipForSegmentPath(descriptorJson); - if (S3Utils.isObjectInBucket(s3Client, config.getBucket(), indexZip)) { - S3Object indexObject = s3Client.getObject(config.getBucket(), descriptorJson); - - try (InputStream is = indexObject.getDataInputStream()) { + if (S3Utils.isObjectInBucketIgnoringPermission(s3Client, config.getBucket(), indexZip)) { + try (S3Object indexObject = s3Client.getObject(config.getBucket(), descriptorJson); + S3ObjectInputStream is = indexObject.getObjectContent()) { + final ObjectMetadata objectMetadata = indexObject.getObjectMetadata(); final DataSegment dataSegment = jsonMapper.readValue(is, DataSegment.class); log.info("Found segment [%s] located at [%s]", dataSegment.getIdentifier(), indexZip); @@ -99,8 +101,10 @@ public Set findSegments(String workingDirPath, boolean updateDescri descriptorJson, indexObject ); - S3Object newDescJsonObject = new S3Object(descriptorJson, jsonMapper.writeValueAsString(dataSegment)); - s3Client.putObject(config.getBucket(), newDescJsonObject); + final ByteArrayInputStream bais = new ByteArrayInputStream( + StringUtils.toUtf8(jsonMapper.writeValueAsString(dataSegment)) + ); + s3Client.putObject(config.getBucket(), descriptorJson, bais, objectMetadata); } } segments.add(dataSegment); @@ -114,7 +118,7 @@ public Set findSegments(String workingDirPath, boolean updateDescri } } } - catch (ServiceException e) { + catch (AmazonServiceException e) { throw new SegmentLoadingException(e, "Problem interacting with S3"); } catch (IOException e) { diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java index b1503e42b75f..4053fdd6056a 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java @@ -19,14 +19,14 @@ package io.druid.storage.s3; +import com.amazonaws.AmazonServiceException; +import com.amazonaws.services.s3.AmazonS3; import com.google.inject.Inject; import io.druid.java.util.common.MapUtils; import io.druid.java.util.common.logger.Logger; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; -import org.jets3t.service.ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; import java.util.Map; @@ -36,11 +36,11 @@ public class S3DataSegmentKiller implements DataSegmentKiller { private static final Logger log = new Logger(S3DataSegmentKiller.class); - private final RestS3Service s3Client; + private final AmazonS3 s3Client; @Inject public S3DataSegmentKiller( - RestS3Service s3Client + AmazonS3 s3Client ) { this.s3Client = s3Client; @@ -55,16 +55,16 @@ public void kill(DataSegment segment) throws SegmentLoadingException String s3Path = MapUtils.getString(loadSpec, "key"); String s3DescriptorPath = S3Utils.descriptorPathForSegmentPath(s3Path); - if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { + if (s3Client.doesObjectExist(s3Bucket, s3Path)) { log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); s3Client.deleteObject(s3Bucket, s3Path); } - if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { + if (s3Client.doesObjectExist(s3Bucket, s3DescriptorPath)) { log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); s3Client.deleteObject(s3Bucket, s3DescriptorPath); } } - catch (ServiceException e) { + catch (AmazonServiceException e) { throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getIdentifier(), e); } } diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index 1c22418e2b4c..e50ea2cca700 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -19,6 +19,13 @@ package io.druid.storage.s3; +import com.amazonaws.AmazonServiceException; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.CopyObjectRequest; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.S3ObjectSummary; +import com.amazonaws.services.s3.model.StorageClass; import com.google.common.base.Predicate; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -34,10 +41,6 @@ import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; -import org.jets3t.service.ServiceException; -import org.jets3t.service.acl.gs.GSAccessControlList; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; import java.io.IOException; import java.util.Map; @@ -46,12 +49,12 @@ public class S3DataSegmentMover implements DataSegmentMover { private static final Logger log = new Logger(S3DataSegmentMover.class); - private final RestS3Service s3Client; + private final AmazonS3 s3Client; private final S3DataSegmentPusherConfig config; @Inject public S3DataSegmentMover( - RestS3Service s3Client, + AmazonS3 s3Client, S3DataSegmentPusherConfig config ) { @@ -103,7 +106,7 @@ public boolean apply(String input) .build() ); } - catch (ServiceException e) { + catch (AmazonServiceException e) { throw new SegmentLoadingException(e, "Unable to move segment[%s]: [%s]", segment.getIdentifier(), e); } } @@ -113,7 +116,7 @@ private void safeMove( final String s3Path, final String targetS3Bucket, final String targetS3Path - ) throws ServiceException, SegmentLoadingException + ) throws SegmentLoadingException { try { S3Utils.retryS3Operation( @@ -129,7 +132,7 @@ private void safeMove( selfCheckingMove(s3Bucket, targetS3Bucket, s3Path, targetS3Path, copyMsg); return null; } - catch (ServiceException | IOException | SegmentLoadingException e) { + catch (AmazonServiceException | IOException | SegmentLoadingException e) { log.info(e, "Error while trying to move " + copyMsg); throw e; } @@ -137,7 +140,7 @@ private void safeMove( ); } catch (Exception e) { - Throwables.propagateIfInstanceOf(e, ServiceException.class); + Throwables.propagateIfInstanceOf(e, AmazonServiceException.class); Throwables.propagateIfInstanceOf(e, SegmentLoadingException.class); throw Throwables.propagate(e); } @@ -155,40 +158,41 @@ private void selfCheckingMove( String s3Path, String targetS3Path, String copyMsg - ) throws ServiceException, IOException, SegmentLoadingException + ) throws IOException, SegmentLoadingException { if (s3Bucket.equals(targetS3Bucket) && s3Path.equals(targetS3Path)) { log.info("No need to move file[s3://%s/%s] onto itself", s3Bucket, s3Path); return; } - if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { - final S3Object[] list = s3Client.listObjects(s3Bucket, s3Path, ""); - if (list.length == 0) { + if (s3Client.doesObjectExist(s3Bucket, s3Path)) { + final ListObjectsV2Result listResult = s3Client.listObjectsV2( + new ListObjectsV2Request() + .withBucketName(s3Bucket) + .withPrefix(s3Path) + .withMaxKeys(1) + ); + if (listResult.getKeyCount() == 0) { // should never happen throw new ISE("Unable to list object [s3://%s/%s]", s3Bucket, s3Path); } - final S3Object s3Object = list[0]; - if (s3Object.getStorageClass() != null && - s3Object.getStorageClass().equals(S3Object.STORAGE_CLASS_GLACIER)) { - throw new ServiceException(StringUtils.format( - "Cannot move file[s3://%s/%s] of storage class glacier, skipping.", - s3Bucket, - s3Path - )); + final S3ObjectSummary objectSummary = listResult.getObjectSummaries().get(0); + if (objectSummary.getStorageClass() != null && + StorageClass.fromValue(StringUtils.toUpperCase(objectSummary.getStorageClass())).equals(StorageClass.Glacier)) { + throw new AmazonServiceException( + StringUtils.format( + "Cannot move file[s3://%s/%s] of storage class glacier, skipping.", + s3Bucket, + s3Path + ) + ); } else { log.info("Moving file %s", copyMsg); - final S3Object target = new S3Object(targetS3Path); + final CopyObjectRequest copyRequest = new CopyObjectRequest(s3Bucket, s3Path, targetS3Bucket, targetS3Path); if (!config.getDisableAcl()) { - target.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + copyRequest.setAccessControlList(S3Utils.grantFullControlToBucketOwner(s3Client, targetS3Bucket)); } - s3Client.copyObject( - s3Bucket, - s3Path, - targetS3Bucket, - target, - false - ); - if (!s3Client.isObjectInBucket(targetS3Bucket, targetS3Path)) { + s3Client.copyObject(copyRequest); + if (!s3Client.doesObjectExist(targetS3Bucket, targetS3Path)) { throw new IOE( "After copy was reported as successful the file doesn't exist in the target location [%s]", copyMsg @@ -199,7 +203,7 @@ private void selfCheckingMove( } } else { // ensure object exists in target location - if (s3Client.isObjectInBucket(targetS3Bucket, targetS3Path)) { + if (s3Client.doesObjectExist(targetS3Bucket, targetS3Path)) { log.info( "Not moving file [s3://%s/%s], already present in target location [s3://%s/%s]", s3Bucket, s3Path, diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java index 1b2765af2094..55a00a76b8b8 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java @@ -19,6 +19,11 @@ package io.druid.storage.s3; +import com.amazonaws.AmazonServiceException; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.AmazonS3Exception; +import com.amazonaws.services.s3.model.S3Object; +import com.amazonaws.services.s3.model.S3ObjectSummary; import com.google.common.base.Predicate; import com.google.common.base.Strings; import com.google.common.base.Throwables; @@ -33,17 +38,15 @@ import io.druid.java.util.common.RE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.UOE; +import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.logger.Logger; import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.loading.URIDataPuller; import io.druid.timeline.DataSegment; -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.StorageObject; import javax.tools.FileObject; import java.io.File; +import java.io.FilterInputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -59,17 +62,15 @@ public class S3DataSegmentPuller implements URIDataPuller { public static final int DEFAULT_RETRY_COUNT = 3; - public static FileObject buildFileObject(final URI uri, final RestS3Service s3Client) throws ServiceException + private static FileObject buildFileObject(final URI uri, final AmazonS3 s3Client) throws AmazonServiceException { final S3Coords coords = new S3Coords(checkURI(uri)); - final StorageObject s3Obj = s3Client.getObjectDetails(coords.bucket, coords.path); + final S3ObjectSummary objectSummary = S3Utils.getSingleObjectSummary(s3Client, coords.bucket, coords.path); final String path = uri.getPath(); return new FileObject() { - final Object inputStreamOpener = new Object(); - volatile boolean streamAcquired = false; - volatile StorageObject storageObject = s3Obj; + S3Object s3Object = null; @Override public URI toUri() @@ -84,22 +85,33 @@ public String getName() return Files.getNameWithoutExtension(path) + (Strings.isNullOrEmpty(ext) ? "" : ("." + ext)); } + /** + * Returns an input stream for a s3 object. The returned input stream is not thread-safe. + */ @Override public InputStream openInputStream() throws IOException { try { - synchronized (inputStreamOpener) { - if (streamAcquired) { - return storageObject.getDataInputStream(); - } + if (s3Object == null) { // lazily promote to full GET - storageObject = s3Client.getObject(s3Obj.getBucketName(), s3Obj.getKey()); - final InputStream stream = storageObject.getDataInputStream(); - streamAcquired = true; - return stream; + s3Object = s3Client.getObject(objectSummary.getBucketName(), objectSummary.getKey()); } + + final InputStream in = s3Object.getObjectContent(); + final Closer closer = Closer.create(); + closer.register(in); + closer.register(s3Object); + + return new FilterInputStream(in) + { + @Override + public void close() throws IOException + { + closer.close(); + } + }; } - catch (ServiceException e) { + catch (AmazonServiceException e) { throw new IOE(e, "Could not load S3 URI [%s]", uri); } } @@ -131,7 +143,7 @@ public Writer openWriter() @Override public long getLastModified() { - return s3Obj.getLastModifiedDate().getTime(); + return objectSummary.getLastModified().getTime(); } @Override @@ -149,11 +161,11 @@ public boolean delete() protected static final String BUCKET = "bucket"; protected static final String KEY = "key"; - protected final RestS3Service s3Client; + protected final AmazonS3 s3Client; @Inject public S3DataSegmentPuller( - RestS3Service s3Client + AmazonS3 s3Client ) { this.s3Client = s3Client; @@ -180,7 +192,7 @@ public InputStream openStream() throws IOException try { return buildFileObject(uri, s3Client).openInputStream(); } - catch (ServiceException e) { + catch (AmazonServiceException e) { if (e.getCause() != null) { if (S3Utils.S3RETRY.apply(e)) { throw new IOException("Recoverable exception", e); @@ -242,7 +254,7 @@ public InputStream getInputStream(URI uri) throws IOException try { return buildFileObject(uri, s3Client).openInputStream(); } - catch (ServiceException e) { + catch (AmazonServiceException e) { throw new IOE(e, "Could not load URI [%s]", uri); } } @@ -259,8 +271,8 @@ public boolean apply(Throwable e) if (e == null) { return false; } - if (e instanceof ServiceException) { - return S3Utils.isServiceExceptionRecoverable((ServiceException) e); + if (e instanceof AmazonServiceException) { + return S3Utils.isServiceExceptionRecoverable((AmazonServiceException) e); } if (S3Utils.S3RETRY.apply(e)) { return true; @@ -284,10 +296,11 @@ public boolean apply(Throwable e) public String getVersion(URI uri) throws IOException { try { - final FileObject object = buildFileObject(uri, s3Client); - return StringUtils.format("%d", object.getLastModified()); + final S3Coords coords = new S3Coords(checkURI(uri)); + final S3ObjectSummary objectSummary = S3Utils.getSingleObjectSummary(s3Client, coords.bucket, coords.path); + return StringUtils.format("%d", objectSummary.getLastModified().getTime()); } - catch (ServiceException e) { + catch (AmazonServiceException e) { if (S3Utils.isServiceExceptionRecoverable(e)) { // The recoverable logic is always true for IOException, so we want to only pass IOException if it is recoverable throw new IOE(e, "Could not fetch last modified timestamp from URI [%s]", uri); @@ -301,10 +314,10 @@ private boolean isObjectInBucket(final S3Coords coords) throws SegmentLoadingExc { try { return S3Utils.retryS3Operation( - () -> S3Utils.isObjectInBucket(s3Client, coords.bucket, coords.path) + () -> S3Utils.isObjectInBucketIgnoringPermission(s3Client, coords.bucket, coords.path) ); } - catch (S3ServiceException | IOException e) { + catch (AmazonS3Exception | IOException e) { throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords); } catch (Exception e) { diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java index 985121bef622..981d24a7ef5e 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java @@ -19,21 +19,20 @@ package io.druid.storage.s3; +import com.amazonaws.AmazonServiceException; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.PutObjectRequest; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; -import io.druid.java.util.emitter.EmittingLogger; import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.emitter.EmittingLogger; import io.druid.segment.SegmentUtils; import io.druid.segment.loading.DataSegmentPusher; import io.druid.timeline.DataSegment; -import org.jets3t.service.ServiceException; -import org.jets3t.service.acl.gs.GSAccessControlList; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; import java.io.File; import java.io.IOException; @@ -46,13 +45,13 @@ public class S3DataSegmentPusher implements DataSegmentPusher { private static final EmittingLogger log = new EmittingLogger(S3DataSegmentPusher.class); - private final RestS3Service s3Client; + private final AmazonS3 s3Client; private final S3DataSegmentPusherConfig config; private final ObjectMapper jsonMapper; @Inject public S3DataSegmentPusher( - RestS3Service s3Client, + AmazonS3 s3Client, S3DataSegmentPusherConfig config, ObjectMapper jsonMapper ) @@ -97,45 +96,43 @@ public DataSegment push(final File indexFilesDir, final DataSegment inSegment, f final File zipOutFile = File.createTempFile("druid", "index.zip"); final long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile); + final DataSegment outSegment = inSegment.withSize(indexSize) + .withLoadSpec(makeLoadSpec(config.getBucket(), s3Path)) + .withBinaryVersion(SegmentUtils.getVersionFromDir(indexFilesDir)); + + final File descriptorFile = File.createTempFile("druid", "descriptor.json"); + // Avoid using Guava in DataSegmentPushers because they might be used with very diverse Guava versions in + // runtime, and because Guava deletes methods over time, that causes incompatibilities. + Files.write(descriptorFile.toPath(), jsonMapper.writeValueAsBytes(outSegment)); + try { return S3Utils.retryS3Operation( () -> { - S3Object toPush = new S3Object(zipOutFile); - putObject(config.getBucket(), s3Path, toPush, replaceExisting); - - final DataSegment outSegment = inSegment.withSize(indexSize) - .withLoadSpec(makeLoadSpec(config.getBucket(), toPush.getKey())) - .withBinaryVersion(SegmentUtils.getVersionFromDir(indexFilesDir)); - - File descriptorFile = File.createTempFile("druid", "descriptor.json"); - // Avoid using Guava in DataSegmentPushers because they might be used with very diverse Guava versions in - // runtime, and because Guava deletes methods over time, that causes incompatibilities. - Files.write(descriptorFile.toPath(), jsonMapper.writeValueAsBytes(outSegment)); - S3Object descriptorObject = new S3Object(descriptorFile); - - putObject( + uploadFileIfPossible(s3Client, config.getBucket(), s3Path, zipOutFile, replaceExisting); + uploadFileIfPossible( + s3Client, config.getBucket(), S3Utils.descriptorPathForSegmentPath(s3Path), - descriptorObject, + descriptorFile, replaceExisting ); - log.info("Deleting zipped index File[%s]", zipOutFile); - zipOutFile.delete(); - - log.info("Deleting descriptor file[%s]", descriptorFile); - descriptorFile.delete(); - return outSegment; } ); } - catch (ServiceException e) { + catch (AmazonServiceException e) { throw new IOException(e); } catch (Exception e) { throw Throwables.propagate(e); } + finally { + log.info("Deleting temporary cached index.zip"); + zipOutFile.delete(); + log.info("Deleting temporary cached descriptor.json"); + descriptorFile.delete(); + } } @Override @@ -163,21 +160,26 @@ private Map makeLoadSpec(String bucket, String key) ); } - private void putObject(String bucketName, String path, S3Object object, boolean replaceExisting) - throws ServiceException + private void uploadFileIfPossible( + AmazonS3 s3Client, + String bucket, + String key, + File file, + boolean replaceExisting + ) { - object.setBucketName(bucketName); - object.setKey(path); - if (!config.getDisableAcl()) { - object.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); - } - - log.info("Pushing %s.", object); - - if (!replaceExisting && S3Utils.isObjectInBucket(s3Client, bucketName, object.getKey())) { - log.info("Skipping push because key [%s] exists && replaceExisting == false", object.getKey()); + if (!replaceExisting && S3Utils.isObjectInBucketIgnoringPermission(s3Client, bucket, key)) { + log.info("Skipping push because key [%s] exists && replaceExisting == false", key); } else { - s3Client.putObject(bucketName, object); + final PutObjectRequest indexFilePutRequest = new PutObjectRequest(bucket, key, file); + + if (!config.getDisableAcl()) { + indexFilePutRequest.setAccessControlList( + S3Utils.grantFullControlToBucketOwner(s3Client, bucket) + ); + } + log.info("Pushing [%s] to bucket[%s] and key[%s].", file, bucket, key); + s3Client.putObject(indexFilePutRequest); } } } diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java index 8b9b3fdd7cc4..ac7839f43631 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java @@ -19,8 +19,12 @@ package io.druid.storage.s3; +import com.amazonaws.ClientConfiguration; +import com.amazonaws.ClientConfigurationFactory; import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.AWSSessionCredentials; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.S3ClientOptions; import com.fasterxml.jackson.core.Version; import com.fasterxml.jackson.databind.Module; import com.google.common.collect.ImmutableList; @@ -28,13 +32,14 @@ import com.google.inject.Provides; import com.google.inject.multibindings.MapBinder; import io.druid.common.aws.AWSCredentialsConfig; +import io.druid.common.aws.AWSEndpointConfig; +import io.druid.common.aws.AWSProxyConfig; import io.druid.data.SearchableVersionedDataFinder; import io.druid.guice.Binders; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.initialization.DruidModule; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.security.AWSCredentials; +import org.apache.commons.lang.StringUtils; import java.util.List; @@ -75,6 +80,8 @@ public void setupModule(SetupContext context) public void configure(Binder binder) { JsonConfigProvider.bind(binder, "druid.s3", AWSCredentialsConfig.class); + JsonConfigProvider.bind(binder, "druid.s3.proxy", AWSProxyConfig.class); + JsonConfigProvider.bind(binder, "druid.s3.endpoint", AWSEndpointConfig.class); MapBinder.newMapBinder(binder, String.class, SearchableVersionedDataFinder.class) .addBinding("s3") .to(S3TimestampVersionedDataFinder.class) @@ -101,15 +108,44 @@ public void configure(Binder binder) @Provides @LazySingleton - public RestS3Service getRestS3Service(AWSCredentialsProvider provider) + public AmazonS3 getAmazonS3Client( + AWSCredentialsProvider provider, + AWSProxyConfig proxyConfig, + AWSEndpointConfig endpointConfig + ) { - if (provider.getCredentials() instanceof AWSSessionCredentials) { - return new RestS3Service(new AWSSessionCredentialsAdapter(provider)); - } else { - return new RestS3Service(new AWSCredentials( - provider.getCredentials().getAWSAccessKeyId(), - provider.getCredentials().getAWSSecretKey() - )); + // AmazonS3ClientBuilder can't be used because it makes integration tests failed + final ClientConfiguration configuration = new ClientConfigurationFactory().getConfig(); + final AmazonS3Client client = new AmazonS3Client(provider, setProxyConfig(configuration, proxyConfig)); + + if (StringUtils.isNotEmpty(endpointConfig.getUrl())) { + if (StringUtils.isNotEmpty(endpointConfig.getServiceName()) && + StringUtils.isNotEmpty(endpointConfig.getSigningRegion())) { + client.setEndpoint(endpointConfig.getUrl(), endpointConfig.getServiceName(), endpointConfig.getSigningRegion()); + } else { + client.setEndpoint(endpointConfig.getUrl()); + } + } + + client.setS3ClientOptions(S3ClientOptions.builder().enableForceGlobalBucketAccess().build()); + + return client; + } + + private static ClientConfiguration setProxyConfig(ClientConfiguration conf, AWSProxyConfig proxyConfig) + { + if (StringUtils.isNotEmpty(proxyConfig.getHost())) { + conf.setProxyHost(proxyConfig.getHost()); + } + if (proxyConfig.getPort() != -1) { + conf.setProxyPort(proxyConfig.getPort()); + } + if (StringUtils.isNotEmpty(proxyConfig.getUsername())) { + conf.setProxyUsername(proxyConfig.getUsername()); + } + if (StringUtils.isNotEmpty(proxyConfig.getPassword())) { + conf.setProxyPassword(proxyConfig.getPassword()); } + return conf; } } diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogs.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogs.java index 426221f508b3..e2f6d4310e43 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogs.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogs.java @@ -19,6 +19,11 @@ package io.druid.storage.s3; +import com.amazonaws.AmazonServiceException; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.AmazonS3Exception; +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.ObjectMetadata; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.io.ByteSource; @@ -27,10 +32,6 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.tasklogs.TaskLogs; -import org.jets3t.service.ServiceException; -import org.jets3t.service.StorageService; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.StorageObject; import java.io.File; import java.io.IOException; @@ -43,11 +44,11 @@ public class S3TaskLogs implements TaskLogs { private static final Logger log = new Logger(S3TaskLogs.class); - private final StorageService service; + private final AmazonS3 service; private final S3TaskLogsConfig config; @Inject - public S3TaskLogs(S3TaskLogsConfig config, RestS3Service service) + public S3TaskLogs(S3TaskLogsConfig config, AmazonS3 service) { this.config = config; this.service = service; @@ -56,12 +57,23 @@ public S3TaskLogs(S3TaskLogsConfig config, RestS3Service service) @Override public Optional streamTaskLog(final String taskid, final long offset) throws IOException { - final String taskKey = getTaskLogKey(taskid); + final String taskKey = getTaskLogKey(taskid, "log"); + return streamTaskFile(offset, taskKey); + } + @Override + public Optional streamTaskReports(String taskid) throws IOException + { + final String taskKey = getTaskLogKey(taskid, "report.json"); + return streamTaskFile(0, taskKey); + } + + private Optional streamTaskFile(final long offset, String taskKey) throws IOException + { try { - final StorageObject objectDetails = service.getObjectDetails(config.getS3Bucket(), taskKey, null, null, null, null); + final ObjectMetadata objectMetadata = service.getObjectMetadata(config.getS3Bucket(), taskKey); - return Optional.of( + return Optional.of( new ByteSource() { @Override @@ -69,36 +81,31 @@ public InputStream openStream() throws IOException { try { final long start; - final long end = objectDetails.getContentLength() - 1; + final long end = objectMetadata.getContentLength() - 1; - if (offset > 0 && offset < objectDetails.getContentLength()) { + if (offset > 0 && offset < objectMetadata.getContentLength()) { start = offset; - } else if (offset < 0 && (-1 * offset) < objectDetails.getContentLength()) { - start = objectDetails.getContentLength() + offset; + } else if (offset < 0 && (-1 * offset) < objectMetadata.getContentLength()) { + start = objectMetadata.getContentLength() + offset; } else { start = 0; } - return service.getObject( - config.getS3Bucket(), - taskKey, - null, - null, - new String[]{objectDetails.getETag()}, - null, - start, - end - ).getDataInputStream(); + final GetObjectRequest request = new GetObjectRequest(config.getS3Bucket(), taskKey) + .withMatchingETagConstraint(objectMetadata.getETag()) + .withRange(start, end); + + return service.getObject(request).getObjectContent(); } - catch (ServiceException e) { + catch (AmazonServiceException e) { throw new IOException(e); } } } ); } - catch (ServiceException e) { - if (404 == e.getResponseCode() + catch (AmazonS3Exception e) { + if (404 == e.getStatusCode() || "NoSuchKey".equals(e.getErrorCode()) || "NoSuchBucket".equals(e.getErrorCode())) { return Optional.absent(); @@ -111,15 +118,25 @@ public InputStream openStream() throws IOException @Override public void pushTaskLog(final String taskid, final File logFile) throws IOException { - final String taskKey = getTaskLogKey(taskid); + final String taskKey = getTaskLogKey(taskid, "log"); log.info("Pushing task log %s to: %s", logFile, taskKey); + pushTaskFile(logFile, taskKey); + } + + @Override + public void pushTaskReports(String taskid, File reportFile) throws IOException + { + final String taskKey = getTaskLogKey(taskid, "report.json"); + log.info("Pushing task reports %s to: %s", reportFile, taskKey); + pushTaskFile(reportFile, taskKey); + } + private void pushTaskFile(final File logFile, String taskKey) throws IOException + { try { S3Utils.retryS3Operation( () -> { - final StorageObject object = new StorageObject(logFile); - object.setKey(taskKey); - service.putObject(config.getS3Bucket(), object); + service.putObject(config.getS3Bucket(), taskKey, logFile); return null; } ); @@ -130,9 +147,9 @@ public void pushTaskLog(final String taskid, final File logFile) throws IOExcept } } - private String getTaskLogKey(String taskid) + private String getTaskLogKey(String taskid, String filename) { - return StringUtils.format("%s/%s/log", config.getS3Prefix(), taskid); + return StringUtils.format("%s/%s/%s", config.getS3Prefix(), taskid, filename); } @Override diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3TimestampVersionedDataFinder.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3TimestampVersionedDataFinder.java index 8014ec8ac88d..2d4724851b7c 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3TimestampVersionedDataFinder.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3TimestampVersionedDataFinder.java @@ -19,16 +19,17 @@ package io.druid.storage.s3; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.S3ObjectSummary; import com.google.common.base.Throwables; import com.google.inject.Inject; import io.druid.data.SearchableVersionedDataFinder; import io.druid.java.util.common.RetryUtils; import io.druid.java.util.common.StringUtils; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; import javax.annotation.Nullable; import java.net.URI; +import java.util.Iterator; import java.util.regex.Pattern; /** @@ -37,8 +38,10 @@ */ public class S3TimestampVersionedDataFinder extends S3DataSegmentPuller implements SearchableVersionedDataFinder { + private static final int MAX_LISTING_KEYS = 1000; + @Inject - public S3TimestampVersionedDataFinder(RestS3Service s3Client) + public S3TimestampVersionedDataFinder(AmazonS3 s3Client) { super(s3Client); } @@ -65,23 +68,27 @@ public URI getLatestVersion(final URI uri, final @Nullable Pattern pattern) final S3Coords coords = new S3Coords(checkURI(uri)); long mostRecent = Long.MIN_VALUE; URI latest = null; - S3Object[] objects = s3Client.listObjects(coords.bucket, coords.path, null); - if (objects == null) { - return null; - } - for (S3Object storageObject : objects) { - storageObject.closeDataInputStream(); - String keyString = storageObject.getKey().substring(coords.path.length()); + final Iterator objectSummaryIterator = S3Utils.objectSummaryIterator( + s3Client, + coords.bucket, + coords.path, + MAX_LISTING_KEYS + ); + while (objectSummaryIterator.hasNext()) { + final S3ObjectSummary objectSummary = objectSummaryIterator.next(); + String keyString = objectSummary.getKey().substring(coords.path.length()); if (keyString.startsWith("/")) { keyString = keyString.substring(1); } if (pattern != null && !pattern.matcher(keyString).matches()) { continue; } - final long latestModified = storageObject.getLastModifiedDate().getTime(); + final long latestModified = objectSummary.getLastModified().getTime(); if (latestModified >= mostRecent) { mostRecent = latestModified; - latest = new URI(StringUtils.format("s3://%s/%s", storageObject.getBucketName(), storageObject.getKey())); + latest = new URI( + StringUtils.format("s3://%s/%s", objectSummary.getBucketName(), objectSummary.getKey()) + ); } } return latest; diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java index 2a9372e96d08..c4fa15761066 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java @@ -19,19 +19,27 @@ package io.druid.storage.s3; +import com.amazonaws.AmazonServiceException; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.AccessControlList; +import com.amazonaws.services.s3.model.AmazonS3Exception; +import com.amazonaws.services.s3.model.CanonicalGrantee; +import com.amazonaws.services.s3.model.Grant; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.Permission; +import com.amazonaws.services.s3.model.S3ObjectSummary; import com.google.common.base.Joiner; import com.google.common.base.Predicate; -import com.google.common.base.Throwables; +import io.druid.java.util.common.ISE; import io.druid.java.util.common.RetryUtils; import io.druid.java.util.common.RetryUtils.Task; -import org.jets3t.service.ServiceException; -import org.jets3t.service.StorageObjectsChunk; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; -import org.jets3t.service.model.StorageObject; import java.io.IOException; +import java.net.URI; import java.util.Iterator; +import java.util.NoSuchElementException; /** * @@ -39,25 +47,12 @@ public class S3Utils { private static final Joiner JOINER = Joiner.on("/").skipNulls(); + private static final String MIMETYPE_JETS3T_DIRECTORY = "application/x-directory"; - public static void closeStreamsQuietly(S3Object s3Obj) - { - if (s3Obj == null) { - return; - } - - try { - s3Obj.closeDataInputStream(); - } - catch (IOException e) { - - } - } - - public static boolean isServiceExceptionRecoverable(ServiceException ex) + static boolean isServiceExceptionRecoverable(AmazonServiceException ex) { final boolean isIOException = ex.getCause() instanceof IOException; - final boolean isTimeout = "RequestTimeout".equals(((ServiceException) ex).getErrorCode()); + final boolean isTimeout = "RequestTimeout".equals(ex.getErrorCode()); return isIOException || isTimeout; } @@ -70,8 +65,8 @@ public boolean apply(Throwable e) return false; } else if (e instanceof IOException) { return true; - } else if (e instanceof ServiceException) { - return isServiceExceptionRecoverable((ServiceException) e); + } else if (e instanceof AmazonServiceException) { + return isServiceExceptionRecoverable((AmazonServiceException) e); } else { return apply(e.getCause()); } @@ -88,91 +83,81 @@ public static T retryS3Operation(Task f) throws Exception return RetryUtils.retry(f, S3RETRY, maxTries); } - public static boolean isObjectInBucket(RestS3Service s3Client, String bucketName, String objectKey) - throws ServiceException + static boolean isObjectInBucketIgnoringPermission(AmazonS3 s3Client, String bucketName, String objectKey) { try { - s3Client.getObjectDetails(bucketName, objectKey); + return s3Client.doesObjectExist(bucketName, objectKey); } - catch (ServiceException e) { - if (404 == e.getResponseCode() - || "NoSuchKey".equals(e.getErrorCode()) - || "NoSuchBucket".equals(e.getErrorCode())) { - return false; - } - if ("AccessDenied".equals(e.getErrorCode())) { + catch (AmazonS3Exception e) { + if (e.getStatusCode() == 404) { // Object is inaccessible to current user, but does exist. return true; } // Something else has gone wrong throw e; } - return true; } - public static Iterator storageObjectsIterator( - final RestS3Service s3Client, + public static Iterator objectSummaryIterator( + final AmazonS3 s3Client, final String bucket, final String prefix, - final long maxListingLength + final int numMaxKeys ) { - return new Iterator() + final ListObjectsV2Request request = new ListObjectsV2Request() + .withBucketName(bucket) + .withPrefix(prefix) + .withMaxKeys(numMaxKeys); + + return new Iterator() { - private StorageObjectsChunk objectsChunk; - private int objectsChunkOffset; + private ListObjectsV2Result result; + private Iterator objectSummaryIterator; - @Override - public boolean hasNext() { - if (objectsChunk == null) { - objectsChunk = listObjectsChunkedAfter(""); - objectsChunkOffset = 0; - } - - if (objectsChunk.getObjects().length <= objectsChunkOffset) { - if (objectsChunk.isListingComplete()) { - return false; - } else { - objectsChunk = listObjectsChunkedAfter(objectsChunk.getPriorLastKey()); - objectsChunkOffset = 0; - } - } + fetchNextBatch(); + } - return true; + private void fetchNextBatch() + { + result = s3Client.listObjectsV2(request); + objectSummaryIterator = result.getObjectSummaries().iterator(); + request.setContinuationToken(result.getContinuationToken()); } - private StorageObjectsChunk listObjectsChunkedAfter(final String priorLastKey) + @Override + public boolean hasNext() { - try { - return retryS3Operation( - () -> s3Client.listObjectsChunked(bucket, prefix, null, maxListingLength, priorLastKey) - ); - } - catch (Exception e) { - throw Throwables.propagate(e); - } + return objectSummaryIterator.hasNext() || result.isTruncated(); } @Override - public StorageObject next() + public S3ObjectSummary next() { if (!hasNext()) { - throw new IllegalStateException(); + throw new NoSuchElementException(); } - StorageObject storageObject = objectsChunk.getObjects()[objectsChunkOffset]; - objectsChunkOffset++; - return storageObject; - } + if (objectSummaryIterator.hasNext()) { + return objectSummaryIterator.next(); + } - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } + if (result.isTruncated()) { + fetchNextBatch(); + } + if (!objectSummaryIterator.hasNext()) { + throw new ISE( + "Failed to further iterate on bucket[%s] and prefix[%s]. The last continuationToken was [%s]", + bucket, + prefix, + result.getContinuationToken() + ); + } + return objectSummaryIterator.next(); + } }; } @@ -184,25 +169,93 @@ public static String constructSegmentPath(String baseKey, String storageDir) ) + "/index.zip"; } - public static String descriptorPathForSegmentPath(String s3Path) + static String descriptorPathForSegmentPath(String s3Path) { return s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; } - public static String indexZipForSegmentPath(String s3Path) + static String indexZipForSegmentPath(String s3Path) { return s3Path.substring(0, s3Path.lastIndexOf("/")) + "/index.zip"; } - public static String toFilename(String key) + static String toFilename(String key) { return toFilename(key, ""); } - public static String toFilename(String key, final String suffix) + static String toFilename(String key, final String suffix) { String filename = key.substring(key.lastIndexOf("/") + 1); // characters after last '/' filename = filename.substring(0, filename.length() - suffix.length()); // remove the suffix from the end return filename; } + + static AccessControlList grantFullControlToBucketOwner(AmazonS3 s3Client, String bucket) + { + final AccessControlList acl = s3Client.getBucketAcl(bucket); + acl.grantAllPermissions(new Grant(new CanonicalGrantee(acl.getOwner().getId()), Permission.FullControl)); + return acl; + } + + public static String extractS3Key(URI uri) + { + return uri.getPath().startsWith("/") ? uri.getPath().substring(1) : uri.getPath(); + } + + // Copied from org.jets3t.service.model.StorageObject.isDirectoryPlaceholder() + public static boolean isDirectoryPlaceholder(String key, ObjectMetadata objectMetadata) + { + // Recognize "standard" directory place-holder indications used by + // Amazon's AWS Console and Panic's Transmit. + if (key.endsWith("/") && objectMetadata.getContentLength() == 0) { + return true; + } + // Recognize s3sync.rb directory placeholders by MD5/ETag value. + if ("d66759af42f282e1ba19144df2d405d0".equals(objectMetadata.getETag())) { + return true; + } + // Recognize place-holder objects created by the Google Storage console + // or S3 Organizer Firefox extension. + if (key.endsWith("_$folder$") && objectMetadata.getContentLength() == 0) { + return true; + } + + // We don't use JetS3t APIs anymore, but the below check is still needed for backward compatibility. + + // Recognize legacy JetS3t directory place-holder objects, only gives + // accurate results if an object's metadata is populated. + if (objectMetadata.getContentLength() == 0 && MIMETYPE_JETS3T_DIRECTORY.equals(objectMetadata.getContentType())) { + return true; + } + return false; + } + + /** + * Gets a single {@link S3ObjectSummary} from s3. Since this method might return a wrong object if there are multiple + * objects that match the given key, this method should be used only when it's guaranteed that the given key is unique + * in the given bucket. + * + * @param s3Client s3 client + * @param bucket s3 bucket + * @param key unique key for the object to be retrieved + */ + public static S3ObjectSummary getSingleObjectSummary(AmazonS3 s3Client, String bucket, String key) + { + final ListObjectsV2Request request = new ListObjectsV2Request() + .withBucketName(bucket) + .withPrefix(key) + .withMaxKeys(1); + final ListObjectsV2Result result = s3Client.listObjectsV2(request); + + if (result.getKeyCount() == 0) { + throw new ISE("Cannot find object for bucket[%s] and key[%s]", bucket, key); + } + final S3ObjectSummary objectSummary = result.getObjectSummaries().get(0); + if (!objectSummary.getBucketName().equals(bucket) || !objectSummary.getKey().equals(key)) { + throw new ISE("Wrong object[%s] for bucket[%s] and key[%s]", objectSummary, bucket, key); + } + + return objectSummary; + } } diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java index 3a5c94471cda..e60210126943 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java @@ -19,8 +19,13 @@ package io.druid.firehose.s3; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3Client; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.deser.std.StdDeserializer; import com.fasterxml.jackson.databind.module.SimpleModule; import com.fasterxml.jackson.module.guice.ObjectMapperModule; import com.google.common.collect.ImmutableList; @@ -29,8 +34,6 @@ import com.google.inject.Injector; import com.google.inject.Provides; import io.druid.initialization.DruidModule; -import io.druid.jackson.DefaultObjectMapper; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.junit.Assert; import org.junit.Test; @@ -42,7 +45,7 @@ */ public class StaticS3FirehoseFactoryTest { - private static final RestS3Service SERVICE = new RestS3Service(null); + private static final AmazonS3Client SERVICE = new AmazonS3Client(); @Test public void testSerde() throws Exception @@ -75,14 +78,14 @@ public void testSerde() throws Exception private static ObjectMapper createObjectMapper(DruidModule baseModule) { - final ObjectMapper baseMapper = new DefaultObjectMapper(); - baseModule.getJacksonModules().forEach(baseMapper::registerModule); - final Injector injector = Guice.createInjector( new ObjectMapperModule(), baseModule ); - return injector.getInstance(ObjectMapper.class); + final ObjectMapper baseMapper = injector.getInstance(ObjectMapper.class); + + baseModule.getJacksonModules().forEach(baseMapper::registerModule); + return baseMapper; } private static class TestS3Module implements DruidModule @@ -90,7 +93,9 @@ private static class TestS3Module implements DruidModule @Override public List getJacksonModules() { - return ImmutableList.of(new SimpleModule()); + // Deserializer is need for AmazonS3Client even though it is injected. + // See https://github.com/FasterXML/jackson-databind/issues/962. + return ImmutableList.of(new SimpleModule().addDeserializer(AmazonS3.class, new ItemDeserializer())); } @Override @@ -100,9 +105,28 @@ public void configure(Binder binder) } @Provides - public RestS3Service getRestS3Service() + public AmazonS3 getAmazonS3Client() { return SERVICE; } } + + public static class ItemDeserializer extends StdDeserializer + { + public ItemDeserializer() + { + this(null); + } + + public ItemDeserializer(Class vc) + { + super(vc); + } + + @Override + public AmazonS3 deserialize(JsonParser jp, DeserializationContext ctxt) + { + throw new UnsupportedOperationException(); + } + } } diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentArchiverTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentArchiverTest.java index d28d5f933895..d93dfc08c553 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentArchiverTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentArchiverTest.java @@ -19,6 +19,7 @@ package io.druid.storage.s3; +import com.amazonaws.services.s3.AmazonS3Client; import com.fasterxml.jackson.databind.BeanProperty; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.InjectableValues; @@ -30,7 +31,6 @@ import io.druid.java.util.common.Intervals; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -65,7 +65,7 @@ public String getArchiveBaseKey() } }; private static final S3DataSegmentPusherConfig PUSHER_CONFIG = new S3DataSegmentPusherConfig(); - private static final RestS3Service S3_SERVICE = EasyMock.createStrictMock(RestS3Service.class); + private static final AmazonS3Client S3_SERVICE = EasyMock.createStrictMock(AmazonS3Client.class); private static final S3DataSegmentPuller PULLER = new S3DataSegmentPuller(S3_SERVICE); private static final DataSegment SOURCE_SEGMENT = DataSegment .builder() diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentFinderTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentFinderTest.java index 12f76126bb4e..5c449faf2e10 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentFinderTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentFinderTest.java @@ -19,6 +19,15 @@ package io.druid.storage.s3; +import com.amazonaws.AmazonServiceException; +import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.model.AmazonS3Exception; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.PutObjectResult; +import com.amazonaws.services.s3.model.S3Object; +import com.amazonaws.services.s3.model.S3ObjectSummary; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Predicate; @@ -31,17 +40,13 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.StringUtils; import io.druid.segment.TestHelper; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NumberedShardSpec; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; -import org.jets3t.service.ServiceException; -import org.jets3t.service.StorageObjectsChunk; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; -import org.jets3t.service.model.StorageObject; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -50,8 +55,12 @@ import org.junit.rules.TemporaryFolder; import javax.annotation.Nullable; +import java.io.ByteArrayInputStream; import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; import java.io.InputStream; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; @@ -105,7 +114,7 @@ public class S3DataSegmentFinderTest @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - RestS3Service mockS3Client; + MockAmazonS3Client mockS3Client; S3DataSegmentPusherConfig config; private String bucket; @@ -122,8 +131,6 @@ public class S3DataSegmentFinderTest private String indexZip4_0; private String indexZip4_1; - - @BeforeClass public static void setUpStatic() { @@ -140,7 +147,7 @@ public void setUp() throws Exception config.setBucket(bucket); config.setBaseKey(baseKey); - mockS3Client = new MockStorageService(temporaryFolder.newFolder()); + mockS3Client = new MockAmazonS3Client(temporaryFolder.newFolder()); descriptor1 = S3Utils.descriptorPathForSegmentPath(baseKey + "/interval1/v1/0/"); @@ -154,17 +161,17 @@ public void setUp() throws Exception indexZip4_0 = S3Utils.indexZipForSegmentPath(descriptor4_0); indexZip4_1 = S3Utils.indexZipForSegmentPath(descriptor4_1); - mockS3Client.putObject(bucket, new S3Object(descriptor1, mapper.writeValueAsString(SEGMENT_1))); - mockS3Client.putObject(bucket, new S3Object(descriptor2, mapper.writeValueAsString(SEGMENT_2))); - mockS3Client.putObject(bucket, new S3Object(descriptor3, mapper.writeValueAsString(SEGMENT_3))); - mockS3Client.putObject(bucket, new S3Object(descriptor4_0, mapper.writeValueAsString(SEGMENT_4_0))); - mockS3Client.putObject(bucket, new S3Object(descriptor4_1, mapper.writeValueAsString(SEGMENT_4_1))); - - mockS3Client.putObject(bucket, new S3Object(indexZip1, "dummy")); - mockS3Client.putObject(bucket, new S3Object(indexZip2, "dummy")); - mockS3Client.putObject(bucket, new S3Object(indexZip3, "dummy")); - mockS3Client.putObject(bucket, new S3Object(indexZip4_0, "dummy")); - mockS3Client.putObject(bucket, new S3Object(indexZip4_1, "dummy")); + mockS3Client.putObject(bucket, descriptor1, mapper.writeValueAsString(SEGMENT_1)); + mockS3Client.putObject(bucket, descriptor2, mapper.writeValueAsString(SEGMENT_2)); + mockS3Client.putObject(bucket, descriptor3, mapper.writeValueAsString(SEGMENT_3)); + mockS3Client.putObject(bucket, descriptor4_0, mapper.writeValueAsString(SEGMENT_4_0)); + mockS3Client.putObject(bucket, descriptor4_1, mapper.writeValueAsString(SEGMENT_4_1)); + + mockS3Client.putObject(bucket, indexZip1, "dummy"); + mockS3Client.putObject(bucket, indexZip2, "dummy"); + mockS3Client.putObject(bucket, indexZip3, "dummy"); + mockS3Client.putObject(bucket, indexZip4_0, "dummy"); + mockS3Client.putObject(bucket, indexZip4_1, "dummy"); } @Test @@ -210,34 +217,34 @@ public void testFindSegments() throws Exception final String serializedSegment4_1 = mapper.writeValueAsString(updatedSegment4_1); Assert.assertNotEquals(serializedSegment1, - IOUtils.toString(mockS3Client.getObject(bucket, descriptor1).getDataInputStream())); + IOUtils.toString(mockS3Client.getObject(bucket, descriptor1).getObjectContent())); Assert.assertNotEquals(serializedSegment2, - IOUtils.toString(mockS3Client.getObject(bucket, descriptor2).getDataInputStream())); + IOUtils.toString(mockS3Client.getObject(bucket, descriptor2).getObjectContent())); Assert.assertNotEquals(serializedSegment3, - IOUtils.toString(mockS3Client.getObject(bucket, descriptor3).getDataInputStream())); + IOUtils.toString(mockS3Client.getObject(bucket, descriptor3).getObjectContent())); Assert.assertNotEquals(serializedSegment4_0, - IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_0).getDataInputStream())); + IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_0).getObjectContent())); Assert.assertNotEquals(serializedSegment4_1, - IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_1).getDataInputStream())); + IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_1).getObjectContent())); final Set segments2 = s3DataSegmentFinder.findSegments("", true); Assert.assertEquals(segments, segments2); Assert.assertEquals(serializedSegment1, - IOUtils.toString(mockS3Client.getObject(bucket, descriptor1).getDataInputStream())); + IOUtils.toString(mockS3Client.getObject(bucket, descriptor1).getObjectContent())); Assert.assertEquals(serializedSegment2, - IOUtils.toString(mockS3Client.getObject(bucket, descriptor2).getDataInputStream())); + IOUtils.toString(mockS3Client.getObject(bucket, descriptor2).getObjectContent())); Assert.assertEquals(serializedSegment3, - IOUtils.toString(mockS3Client.getObject(bucket, descriptor3).getDataInputStream())); + IOUtils.toString(mockS3Client.getObject(bucket, descriptor3).getObjectContent())); Assert.assertEquals(serializedSegment4_0, - IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_0).getDataInputStream())); + IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_0).getObjectContent())); Assert.assertEquals(serializedSegment4_1, - IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_1).getDataInputStream())); + IOUtils.toString(mockS3Client.getObject(bucket, descriptor4_1).getObjectContent())); } @Test(expected = SegmentLoadingException.class) - public void testFindSegmentsFail() throws SegmentLoadingException, ServiceException + public void testFindSegmentsFail() throws SegmentLoadingException { mockS3Client.deleteObject(bucket, indexZip4_1); @@ -275,21 +282,8 @@ public void testFindSegmentsUpdateLoadSpec() throws Exception final String descriptorPath = S3Utils.descriptorPathForSegmentPath(segmentPath); final String indexPath = S3Utils.indexZipForSegmentPath(segmentPath); - mockS3Client.putObject( - config.getBucket(), - new S3Object( - descriptorPath, - mapper.writeValueAsString(segmentMissingLoadSpec) - ) - ); - - mockS3Client.putObject( - config.getBucket(), - new S3Object( - indexPath, - "dummy" - ) - ); + mockS3Client.putObject(config.getBucket(), descriptorPath, mapper.writeValueAsString(segmentMissingLoadSpec)); + mockS3Client.putObject(config.getBucket(), indexPath, "dummy"); Set segments = s3DataSegmentFinder.findSegments(segmentPath, false); Assert.assertEquals(1, segments.size()); @@ -308,24 +302,34 @@ private String getDescriptorPath(DataSegment segment) return S3Utils.descriptorPathForSegmentPath(String.valueOf(segment.getLoadSpec().get("key"))); } - private static class MockStorageService extends RestS3Service + private static class MockAmazonS3Client extends AmazonS3Client { private final File baseDir; private final Map> storage = Maps.newHashMap(); - public MockStorageService(File baseDir) + public MockAmazonS3Client(File baseDir) { - super(null); + super(); this.baseDir = baseDir; } @Override - public StorageObjectsChunk listObjectsChunked( - final String bucketName, final String prefix, final String delimiter, - final long maxListingLength, final String priorLastKey - ) throws ServiceException + public boolean doesObjectExist(String bucketName, String objectName) + { + final Set keys = storage.get(bucketName); + if (keys != null) { + return keys.contains(objectName); + } + return false; + } + + @Override + public ListObjectsV2Result listObjectsV2(ListObjectsV2Request listObjectsV2Request) { - List keysOrigin = Lists.newArrayList(storage.get(bucketName)); + final String bucketName = listObjectsV2Request.getBucketName(); + final String prefix = listObjectsV2Request.getPrefix(); + + final List keysOrigin = Lists.newArrayList(storage.get(bucketName)); Predicate prefixFilter = new Predicate() { @@ -341,11 +345,11 @@ public boolean apply(@Nullable String input) ); int startOffset = 0; - if (priorLastKey != null) { - startOffset = keys.indexOf(priorLastKey) + 1; + if (listObjectsV2Request.getContinuationToken() != null) { + startOffset = keys.indexOf(listObjectsV2Request.getContinuationToken()) + 1; } - int endOffset = startOffset + (int) maxListingLength; // exclusive + int endOffset = startOffset + listObjectsV2Request.getMaxKeys(); // exclusive if (endOffset > keys.size()) { endOffset = keys.size(); } @@ -355,64 +359,73 @@ public boolean apply(@Nullable String input) newPriorLastkey = null; } - List objects = Lists.newArrayList(); + List objects = new ArrayList<>(); for (String objectKey : keys.subList(startOffset, endOffset)) { - objects.add(getObjectDetails(bucketName, objectKey)); + final S3ObjectSummary objectSummary = new S3ObjectSummary(); + objectSummary.setBucketName(bucketName); + objectSummary.setKey(objectKey); + objects.add(objectSummary); } - return new StorageObjectsChunk( - prefix, delimiter, objects.toArray(new StorageObject[]{}), null, newPriorLastkey); + final ListObjectsV2Result result = new ListObjectsV2Result(); + result.setBucketName(bucketName); + result.setKeyCount(objects.size()); + result.getObjectSummaries().addAll(objects); + result.setContinuationToken(newPriorLastkey); + result.setTruncated(newPriorLastkey != null); + + return result; } @Override - public StorageObject getObjectDetails(String bucketName, String objectKey) throws ServiceException + public S3Object getObject(String bucketName, String objectKey) { - if (!storage.containsKey(bucketName)) { - ServiceException ex = new ServiceException(); - ex.setResponseCode(404); + AmazonServiceException ex = new AmazonS3Exception("S3DataSegmentFinderTest"); + ex.setStatusCode(404); ex.setErrorCode("NoSuchBucket"); throw ex; } if (!storage.get(bucketName).contains(objectKey)) { - ServiceException ex = new ServiceException(); - ex.setResponseCode(404); + AmazonServiceException ex = new AmazonS3Exception("S3DataSegmentFinderTest"); + ex.setStatusCode(404); ex.setErrorCode("NoSuchKey"); throw ex; } final File objectPath = new File(baseDir, objectKey); - StorageObject storageObject = new StorageObject(); + S3Object storageObject = new S3Object(); storageObject.setBucketName(bucketName); storageObject.setKey(objectKey); - storageObject.setDataInputFile(objectPath); + try { + storageObject.setObjectContent(new FileInputStream(objectPath)); + } + catch (FileNotFoundException e) { + AmazonServiceException ex = new AmazonS3Exception("S3DataSegmentFinderTest", e); + ex.setStatusCode(500); + ex.setErrorCode("InternalError"); + throw ex; + } return storageObject; } @Override - public S3Object getObject(String bucketName, String objectKey) + public PutObjectResult putObject(String bucketName, String key, String data) { - final File objectPath = new File(baseDir, objectKey); - S3Object s3Object = new S3Object(); - s3Object.setBucketName(bucketName); - s3Object.setKey(objectKey); - s3Object.setDataInputFile(objectPath); - - return s3Object; - + return putObject(bucketName, key, new ByteArrayInputStream(StringUtils.toUtf8(data)), null); } @Override - public S3Object putObject(final String bucketName, final S3Object object) + public PutObjectResult putObject(String bucketName, String key, InputStream input, ObjectMetadata metadata) { if (!storage.containsKey(bucketName)) { - storage.put(bucketName, Sets.newHashSet()); + storage.put(bucketName, Sets.newHashSet()); } - storage.get(bucketName).add(object.getKey()); + storage.get(bucketName).add(key); - final File objectPath = new File(baseDir, object.getKey()); + final File objectPath = new File(baseDir, key); if (!objectPath.getParentFile().exists()) { objectPath.getParentFile().mkdirs(); @@ -420,7 +433,7 @@ public S3Object putObject(final String bucketName, final S3Object object) try { try ( - InputStream in = object.getDataInputStream() + InputStream in = input ) { FileUtils.copyInputStreamToFile(in, objectPath); } @@ -429,7 +442,7 @@ public S3Object putObject(final String bucketName, final S3Object object) throw Throwables.propagate(e); } - return object; + return new PutObjectResult(); } @Override diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java index 95ddcdfb7dd9..a848eb180822 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java @@ -19,6 +19,22 @@ package io.druid.storage.s3; +import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.model.AccessControlList; +import com.amazonaws.services.s3.model.AmazonS3Exception; +import com.amazonaws.services.s3.model.CanonicalGrantee; +import com.amazonaws.services.s3.model.CopyObjectRequest; +import com.amazonaws.services.s3.model.CopyObjectResult; +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import com.amazonaws.services.s3.model.Grant; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.Owner; +import com.amazonaws.services.s3.model.Permission; +import com.amazonaws.services.s3.model.PutObjectResult; +import com.amazonaws.services.s3.model.S3ObjectSummary; +import com.amazonaws.services.s3.model.StorageClass; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; @@ -28,12 +44,11 @@ import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; -import org.jets3t.service.model.StorageObject; import org.junit.Assert; import org.junit.Test; +import java.io.File; +import java.util.HashSet; import java.util.Map; import java.util.Set; @@ -59,11 +74,17 @@ public class S3DataSegmentMoverTest @Test public void testMove() throws Exception { - MockStorageService mockS3Client = new MockStorageService(); + MockAmazonS3Client mockS3Client = new MockAmazonS3Client(); S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client, new S3DataSegmentPusherConfig()); - mockS3Client.putObject("main", new S3Object("baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip")); - mockS3Client.putObject("main", new S3Object("baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/descriptor.json")); + mockS3Client.putObject( + "main", + "baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip" + ); + mockS3Client.putObject( + "main", + "baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/descriptor.json" + ); DataSegment movedSegment = mover.move( sourceSegment, @@ -79,11 +100,17 @@ public void testMove() throws Exception @Test public void testMoveNoop() throws Exception { - MockStorageService mockS3Client = new MockStorageService(); + MockAmazonS3Client mockS3Client = new MockAmazonS3Client(); S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client, new S3DataSegmentPusherConfig()); - mockS3Client.putObject("archive", new S3Object("targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip")); - mockS3Client.putObject("archive", new S3Object("targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/descriptor.json")); + mockS3Client.putObject( + "archive", + "targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip" + ); + mockS3Client.putObject( + "archive", + "targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/descriptor.json" + ); DataSegment movedSegment = mover.move( sourceSegment, @@ -100,7 +127,7 @@ public void testMoveNoop() throws Exception @Test(expected = SegmentLoadingException.class) public void testMoveException() throws Exception { - MockStorageService mockS3Client = new MockStorageService(); + MockAmazonS3Client mockS3Client = new MockAmazonS3Client(); S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client, new S3DataSegmentPusherConfig()); mover.move( @@ -112,7 +139,7 @@ public void testMoveException() throws Exception @Test public void testIgnoresGoneButAlreadyMoved() throws Exception { - MockStorageService mockS3Client = new MockStorageService(); + MockAmazonS3Client mockS3Client = new MockAmazonS3Client(); S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client, new S3DataSegmentPusherConfig()); mover.move(new DataSegment( "test", @@ -135,7 +162,7 @@ public void testIgnoresGoneButAlreadyMoved() throws Exception @Test(expected = SegmentLoadingException.class) public void testFailsToMoveMissing() throws Exception { - MockStorageService mockS3Client = new MockStorageService(); + MockAmazonS3Client mockS3Client = new MockAmazonS3Client(); S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client, new S3DataSegmentPusherConfig()); mover.move(new DataSegment( "test", @@ -155,15 +182,15 @@ public void testFailsToMoveMissing() throws Exception ), ImmutableMap.of("bucket", "DOES NOT EXIST", "baseKey", "baseKey2")); } - private static class MockStorageService extends RestS3Service + private static class MockAmazonS3Client extends AmazonS3Client { Map> storage = Maps.newHashMap(); boolean copied = false; boolean deletedOld = false; - private MockStorageService() + private MockAmazonS3Client() { - super(null); + super(); } public boolean didMove() @@ -172,37 +199,68 @@ public boolean didMove() } @Override - public boolean isObjectInBucket(String bucketName, String objectKey) + public AccessControlList getBucketAcl(String bucketName) + { + final AccessControlList acl = new AccessControlList(); + acl.setOwner(new Owner("ownerId", "owner")); + acl.grantAllPermissions(new Grant(new CanonicalGrantee(acl.getOwner().getId()), Permission.FullControl)); + return acl; + } + + @Override + public ObjectMetadata getObjectMetadata(GetObjectMetadataRequest getObjectMetadataRequest) + { + return new ObjectMetadata(); + } + + @Override + public boolean doesObjectExist(String bucketName, String objectKey) { Set objects = storage.get(bucketName); return (objects != null && objects.contains(objectKey)); } @Override - public S3Object[] listObjects(String bucketName, String objectKey, String separator) + public ListObjectsV2Result listObjectsV2(ListObjectsV2Request listObjectsV2Request) { - if (isObjectInBucket(bucketName, objectKey)) { - final S3Object object = new S3Object(objectKey); - object.setStorageClass(S3Object.STORAGE_CLASS_STANDARD); - return new S3Object[]{object}; + final String bucketName = listObjectsV2Request.getBucketName(); + final String objectKey = listObjectsV2Request.getPrefix(); + if (doesObjectExist(bucketName, objectKey)) { + final S3ObjectSummary objectSummary = new S3ObjectSummary(); + objectSummary.setBucketName(bucketName); + objectSummary.setKey(objectKey); + objectSummary.setStorageClass(StorageClass.Standard.name()); + + final ListObjectsV2Result result = new ListObjectsV2Result(); + result.setBucketName(bucketName); + result.setPrefix(objectKey); + result.setKeyCount(1); + result.getObjectSummaries().add(objectSummary); + result.setTruncated(true); + return result; + } else { + return new ListObjectsV2Result(); } - return new S3Object[]{}; } @Override - public Map copyObject( - String sourceBucketName, - String sourceObjectKey, - String destinationBucketName, - StorageObject destinationObject, - boolean replaceMetadata - ) + public CopyObjectResult copyObject(CopyObjectRequest copyObjectRequest) { + final String sourceBucketName = copyObjectRequest.getSourceBucketName(); + final String sourceObjectKey = copyObjectRequest.getSourceKey(); + final String destinationBucketName = copyObjectRequest.getDestinationBucketName(); + final String destinationObjectKey = copyObjectRequest.getDestinationKey(); copied = true; - if (isObjectInBucket(sourceBucketName, sourceObjectKey)) { - this.putObject(destinationBucketName, new S3Object(destinationObject.getKey())); + if (doesObjectExist(sourceBucketName, sourceObjectKey)) { + storage.computeIfAbsent(destinationBucketName, k -> new HashSet<>()) + .add(destinationObjectKey); + return new CopyObjectResult(); + } else { + final AmazonS3Exception exception = new AmazonS3Exception("S3DataSegmentMoverTest"); + exception.setErrorCode("NoSuchKey"); + exception.setStatusCode(404); + throw exception; } - return null; } @Override @@ -212,14 +270,19 @@ public void deleteObject(String bucket, String objectKey) storage.get(bucket).remove(objectKey); } + public PutObjectResult putObject(String bucketName, String key) + { + return putObject(bucketName, key, (File) null); + } + @Override - public S3Object putObject(String bucketName, S3Object object) + public PutObjectResult putObject(String bucketName, String key, File file) { if (!storage.containsKey(bucketName)) { storage.put(bucketName, Sets.newHashSet()); } - storage.get(bucketName).add(object.getKey()); - return object; + storage.get(bucketName).add(key); + return new PutObjectResult(); } } } diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPullerTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPullerTest.java index 303bf657d6e5..8bc028a64f0a 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPullerTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPullerTest.java @@ -19,9 +19,21 @@ package io.druid.storage.s3; +import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.model.AmazonS3Exception; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.S3Object; +import com.amazonaws.services.s3.model.S3ObjectSummary; import io.druid.java.util.common.FileUtils; import io.druid.java.util.common.StringUtils; import io.druid.segment.loading.SegmentLoadingException; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; @@ -30,15 +42,6 @@ import java.net.URI; import java.util.Date; import java.util.zip.GZIPOutputStream; -import org.easymock.EasyMock; -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; /** * @@ -50,26 +53,29 @@ public class S3DataSegmentPullerTest public TemporaryFolder temporaryFolder = new TemporaryFolder(); @Test - public void testSimpleGetVersion() throws ServiceException, IOException + public void testSimpleGetVersion() throws IOException { String bucket = "bucket"; String keyPrefix = "prefix/dir/0"; - RestS3Service s3Client = EasyMock.createStrictMock(RestS3Service.class); + AmazonS3Client s3Client = EasyMock.createStrictMock(AmazonS3Client.class); - S3Object object0 = new S3Object(); + final S3ObjectSummary objectSummary = new S3ObjectSummary(); + objectSummary.setBucketName(bucket); + objectSummary.setKey(keyPrefix + "/renames-0.gz"); + objectSummary.setLastModified(new Date(0)); - object0.setBucketName(bucket); - object0.setKey(keyPrefix + "/renames-0.gz"); - object0.setLastModifiedDate(new Date(0)); + final ListObjectsV2Result result = new ListObjectsV2Result(); + result.setKeyCount(1); + result.getObjectSummaries().add(objectSummary); - EasyMock.expect(s3Client.getObjectDetails(EasyMock.eq(bucket), EasyMock.eq(object0.getKey()))) - .andReturn(object0) + EasyMock.expect(s3Client.listObjectsV2(EasyMock.anyObject(ListObjectsV2Request.class))) + .andReturn(result) .once(); S3DataSegmentPuller puller = new S3DataSegmentPuller(s3Client); EasyMock.replay(s3Client); - String version = puller.getVersion(URI.create(StringUtils.format("s3://%s/%s", bucket, object0.getKey()))); + String version = puller.getVersion(URI.create(StringUtils.format("s3://%s/%s", bucket, objectSummary.getKey()))); EasyMock.verify(s3Client); @@ -77,11 +83,11 @@ public void testSimpleGetVersion() throws ServiceException, IOException } @Test - public void testGZUncompress() throws ServiceException, IOException, SegmentLoadingException + public void testGZUncompress() throws IOException, SegmentLoadingException { final String bucket = "bucket"; final String keyPrefix = "prefix/dir/0"; - final RestS3Service s3Client = EasyMock.createStrictMock(RestS3Service.class); + final AmazonS3Client s3Client = EasyMock.createStrictMock(AmazonS3Client.class); final byte[] value = bucket.getBytes("utf8"); final File tmpFile = temporaryFolder.newFile("gzTest.gz"); @@ -91,19 +97,27 @@ public void testGZUncompress() throws ServiceException, IOException, SegmentLoad } final S3Object object0 = new S3Object(); - object0.setBucketName(bucket); object0.setKey(keyPrefix + "/renames-0.gz"); - object0.setLastModifiedDate(new Date(0)); - object0.setDataInputStream(new FileInputStream(tmpFile)); + object0.getObjectMetadata().setLastModified(new Date(0)); + object0.setObjectContent(new FileInputStream(tmpFile)); + + final S3ObjectSummary objectSummary = new S3ObjectSummary(); + objectSummary.setBucketName(bucket); + objectSummary.setKey(keyPrefix + "/renames-0.gz"); + objectSummary.setLastModified(new Date(0)); + + final ListObjectsV2Result listObjectsResult = new ListObjectsV2Result(); + listObjectsResult.setKeyCount(1); + listObjectsResult.getObjectSummaries().add(objectSummary); final File tmpDir = temporaryFolder.newFolder("gzTestDir"); - EasyMock.expect(s3Client.getObjectDetails(EasyMock.eq(object0.getBucketName()), EasyMock.eq(object0.getKey()))) - .andReturn(null) + EasyMock.expect(s3Client.doesObjectExist(EasyMock.eq(object0.getBucketName()), EasyMock.eq(object0.getKey()))) + .andReturn(true) .once(); - EasyMock.expect(s3Client.getObjectDetails(EasyMock.eq(object0.getBucketName()), EasyMock.eq(object0.getKey()))) - .andReturn(object0) + EasyMock.expect(s3Client.listObjectsV2(EasyMock.anyObject(ListObjectsV2Request.class))) + .andReturn(listObjectsResult) .once(); EasyMock.expect(s3Client.getObject(EasyMock.eq(object0.getBucketName()), EasyMock.eq(object0.getKey()))) .andReturn(object0) @@ -126,11 +140,11 @@ public void testGZUncompress() throws ServiceException, IOException, SegmentLoad } @Test - public void testGZUncompressRetries() throws ServiceException, IOException, SegmentLoadingException + public void testGZUncompressRetries() throws IOException, SegmentLoadingException { final String bucket = "bucket"; final String keyPrefix = "prefix/dir/0"; - final RestS3Service s3Client = EasyMock.createStrictMock(RestS3Service.class); + final AmazonS3Client s3Client = EasyMock.createStrictMock(AmazonS3Client.class); final byte[] value = bucket.getBytes("utf8"); final File tmpFile = temporaryFolder.newFile("gzTest.gz"); @@ -143,25 +157,34 @@ public void testGZUncompressRetries() throws ServiceException, IOException, Segm object0.setBucketName(bucket); object0.setKey(keyPrefix + "/renames-0.gz"); - object0.setLastModifiedDate(new Date(0)); - object0.setDataInputStream(new FileInputStream(tmpFile)); + object0.getObjectMetadata().setLastModified(new Date(0)); + object0.setObjectContent(new FileInputStream(tmpFile)); + + final S3ObjectSummary objectSummary = new S3ObjectSummary(); + objectSummary.setBucketName(bucket); + objectSummary.setKey(keyPrefix + "/renames-0.gz"); + objectSummary.setLastModified(new Date(0)); + + final ListObjectsV2Result listObjectsResult = new ListObjectsV2Result(); + listObjectsResult.setKeyCount(1); + listObjectsResult.getObjectSummaries().add(objectSummary); File tmpDir = temporaryFolder.newFolder("gzTestDir"); - S3ServiceException exception = new S3ServiceException(); + AmazonS3Exception exception = new AmazonS3Exception("S3DataSegmentPullerTest"); exception.setErrorCode("NoSuchKey"); - exception.setResponseCode(404); - EasyMock.expect(s3Client.getObjectDetails(EasyMock.eq(object0.getBucketName()), EasyMock.eq(object0.getKey()))) - .andReturn(null) + exception.setStatusCode(404); + EasyMock.expect(s3Client.doesObjectExist(EasyMock.eq(object0.getBucketName()), EasyMock.eq(object0.getKey()))) + .andReturn(true) .once(); - EasyMock.expect(s3Client.getObjectDetails(EasyMock.eq(object0.getBucketName()), EasyMock.eq(object0.getKey()))) - .andReturn(object0) + EasyMock.expect(s3Client.listObjectsV2(EasyMock.anyObject(ListObjectsV2Request.class))) + .andReturn(listObjectsResult) .once(); EasyMock.expect(s3Client.getObject(EasyMock.eq(bucket), EasyMock.eq(object0.getKey()))) .andThrow(exception) .once(); - EasyMock.expect(s3Client.getObjectDetails(EasyMock.eq(object0.getBucketName()), EasyMock.eq(object0.getKey()))) - .andReturn(object0) + EasyMock.expect(s3Client.listObjectsV2(EasyMock.anyObject(ListObjectsV2Request.class))) + .andReturn(listObjectsResult) .once(); EasyMock.expect(s3Client.getObject(EasyMock.eq(bucket), EasyMock.eq(object0.getKey()))) .andReturn(object0) diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherTest.java index f26bd1610398..c787ed7d900c 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherTest.java @@ -19,6 +19,14 @@ package io.druid.storage.s3; +import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.model.AccessControlList; +import com.amazonaws.services.s3.model.CanonicalGrantee; +import com.amazonaws.services.s3.model.Grant; +import com.amazonaws.services.s3.model.Owner; +import com.amazonaws.services.s3.model.Permission; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.PutObjectResult; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -31,14 +39,13 @@ import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.IAnswer; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import java.io.File; +import java.io.FileInputStream; /** */ @@ -65,27 +72,38 @@ public void setValue(T value) @Test public void testPush() throws Exception { - RestS3Service s3Client = EasyMock.createStrictMock(RestS3Service.class); + AmazonS3Client s3Client = EasyMock.createStrictMock(AmazonS3Client.class); - Capture capturedS3Object = Capture.newInstance(); + final AccessControlList acl = new AccessControlList(); + acl.setOwner(new Owner("ownerId", "owner")); + acl.grantAllPermissions(new Grant(new CanonicalGrantee(acl.getOwner().getId()), Permission.FullControl)); + EasyMock.expect(s3Client.getBucketAcl(EasyMock.eq("bucket"))).andReturn(acl).once(); + + EasyMock.expect(s3Client.putObject(EasyMock.anyObject())) + .andReturn(new PutObjectResult()) + .once(); + + EasyMock.expect(s3Client.getBucketAcl(EasyMock.eq("bucket"))).andReturn(acl).once(); + + Capture capturedPutRequest = Capture.newInstance(); ValueContainer capturedS3SegmentJson = new ValueContainer<>(); - EasyMock.expect(s3Client.putObject(EasyMock.anyString(), EasyMock.capture(capturedS3Object))) + EasyMock.expect(s3Client.putObject(EasyMock.capture(capturedPutRequest))) .andAnswer( - new IAnswer() + new IAnswer() { @Override - public S3Object answer() throws Throwable + public PutObjectResult answer() throws Throwable { capturedS3SegmentJson.setValue( - IOUtils.toString(capturedS3Object.getValue().getDataInputStream(), "utf-8") + IOUtils.toString(new FileInputStream(capturedPutRequest.getValue().getFile()), "utf-8") ); - return null; + return new PutObjectResult(); } } ) - .atLeastOnce(); - EasyMock.replay(s3Client); + .once(); + EasyMock.replay(s3Client); S3DataSegmentPusherConfig config = new S3DataSegmentPusherConfig(); config.setBucket("bucket"); diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3TimestampVersionedDataFinderTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3TimestampVersionedDataFinderTest.java index c2bdc347334d..ce19c9b10fc7 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3TimestampVersionedDataFinderTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3TimestampVersionedDataFinderTest.java @@ -19,11 +19,12 @@ package io.druid.storage.s3; +import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.S3ObjectSummary; import io.druid.java.util.common.StringUtils; import org.easymock.EasyMock; -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; import org.junit.Assert; import org.junit.Test; @@ -35,25 +36,31 @@ public class S3TimestampVersionedDataFinderTest { @Test - public void testSimpleLatestVersion() throws S3ServiceException + public void testSimpleLatestVersion() { String bucket = "bucket"; String keyPrefix = "prefix/dir/0"; - RestS3Service s3Client = EasyMock.createStrictMock(RestS3Service.class); + AmazonS3Client s3Client = EasyMock.createStrictMock(AmazonS3Client.class); - S3Object object0 = new S3Object(), object1 = new S3Object(); + S3ObjectSummary object0 = new S3ObjectSummary(), object1 = new S3ObjectSummary(); object0.setBucketName(bucket); object0.setKey(keyPrefix + "/renames-0.gz"); - object0.setLastModifiedDate(new Date(0)); + object0.setLastModified(new Date(0)); object1.setBucketName(bucket); object1.setKey(keyPrefix + "/renames-1.gz"); - object1.setLastModifiedDate(new Date(1)); + object1.setLastModified(new Date(1)); - EasyMock.expect(s3Client.listObjects(EasyMock.eq(bucket), EasyMock.anyString(), EasyMock.isNull())).andReturn( - new S3Object[]{object0, object1} - ).once(); + final ListObjectsV2Result result = new ListObjectsV2Result(); + result.getObjectSummaries().add(object0); + result.getObjectSummaries().add(object1); + result.setKeyCount(2); + result.setTruncated(false); + + EasyMock.expect(s3Client.listObjectsV2(EasyMock.anyObject(ListObjectsV2Request.class))) + .andReturn(result) + .once(); S3TimestampVersionedDataFinder finder = new S3TimestampVersionedDataFinder(s3Client); Pattern pattern = Pattern.compile("renames-[0-9]*\\.gz"); @@ -71,25 +78,19 @@ public void testSimpleLatestVersion() throws S3ServiceException } @Test - public void testMissing() throws S3ServiceException + public void testMissing() { String bucket = "bucket"; String keyPrefix = "prefix/dir/0"; - RestS3Service s3Client = EasyMock.createStrictMock(RestS3Service.class); + AmazonS3Client s3Client = EasyMock.createStrictMock(AmazonS3Client.class); - S3Object object0 = new S3Object(), object1 = new S3Object(); + final ListObjectsV2Result result = new ListObjectsV2Result(); + result.setKeyCount(0); + result.setTruncated(false); - object0.setBucketName(bucket); - object0.setKey(keyPrefix + "/renames-0.gz"); - object0.setLastModifiedDate(new Date(0)); - - object1.setBucketName(bucket); - object1.setKey(keyPrefix + "/renames-1.gz"); - object1.setLastModifiedDate(new Date(1)); - - EasyMock.expect(s3Client.listObjects(EasyMock.eq(bucket), EasyMock.anyString(), EasyMock.isNull())).andReturn( - null - ).once(); + EasyMock.expect(s3Client.listObjectsV2(EasyMock.anyObject(ListObjectsV2Request.class))) + .andReturn(result) + .once(); S3TimestampVersionedDataFinder finder = new S3TimestampVersionedDataFinder(s3Client); Pattern pattern = Pattern.compile("renames-[0-9]*\\.gz"); @@ -105,21 +106,26 @@ public void testMissing() throws S3ServiceException } @Test - public void testFindSelf() throws S3ServiceException + public void testFindSelf() { String bucket = "bucket"; String keyPrefix = "prefix/dir/0"; - RestS3Service s3Client = EasyMock.createStrictMock(RestS3Service.class); + AmazonS3Client s3Client = EasyMock.createStrictMock(AmazonS3Client.class); - S3Object object0 = new S3Object(); + S3ObjectSummary object0 = new S3ObjectSummary(); object0.setBucketName(bucket); object0.setKey(keyPrefix + "/renames-0.gz"); - object0.setLastModifiedDate(new Date(0)); + object0.setLastModified(new Date(0)); - EasyMock.expect(s3Client.listObjects(EasyMock.eq(bucket), EasyMock.anyString(), EasyMock.isNull())).andReturn( - new S3Object[]{object0} - ).once(); + final ListObjectsV2Result result = new ListObjectsV2Result(); + result.getObjectSummaries().add(object0); + result.setKeyCount(1); + result.setTruncated(false); + + EasyMock.expect(s3Client.listObjectsV2(EasyMock.anyObject(ListObjectsV2Request.class))) + .andReturn(result) + .once(); S3TimestampVersionedDataFinder finder = new S3TimestampVersionedDataFinder(s3Client); Pattern pattern = Pattern.compile("renames-[0-9]*\\.gz"); @@ -137,21 +143,26 @@ public void testFindSelf() throws S3ServiceException } @Test - public void testFindExact() throws S3ServiceException + public void testFindExact() { String bucket = "bucket"; String keyPrefix = "prefix/dir/0"; - RestS3Service s3Client = EasyMock.createStrictMock(RestS3Service.class); + AmazonS3Client s3Client = EasyMock.createStrictMock(AmazonS3Client.class); - S3Object object0 = new S3Object(); + S3ObjectSummary object0 = new S3ObjectSummary(); object0.setBucketName(bucket); object0.setKey(keyPrefix + "/renames-0.gz"); - object0.setLastModifiedDate(new Date(0)); + object0.setLastModified(new Date(0)); + + final ListObjectsV2Result result = new ListObjectsV2Result(); + result.getObjectSummaries().add(object0); + result.setKeyCount(1); + result.setTruncated(false); - EasyMock.expect(s3Client.listObjects(EasyMock.eq(bucket), EasyMock.anyString(), EasyMock.isNull())).andReturn( - new S3Object[]{object0} - ).once(); + EasyMock.expect(s3Client.listObjectsV2(EasyMock.anyObject(ListObjectsV2Request.class))) + .andReturn(result) + .once(); S3TimestampVersionedDataFinder finder = new S3TimestampVersionedDataFinder(s3Client); diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java index a3bf27a40b47..a7716e2a5ed8 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java @@ -24,6 +24,8 @@ import com.amazonaws.auth.AWSSessionCredentials; import com.google.common.io.Files; import io.druid.common.aws.AWSCredentialsConfig; +import io.druid.common.aws.AWSEndpointConfig; +import io.druid.common.aws.AWSProxyConfig; import io.druid.guice.AWSModule; import io.druid.metadata.DefaultPasswordProvider; import org.easymock.EasyMock; @@ -58,7 +60,7 @@ public void testWithFixedAWSKeys() assertEquals(credentials.getAWSSecretKey(), "secretKeySample"); // try to create - s3Module.getRestS3Service(provider); + s3Module.getAmazonS3Client(provider, new AWSProxyConfig(), new AWSEndpointConfig()); } @Rule @@ -86,6 +88,6 @@ public void testWithFileSessionCredentials() throws IOException assertEquals(sessionCredentials.getSessionToken(), "sessionTokenSample"); // try to create - s3Module.getRestS3Service(provider); + s3Module.getAmazonS3Client(provider, new AWSProxyConfig(), new AWSEndpointConfig()); } } diff --git a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/StandardDeviationPostAggregator.java b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/StandardDeviationPostAggregator.java index 443d0126cb7f..f7034844d818 100644 --- a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/StandardDeviationPostAggregator.java +++ b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/StandardDeviationPostAggregator.java @@ -107,7 +107,8 @@ public String toString() return "StandardDeviationPostAggregator{" + "name='" + name + '\'' + ", fieldName='" + fieldName + '\'' + - ", isVariancePop='" + isVariancePop + '\'' + + ", estimator='" + estimator + '\'' + + ", isVariancePop=" + isVariancePop + '}'; } @@ -116,6 +117,7 @@ public byte[] getCacheKey() { return new CacheKeyBuilder(PostAggregatorIds.VARIANCE_STANDARD_DEVIATION) .appendString(fieldName) + .appendString(estimator) .appendBoolean(isVariancePop) .build(); } diff --git a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorCollector.java b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorCollector.java index 3e98c32dc2f1..2472d2ca3ae2 100644 --- a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorCollector.java +++ b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorCollector.java @@ -59,20 +59,15 @@ public static VarianceAggregatorCollector from(ByteBuffer buffer) return new VarianceAggregatorCollector(buffer.getLong(), buffer.getDouble(), buffer.getDouble()); } - public static final Comparator COMPARATOR = new Comparator() - { - @Override - public int compare(VarianceAggregatorCollector o1, VarianceAggregatorCollector o2) - { - int compare = Longs.compare(o1.count, o2.count); + public static final Comparator COMPARATOR = (o1, o2) -> { + int compare = Longs.compare(o1.count, o2.count); + if (compare == 0) { + compare = Doubles.compare(o1.sum, o2.sum); if (compare == 0) { - compare = Doubles.compare(o1.sum, o2.sum); - if (compare == 0) { - compare = Doubles.compare(o1.nvariance, o2.nvariance); - } + compare = Doubles.compare(o1.nvariance, o2.nvariance); } - return compare; } + return compare; }; void fold(@Nullable VarianceAggregatorCollector other) @@ -114,13 +109,6 @@ public VarianceAggregatorCollector() this(0, 0, 0); } - public void reset() - { - count = 0; - sum = 0; - nvariance = 0; - } - void copyFrom(VarianceAggregatorCollector other) { this.count = other.count; diff --git a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java index aa8684478352..86657977e820 100644 --- a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java +++ b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java @@ -34,13 +34,13 @@ import io.druid.query.aggregation.NoopAggregator; import io.druid.query.aggregation.NoopBufferAggregator; import io.druid.query.aggregation.ObjectAggregateCombiner; +import io.druid.query.cache.CacheKeyBuilder; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ColumnValueSelector; import io.druid.segment.NilColumnValueSelector; import org.apache.commons.codec.binary.Base64; import java.nio.ByteBuffer; -import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -184,7 +184,7 @@ public AggregatorFactory getCombiningFactory() @Override public List getRequiredColumns() { - return Arrays.asList(new VarianceAggregatorFactory(fieldName, fieldName, estimator, inputType)); + return Collections.singletonList(new VarianceAggregatorFactory(fieldName, fieldName, estimator, inputType)); } @Override @@ -258,25 +258,23 @@ public List requiredFields() @Override public byte[] getCacheKey() { - byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); - byte[] inputTypeBytes = StringUtils.toUtf8(inputType); - return ByteBuffer.allocate(2 + fieldNameBytes.length + 1 + inputTypeBytes.length) - .put(AggregatorUtil.VARIANCE_CACHE_TYPE_ID) - .put(isVariancePop ? (byte) 1 : 0) - .put(fieldNameBytes) - .put((byte) 0xFF) - .put(inputTypeBytes) - .array(); + return new CacheKeyBuilder(AggregatorUtil.VARIANCE_CACHE_TYPE_ID) + .appendString(fieldName) + .appendString(inputType) + .appendBoolean(isVariancePop) + .appendString(estimator) + .build(); } @Override public String toString() { - return getClass().getSimpleName() + "{" + + return "VarianceAggregatorFactory{" + "fieldName='" + fieldName + '\'' + ", name='" + name + '\'' + - ", isVariancePop='" + isVariancePop + '\'' + + ", estimator='" + estimator + '\'' + ", inputType='" + inputType + '\'' + + ", isVariancePop=" + isVariancePop + '}'; } @@ -289,29 +287,18 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - VarianceAggregatorFactory that = (VarianceAggregatorFactory) o; - - if (!Objects.equals(name, that.name)) { - return false; - } - if (!Objects.equals(isVariancePop, that.isVariancePop)) { - return false; - } - if (!Objects.equals(inputType, that.inputType)) { - return false; - } - - return true; + return isVariancePop == that.isVariancePop && + Objects.equals(fieldName, that.fieldName) && + Objects.equals(name, that.name) && + Objects.equals(estimator, that.estimator) && + Objects.equals(inputType, that.inputType); } @Override public int hashCode() { - int result = fieldName.hashCode(); - result = 31 * result + Objects.hashCode(name); - result = 31 * result + Objects.hashCode(isVariancePop); - result = 31 * result + Objects.hashCode(inputType); - return result; + + return Objects.hash(fieldName, name, estimator, inputType, isVariancePop); } } diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index e2bf002c3d55..3a9c7740a033 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -56,23 +56,6 @@ com.google.guava guava - - - net.java.dev.jets3t - jets3t - test - - - - org.apache.httpcomponents - httpclient - test - - - org.apache.httpcomponents - httpcore - test - org.apache.hadoop hadoop-client @@ -100,6 +83,21 @@ + + com.amazonaws + aws-java-sdk-bundle + test + + + org.apache.httpcomponents + httpclient + test + + + org.apache.httpcomponents + httpcore + test + junit junit 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 9a6c5d9ecc96..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 @@ -395,7 +440,7 @@ public static class DetermineHashedPartitionsPartitioner public int getPartition(LongWritable interval, BytesWritable text, int numPartitions) { - if (config.get("mapred.job.tracker").equals("local") || determineIntervals) { + if ("local".equals(config.get("mapred.job.tracker")) || determineIntervals) { return 0; } else { return reducerLookup.get(interval); 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/hadoop/DatasourceInputFormatTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceInputFormatTest.java index 0617b8ca55da..1b4af099662f 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceInputFormatTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceInputFormatTest.java @@ -19,7 +19,6 @@ package io.druid.indexer.hadoop; -import com.google.common.base.Charsets; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -51,6 +50,7 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; import java.util.stream.Stream; @@ -281,7 +281,7 @@ public void testGetSplitsUsingDefaultSupplier() throws Exception { // Use the builtin supplier, reading from the local filesystem, rather than testFormatter. final File tmpFile = temporaryFolder.newFile("something:with:colons"); - Files.write("dummy", tmpFile, Charsets.UTF_8); + Files.write("dummy", tmpFile, StandardCharsets.UTF_8); final ImmutableList mySegments = ImmutableList.of( WindowedDataSegment.of( 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 new file mode 100644 index 000000000000..335b75bc3c66 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskReport.java @@ -0,0 +1,55 @@ +/* + * 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.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.collect.Maps; + +import java.util.Map; + +/** + * TaskReport objects contain additional information about an indexing task, such as row statistics, errors, and + * published segments. They are kept in deep storage along with task logs. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "ingestionStatsAndErrors", value = IngestionStatsAndErrorsTaskReport.class) +}) +public interface TaskReport +{ + String getTaskId(); + + String getReportKey(); + + /** + * @return A JSON-serializable Object that contains a TaskReport's information + */ + Object getPayload(); + + static Map buildTaskReports(TaskReport... taskReports) + { + Map taskReportMap = Maps.newHashMap(); + for (TaskReport taskReport : taskReports) { + taskReportMap.put(taskReport.getReportKey(), taskReport); + } + return taskReportMap; + } +} 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 new file mode 100644 index 000000000000..392fdc527aea --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskReportFileWriter.java @@ -0,0 +1,59 @@ +/* + * 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.databind.ObjectMapper; +import io.druid.java.util.common.logger.Logger; +import org.apache.commons.io.FileUtils; + +import java.io.File; +import java.util.Map; + +public class TaskReportFileWriter +{ + private static final Logger log = new Logger(TaskReportFileWriter.class); + + private final File reportsFile; + private ObjectMapper objectMapper; + + public TaskReportFileWriter(File reportFile) + { + this.reportsFile = reportFile; + } + + public void write(Map reports) + { + try { + final File reportsFileParent = reportsFile.getParentFile(); + if (reportsFileParent != null) { + FileUtils.forceMkdir(reportsFileParent); + } + objectMapper.writeValue(reportsFile, reports); + } + catch (Exception e) { + log.error(e, "Encountered exception in write()."); + } + } + + public void setObjectMapper(ObjectMapper objectMapper) + { + this.objectMapper = objectMapper; + } +} 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/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index dd132769192f..9deab2bc497c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -90,6 +90,7 @@ public class TaskToolbox private final Cache cache; private final CacheConfig cacheConfig; private final IndexMergerV9 indexMergerV9; + private final TaskReportFileWriter taskReportFileWriter; private final DruidNodeAnnouncer druidNodeAnnouncer; private final DruidNode druidNode; @@ -120,7 +121,8 @@ public TaskToolbox( DruidNodeAnnouncer druidNodeAnnouncer, DruidNode druidNode, LookupNodeService lookupNodeService, - DataNodeService dataNodeService + DataNodeService dataNodeService, + TaskReportFileWriter taskReportFileWriter ) { this.config = config; @@ -147,6 +149,8 @@ public TaskToolbox( this.druidNode = druidNode; this.lookupNodeService = lookupNodeService; this.dataNodeService = dataNodeService; + this.taskReportFileWriter = taskReportFileWriter; + this.taskReportFileWriter.setObjectMapper(this.objectMapper); } public TaskConfig getConfig() @@ -303,4 +307,9 @@ public DruidNode getDruidNode() { return druidNode; } + + public TaskReportFileWriter getTaskReportFileWriter() + { + return taskReportFileWriter; + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index c17b23fe210a..1a35ec040f8e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -78,6 +78,7 @@ public class TaskToolboxFactory private final DruidNode druidNode; private final LookupNodeService lookupNodeService; private final DataNodeService dataNodeService; + private final TaskReportFileWriter taskReportFileWriter; @Inject public TaskToolboxFactory( @@ -103,7 +104,8 @@ public TaskToolboxFactory( DruidNodeAnnouncer druidNodeAnnouncer, @RemoteChatHandler DruidNode druidNode, LookupNodeService lookupNodeService, - DataNodeService dataNodeService + DataNodeService dataNodeService, + TaskReportFileWriter taskReportFileWriter ) { this.config = config; @@ -129,6 +131,7 @@ public TaskToolboxFactory( this.druidNode = druidNode; this.lookupNodeService = lookupNodeService; this.dataNodeService = dataNodeService; + this.taskReportFileWriter = taskReportFileWriter; } public TaskToolbox build(Task task) @@ -158,7 +161,8 @@ public TaskToolbox build(Task task) druidNodeAnnouncer, druidNode, lookupNodeService, - dataNodeService + dataNodeService, + taskReportFileWriter ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java index 6a9370324d0a..9152c8732109 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java @@ -31,7 +31,7 @@ public class TaskConfig { public static final List DEFAULT_DEFAULT_HADOOP_COORDINATES = ImmutableList.of( - "org.apache.hadoop:hadoop-client:2.7.3" + "org.apache.hadoop:hadoop-client:2.8.3" ); private static final Period DEFAULT_DIRECTORY_LOCK_TIMEOUT = new Period("PT10M"); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index a7084f69d436..06c6069ae009 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 @@ -65,6 +65,10 @@ private static File createNewBasePersistDirectory() @Nullable private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + private final boolean logParseExceptions; + private final int maxParseExceptions; + private final int maxSavedParseExceptions; + @JsonCreator public RealtimeAppenderatorTuningConfig( @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, @@ -77,7 +81,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; @@ -100,6 +107,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 @@ -176,6 +194,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( @@ -189,7 +225,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 0bba29fbe881..5b25a3f04085 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 @@ -130,7 +130,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 3181b252544a..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,6 +396,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null } log.info("Job done!"); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); return TaskStatus.success(getId()); } @@ -386,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. @@ -403,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..85d2e333cf32 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 @@ -51,6 +51,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.Pair; +import io.druid.java.util.common.RE; import io.druid.java.util.common.granularity.NoneGranularity; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.jackson.JacksonUtils; @@ -65,6 +66,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; @@ -78,6 +80,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -102,6 +105,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 +118,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 +132,7 @@ public CompactionTask( this.tuningConfig = tuningConfig; this.jsonMapper = jsonMapper; this.segmentProvider = segments == null ? new SegmentProvider(dataSource, interval) : new SegmentProvider(segments); + this.authorizerMapper = authorizerMapper; } @JsonProperty @@ -189,26 +197,37 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception jsonMapper ); - indexTaskSpec = new IndexTask( - getId(), - getGroupId(), - getTaskResource(), - getDataSource(), - ingestionSpec, - getContext() - ); - } - - if (indexTaskSpec.getIngestionSchema() == null) { - log.info("Cannot find segments for interval"); + if (ingestionSpec != null) { + indexTaskSpec = new IndexTask( + getId(), + getGroupId(), + getTaskResource(), + getDataSource(), + ingestionSpec, + getContext(), + authorizerMapper, + null + ); + } } - final String json = jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(indexTaskSpec); - log.info("Generated compaction task details: " + json); + if (indexTaskSpec == null) { + log.warn("Failed to generate compaction spec"); + return TaskStatus.failure(getId()); + } else { + final String json = jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(indexTaskSpec); + log.info("Generated compaction task details: " + json); - return indexTaskSpec.run(toolbox); + return indexTaskSpec.run(toolbox); + } } + /** + * Generate {@link IndexIngestionSpec} from input segments. + + * @return null if input segments don't exist. Otherwise, a generated ingestionSpec. + */ + @Nullable @VisibleForTesting static IndexIngestionSpec createIngestionSchema( TaskToolbox toolbox, @@ -281,12 +300,22 @@ private static DataSchema createDataSchema( throws IOException { // find metadata for interval - final List queryableIndices = loadSegments(timelineSegments, segmentFileMap, indexIO); + final List> queryableIndexAndSegments = loadSegments( + timelineSegments, + segmentFileMap, + indexIO + ); // find merged aggregators - final List aggregatorFactories = queryableIndices + for (Pair pair : queryableIndexAndSegments) { + final QueryableIndex index = pair.lhs; + if (index.getMetadata() == null) { + throw new RE("Index metadata doesn't exist for segment[%s]", pair.rhs.getIdentifier()); + } + } + final List aggregatorFactories = queryableIndexAndSegments .stream() - .map(index -> index.getMetadata().getAggregators()) + .map(pair -> pair.lhs.getMetadata().getAggregators()) // We have already done null check on index.getMetadata() .collect(Collectors.toList()); final AggregatorFactory[] mergedAggregators = AggregatorFactory.mergeAggregators(aggregatorFactories); @@ -296,7 +325,11 @@ private static DataSchema createDataSchema( // find granularity spec // set rollup only if rollup is set for all segments - final boolean rollup = queryableIndices.stream().allMatch(index -> index.getMetadata().isRollup()); + final boolean rollup = queryableIndexAndSegments.stream().allMatch(pair -> { + // We have already checked getMetadata() doesn't return null + final Boolean isRollup = pair.lhs.getMetadata().isRollup(); + return isRollup != null && isRollup; + }); final GranularitySpec granularitySpec = new ArbitraryGranularitySpec( new NoneGranularity(), rollup, @@ -305,7 +338,7 @@ private static DataSchema createDataSchema( // find unique dimensions final DimensionsSpec finalDimensionsSpec = dimensionsSpec == null ? - createDimensionsSpec(queryableIndices) : + createDimensionsSpec(queryableIndexAndSegments) : dimensionsSpec; final InputRowParser parser = new NoopInputRowParser(new TimeAndDimsParseSpec(null, finalDimensionsSpec)); @@ -319,7 +352,7 @@ private static DataSchema createDataSchema( ); } - private static DimensionsSpec createDimensionsSpec(List queryableIndices) + private static DimensionsSpec createDimensionsSpec(List> queryableIndices) { final BiMap uniqueDims = HashBiMap.create(); final Map dimensionSchemaMap = new HashMap<>(); @@ -329,9 +362,24 @@ private static DimensionsSpec createDimensionsSpec(List queryabl // Dimensions are extracted from the recent segments to olders because recent segments are likely to be queried more // frequently, and thus the performance should be optimized for recent ones rather than old ones. - // timelineSegments are sorted in order of interval + // timelineSegments are sorted in order of interval, but we do a sanity check here. + final Comparator intervalComparator = Comparators.intervalsByStartThenEnd(); + for (int i = 0; i < queryableIndices.size() - 1; i++) { + final Interval shouldBeSmaller = queryableIndices.get(i).lhs.getDataInterval(); + final Interval shouldBeLarger = queryableIndices.get(i + 1).lhs.getDataInterval(); + Preconditions.checkState( + intervalComparator.compare(shouldBeSmaller, shouldBeLarger) <= 0, + "QueryableIndexes are not sorted! Interval[%s] of segment[%s] is laster than interval[%s] of segment[%s]", + shouldBeSmaller, + queryableIndices.get(i).rhs.getIdentifier(), + shouldBeLarger, + queryableIndices.get(i + 1).rhs.getIdentifier() + ); + } + int index = 0; - for (QueryableIndex queryableIndex : Lists.reverse(queryableIndices)) { + for (Pair pair : Lists.reverse(queryableIndices)) { + final QueryableIndex queryableIndex = pair.lhs; final Map dimensionHandlerMap = queryableIndex.getDimensionHandlers(); for (String dimension : queryableIndex.getAvailableDimensions()) { @@ -377,23 +425,22 @@ private static DimensionsSpec createDimensionsSpec(List queryabl return new DimensionsSpec(dimensionSchemas, null, null); } - private static List loadSegments( + private static List> loadSegments( List> timelineSegments, Map segmentFileMap, IndexIO indexIO ) throws IOException { - final List segments = new ArrayList<>(); + final List> segments = new ArrayList<>(); for (TimelineObjectHolder timelineSegment : timelineSegments) { final PartitionHolder partitionHolder = timelineSegment.getObject(); for (PartitionChunk chunk : partitionHolder) { final DataSegment segment = chunk.getObject(); - segments.add( - indexIO.loadIndex( - Preconditions.checkNotNull(segmentFileMap.get(segment), "File for segment %s", segment.getIdentifier()) - ) + final QueryableIndex queryableIndex = indexIO.loadIndex( + Preconditions.checkNotNull(segmentFileMap.get(segment), "File for segment %s", segment.getIdentifier()) ); + segments.add(Pair.of(queryableIndex, segment)); } } 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 4386b5b35953..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; @@ -229,44 +351,194 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception specVersion, version ); + toolbox.getTaskReportFileWriter().write(null); return TaskStatus.failure(getId()); } } 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); - return TaskStatus.success(getId()); - } else { - 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( @@ -275,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( @@ -315,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 199aacdd8bb5..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 @@ -35,7 +35,6 @@ import java.io.File; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.net.MalformedURLException; import java.net.URISyntaxException; import java.net.URL; import java.net.URLClassLoader; @@ -127,7 +126,6 @@ public boolean apply(@Nullable URL input) * * @param toolbox The toolbox to pull the default coordinates from if not present in the task * @return An isolated URLClassLoader not tied by parent chain to the ApplicationClassLoader - * @throws MalformedURLException from Initialization.getClassLoaderForExtension */ protected ClassLoader buildClassLoader(final TaskToolbox toolbox) { @@ -225,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 7a4dff26c85b..aecdc11ef010 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,6 +72,7 @@ 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; @@ -75,18 +84,31 @@ 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,129 @@ 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)) { - return TaskStatus.success(getId()); - } else { - 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) { return versions.entrySet().stream() @@ -280,16 +524,15 @@ private static String findVersion(Map versions, Interval inter private static boolean isGuaranteedRollup(IndexIOConfig ioConfig, IndexTuningConfig tuningConfig) { Preconditions.checkState( - !(tuningConfig.isForceGuaranteedRollup() && - (tuningConfig.isForceExtendableShardSpecs() || ioConfig.isAppendToExisting())), - "Perfect rollup cannot be guaranteed with extendable shardSpecs" + !tuningConfig.isForceGuaranteedRollup() || !ioConfig.isAppendToExisting(), + "Perfect rollup cannot be guaranteed when appending to existing dataSources" ); return tuningConfig.isForceGuaranteedRollup(); } private static boolean isExtendableShardSpecs(IndexIOConfig ioConfig, IndexTuningConfig tuningConfig) { - return !isGuaranteedRollup(ioConfig, tuningConfig); + return tuningConfig.isForceExtendableShardSpecs() || ioConfig.isAppendToExisting(); } /** @@ -385,7 +628,7 @@ private static ShardSpecs createShardSpecWithoutInputScan( return new ShardSpecs(shardSpecs); } - private static ShardSpecs createShardSpecsFromInput( + private ShardSpecs createShardSpecsFromInput( ObjectMapper jsonMapper, IndexIngestionSpec ingestionSchema, FirehoseFactory firehoseFactory, @@ -447,7 +690,7 @@ private static ShardSpecs createShardSpecsFromInput( return new ShardSpecs(intervalToShardSpecs); } - private static Map> collectIntervalsAndShardSpecs( + private Map> collectIntervalsAndShardSpecs( ObjectMapper jsonMapper, IndexIngestionSpec ingestionSchema, FirehoseFactory firehoseFactory, @@ -457,6 +700,11 @@ private static Map> collectIntervalsAnd boolean determineNumPartitions ) throws IOException { + determinePartitionsFireDepartmentMetrics = new FireDepartmentMetrics(); + determinePartitionsMetricsGetter = new FireDepartmentMetricsTaskMetricsGetter( + determinePartitionsFireDepartmentMetrics + ); + final Map> hllCollectors = new TreeMap<>( Comparators.intervalsByStartThenEnd() ); @@ -467,12 +715,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; } @@ -480,9 +730,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(); @@ -506,12 +764,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..."); } } } @@ -559,7 +826,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, @@ -572,7 +839,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( @@ -650,7 +918,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) ) { @@ -661,13 +929,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; } @@ -700,15 +976,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); @@ -719,15 +995,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( @@ -737,7 +1019,9 @@ dataSchema, new RealtimeIOConfig(null, null, null), null ) ) ); - return true; + + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + return TaskStatus.success(getId()); } } catch (TimeoutException | ExecutionException e) { @@ -745,6 +1029,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 @@ -941,10 +1248,26 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi private final IndexSpec indexSpec; private final File basePersistDirectory; private final int maxPendingPersists; + + /** + * This flag is to force to always use an extendableShardSpec (like {@link NumberedShardSpec} even if + * {@link #forceGuaranteedRollup} is set. + */ private final boolean forceExtendableShardSpecs; + + /** + * This flag is to force _perfect rollup mode_. {@link IndexTask} will scan the whole input data twice to 1) figure + * out proper shard specs for each segment and 2) generate segments. Note that perfect rollup mode basically assumes + * that no more data will be appended in the future. As a result, in perfect rollup mode, {@link NoneShardSpec} and + * {@link HashBasedNumberedShardSpec} are used for a single shard and two or shards, respectively. + */ private final boolean forceGuaranteedRollup; private final boolean reportParseExceptions; private final long pushTimeout; + private final boolean logParseExceptions; + private final int maxParseExceptions; + private final int maxSavedParseExceptions; + @Nullable private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; @@ -961,10 +1284,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( @@ -979,13 +1305,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); + this(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); } private IndexTuningConfig( @@ -1000,7 +1329,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( @@ -1024,12 +1356,18 @@ private IndexTuningConfig( this.pushTimeout = pushTimeout == null ? DEFAULT_PUSH_TIMEOUT : pushTimeout; this.basePersistDirectory = basePersistDirectory; - Preconditions.checkArgument( - !(this.forceExtendableShardSpecs && this.forceGuaranteedRollup), - "Perfect rollup cannot be guaranteed with extendable shardSpecs" - ); - this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + + if (this.reportParseExceptions) { + 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) @@ -1066,7 +1404,10 @@ public IndexTuningConfig withBasePersistDirectory(File dir) reportParseExceptions, pushTimeout, dir, - segmentWriteOutMediumFactory + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); } @@ -1150,6 +1491,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() { @@ -1185,7 +1544,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 @@ -1203,7 +1565,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/common/task/MergeTaskBase.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java index 747cf58d4162..7f2cae1a08e5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Objects; @@ -34,9 +33,6 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.common.hash.Hashing; -import io.druid.java.util.emitter.EmittingLogger; -import io.druid.java.util.emitter.service.ServiceEmitter; -import io.druid.java.util.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; @@ -45,8 +41,11 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; -import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.java.util.emitter.EmittingLogger; +import io.druid.java.util.emitter.service.ServiceEmitter; +import io.druid.java.util.emitter.service.ServiceMetricEvent; import io.druid.segment.IndexIO; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.joda.time.DateTime; @@ -54,6 +53,7 @@ import javax.annotation.Nullable; import java.io.File; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; import java.util.Set; @@ -307,7 +307,7 @@ public String apply(DataSegment x) return StringUtils.format( "%s_%s", dataSource, - Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString() + Hashing.sha1().hashString(segmentIDs, StandardCharsets.UTF_8).toString() ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/FileTaskLogs.java b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/FileTaskLogs.java index 1c09b56cd8f9..579234e6c07e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/FileTaskLogs.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/FileTaskLogs.java @@ -53,7 +53,7 @@ public FileTaskLogs( public void pushTaskLog(final String taskid, File file) throws IOException { if (config.getDirectory().exists() || config.getDirectory().mkdirs()) { - final File outputFile = fileForTask(taskid); + final File outputFile = fileForTask(taskid, file.getName()); Files.copy(file, outputFile); log.info("Wrote task log to: %s", outputFile); } else { @@ -61,10 +61,22 @@ public void pushTaskLog(final String taskid, File file) throws IOException } } + @Override + public void pushTaskReports(String taskid, File reportFile) throws IOException + { + if (config.getDirectory().exists() || config.getDirectory().mkdirs()) { + final File outputFile = fileForTask(taskid, reportFile.getName()); + Files.copy(reportFile, outputFile); + log.info("Wrote task report to: %s", outputFile); + } else { + throw new IOE("Unable to create task report dir[%s]", config.getDirectory()); + } + } + @Override public Optional streamTaskLog(final String taskid, final long offset) { - final File file = fileForTask(taskid); + final File file = fileForTask(taskid, "log"); if (file.exists()) { return Optional.of( new ByteSource() @@ -81,9 +93,29 @@ public InputStream openStream() throws IOException } } - private File fileForTask(final String taskid) + @Override + public Optional streamTaskReports(final String taskid) + { + final File file = fileForTask(taskid, "report.json"); + if (file.exists()) { + return Optional.of( + new ByteSource() + { + @Override + public InputStream openStream() throws IOException + { + return LogUtils.streamFile(file, 0); + } + } + ); + } else { + return Optional.absent(); + } + } + + private File fileForTask(final String taskid, String filename) { - return new File(config.getDirectory(), StringUtils.format("%s.log", taskid)); + return new File(config.getDirectory(), StringUtils.format("%s.%s", taskid, filename)); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/SwitchingTaskLogStreamer.java b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/SwitchingTaskLogStreamer.java index cb8e0b34325d..3b7e33baa76c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/SwitchingTaskLogStreamer.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/SwitchingTaskLogStreamer.java @@ -53,4 +53,17 @@ public Optional streamTaskLog(String taskid, long offset) throws IOE return Optional.absent(); } + + @Override + public Optional streamTaskReports(String taskid) throws IOException + { + for (TaskLogStreamer provider : providers) { + final Optional stream = provider.streamTaskReports(taskid); + if (stream.isPresent()) { + return stream; + } + } + + return Optional.absent(); + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java index 519c172f35f6..041dfd879f9d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java @@ -260,6 +260,7 @@ public TaskStatus call() final File taskFile = new File(taskDir, "task.json"); final File statusFile = new File(attemptDir, "status.json"); final File logFile = new File(taskDir, "log"); + final File reportsFile = new File(attemptDir, "report.json"); // time to adjust process holders synchronized (tasks) { @@ -408,6 +409,7 @@ public TaskStatus call() command.add("peon"); command.add(taskFile.toString()); command.add(statusFile.toString()); + command.add(reportsFile.toString()); String nodeType = task.getNodeType(); if (nodeType != null) { command.add("--nodeType"); @@ -459,6 +461,9 @@ public TaskStatus call() Thread.currentThread().setName(priorThreadName); // Upload task logs taskLogPusher.pushTaskLog(task.getId(), logFile); + if (reportsFile.exists()) { + taskLogPusher.pushTaskReports(task.getId(), reportsFile); + } } TaskStatus status; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index 54427e435c24..8b06b195157b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Preconditions; @@ -92,6 +91,7 @@ import java.io.InputStream; import java.net.MalformedURLException; import java.net.URL; +import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -128,7 +128,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer { private static final EmittingLogger log = new EmittingLogger(RemoteTaskRunner.class); - private static final StatusResponseHandler RESPONSE_HANDLER = new StatusResponseHandler(Charsets.UTF_8); + private static final StatusResponseHandler RESPONSE_HANDLER = new StatusResponseHandler(StandardCharsets.UTF_8); private static final Joiner JOINER = Joiner.on("/"); private final ObjectMapper jsonMapper; @@ -968,7 +968,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) announcement.getTaskType(), zkWorker.getWorker(), TaskLocation.unknown(), - runningTasks.get(taskId).getDataSource() + announcement.getTaskDataSource() ); final RemoteTaskRunnerWorkItem existingItem = runningTasks.putIfAbsent( taskId, diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index a5cde1656476..32d856a62a7a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -636,7 +636,7 @@ public void unlock(final Task task, final Interval interval) return; } - final List possesHolder = dsRunning.get(interval); + final List possesHolder = dsRunning.get(interval); if (possesHolder == null || possesHolder.isEmpty()) { return; } 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/hrtr/WorkerHolder.java b/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/WorkerHolder.java index 199abb6821c8..7adc43419d9e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/WorkerHolder.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/WorkerHolder.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; -import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Sets; @@ -50,6 +49,7 @@ import java.net.MalformedURLException; import java.net.URL; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -73,7 +73,7 @@ public class WorkerHolder { }; - private static final StatusResponseHandler RESPONSE_HANDLER = new StatusResponseHandler(Charsets.UTF_8); + private static final StatusResponseHandler RESPONSE_HANDLER = new StatusResponseHandler(StandardCharsets.UTF_8); private final Worker worker; private Worker disabledWorker; @@ -387,7 +387,8 @@ public void fullSync(List changes) announcement.getTaskType(), announcement.getTaskResource(), TaskStatus.failure(announcement.getTaskId()), - announcement.getTaskLocation() + announcement.getTaskLocation(), + announcement.getTaskDataSource() )); } } @@ -423,7 +424,8 @@ public void deltaSync(List changes) announcement.getTaskType(), announcement.getTaskResource(), TaskStatus.failure(announcement.getTaskId()), - announcement.getTaskLocation() + announcement.getTaskLocation(), + announcement.getTaskDataSource() )); } } else if (change instanceof WorkerHistoryItem.Metadata) { 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 8be1dcdbe95e..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(); @@ -732,6 +734,33 @@ public Response doGetLog( } } + @GET + @Path("/task/{taskid}/reports") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(TaskResourceFilter.class) + public Response doGetReports( + @PathParam("taskid") final String taskid + ) + { + try { + final Optional stream = taskLogStreamer.streamTaskReports(taskid); + if (stream.isPresent()) { + return Response.ok(stream.get().openStream()).build(); + } else { + return Response.status(Response.Status.NOT_FOUND) + .entity( + "No task reports were found for this task. " + + "The task may not exist, or it may not have completed yet." + ) + .build(); + } + } + catch (Exception e) { + log.warn(e, "Failed to stream task reports for task %s", taskid); + return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build(); + } + } + @GET @Path("/dataSources/{dataSource}") @Produces(MediaType.APPLICATION_JSON) @@ -777,7 +806,8 @@ public TaskStatusPlus apply(TaskRunnerWorkItem workItem) null, null, workItem.getLocation(), - workItem.getDataSource() + workItem.getDataSource(), + null ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/TaskAnnouncement.java b/indexing-service/src/main/java/io/druid/indexing/worker/TaskAnnouncement.java index 5079d8b04b48..7ab4cc6129e3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/TaskAnnouncement.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/TaskAnnouncement.java @@ -28,6 +28,8 @@ import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.TaskResource; +import javax.annotation.Nullable; + /** * Used by workers to announce the status of tasks they are currently running. This class is immutable. */ @@ -38,9 +40,12 @@ public class TaskAnnouncement private final TaskResource taskResource; private final TaskLocation taskLocation; + @Nullable + private final String taskDataSource; // nullable for backward compatibility + public static TaskAnnouncement create(Task task, TaskStatus status, TaskLocation location) { - return create(task.getId(), task.getType(), task.getTaskResource(), status, location); + return create(task.getId(), task.getType(), task.getTaskResource(), status, location, task.getDataSource()); } public static TaskAnnouncement create( @@ -48,11 +53,12 @@ public static TaskAnnouncement create( String taskType, TaskResource resource, TaskStatus status, - TaskLocation location + TaskLocation location, + String taskDataSource ) { Preconditions.checkArgument(status.getId().equals(taskId), "task id == status id"); - return new TaskAnnouncement(null, taskType, null, status, resource, location); + return new TaskAnnouncement(null, taskType, null, status, resource, location, taskDataSource); } @JsonCreator @@ -62,7 +68,8 @@ private TaskAnnouncement( @JsonProperty("status") TaskState status, @JsonProperty("taskStatus") TaskStatus taskStatus, @JsonProperty("taskResource") TaskResource taskResource, - @JsonProperty("taskLocation") TaskLocation taskLocation + @JsonProperty("taskLocation") TaskLocation taskLocation, + @JsonProperty("taskDataSource") String taskDataSource ) { this.taskType = taskType; @@ -74,6 +81,7 @@ private TaskAnnouncement( } this.taskResource = taskResource == null ? new TaskResource(this.taskStatus.getId(), 1) : taskResource; this.taskLocation = taskLocation == null ? TaskLocation.unknown() : taskLocation; + this.taskDataSource = taskDataSource; } @JsonProperty("id") @@ -112,13 +120,21 @@ public TaskLocation getTaskLocation() return taskLocation; } + @JsonProperty("taskDataSource") + public String getTaskDataSource() + { + return taskDataSource; + } + @Override public String toString() { return "TaskAnnouncement{" + - "taskStatus=" + taskStatus + + "taskType=" + taskType + + ", taskStatus=" + taskStatus + ", taskResource=" + taskResource + ", taskLocation=" + taskLocation + + ", taskDataSource=" + taskDataSource + '}'; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java index 66eb0a40d931..d5ff7d3854cb 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java @@ -136,7 +136,8 @@ private void cleanupStaleAnnouncements() throws Exception announcement.getTaskType(), announcement.getTaskResource(), completionStatus, - TaskLocation.unknown() + TaskLocation.unknown(), + announcement.getTaskDataSource() ) ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java index d06830eaf6d7..98e6382a43ae 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java @@ -37,6 +37,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.lifecycle.LifecycleStart; import io.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.commons.io.FileUtils; import java.io.File; import java.io.IOException; @@ -190,7 +191,7 @@ public TaskStatus apply(TaskStatus taskStatus) final File statusFileParent = statusFile.getParentFile(); if (statusFileParent != null) { - statusFileParent.mkdirs(); + FileUtils.forceMkdir(statusFileParent); } jsonMapper.writeValue(statusFile, taskStatus); diff --git a/indexing-service/src/main/resources/indexer_static/console.html b/indexing-service/src/main/resources/indexer_static/console.html index d397b1bbbea8..bfc3dd546272 100644 --- a/indexing-service/src/main/resources/indexer_static/console.html +++ b/indexing-service/src/main/resources/indexer_static/console.html @@ -36,7 +36,7 @@
-
Coordinator Console
+
Overlord Console