From 095bb32ffd57bccdf59f93e3462e51d01285618e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 4 Nov 2019 22:29:34 -0800 Subject: [PATCH 01/29] Add InputSource and InputFormat interfaces --- .../data/input/FiniteFirehoseFactory.java | 1 + .../FirehoseFactoryToInputSourceAdaptor.java | 96 ++++++ .../apache/druid/data/input/InputSource.java | 73 +++++ .../druid/data/input/InputSourceReader.java | 41 +++ .../druid/data/input/SplitHintSpec.java | 2 + .../apache/druid/data/input/SplitReader.java | 40 +++ .../apache/druid/data/input/SplitSource.java | 111 +++++++ .../apache/druid/data/input/TextReader.java | 134 ++++++++ .../druid/data/input/impl/ByteSource.java | 63 ++++ .../druid/data/input/impl/CSVParseSpec.java | 11 +- .../druid/data/input/impl/CsvInputFormat.java | 144 +++++++++ .../druid/data/input/impl/CsvReader.java | 134 ++++++++ .../druid/data/input/impl/FileSource.java | 84 +++++ .../FirehoseToInputSourceReaderAdaptor.java | 122 ++++++++ .../data/input/impl/HttpInputSource.java | 142 +++++++++ .../druid/data/input/impl/HttpSource.java | 89 ++++++ .../druid/data/input/impl/InputFormat.java | 54 ++++ .../druid/data/input/impl/InputRowParser.java | 1 + .../druid/data/input/impl/JSONParseSpec.java | 8 + .../data/input/impl/JsonInputFormat.java | 94 ++++++ .../druid/data/input/impl/JsonReader.java | 75 +++++ .../data/input/impl/LocalInputSource.java | 150 +++++++++ .../data/input/impl/MapInputRowParser.java | 43 ++- .../data/input/impl/NestedInputFormat.java | 65 ++++ .../druid/data/input/impl/ParseSpec.java | 12 + .../data/input/impl/SplitIteratingReader.java | 138 ++++++++ .../input/impl/SplittableInputSource.java | 69 ++++ .../druid/data/input/impl/TimestampSpec.java | 7 +- .../java/org/apache/druid/indexer/Checks.java | 2 +- .../org/apache/druid/indexer/Property.java | 5 +- .../druid/java/util/common/FileUtils.java | 27 ++ .../parsers/AbstractFlatTextFormatParser.java | 18 +- .../common/parsers/CloseableIterator.java | 31 ++ .../apache/druid/timeline/DataSegment.java | 1 + ...rehoseFactoryToInputSourceAdaptorTest.java | 152 +++++++++ .../data/input/impl/CsvInputFormatTest.java | 70 +++++ .../druid/data/input/impl/CsvReaderTest.java | 178 +++++++++++ .../data/input/impl/HttpInputSourceTest.java | 47 +++ .../data/input/impl/JsonInputFormatTest.java | 58 ++++ .../druid/data/input/impl/JsonReaderTest.java | 116 +++++++ .../data/input/impl/LocalInputSourceTest.java | 44 +++ .../data/input/impl/NoopFirehoseFactory.java | 31 ++ .../data/input/impl/NoopInputFormat.java | 43 +++ .../data/input/impl/NoopInputSource.java | 46 +++ .../input/impl/SplitIteratingReaderTest.java | 98 ++++++ docs/ingestion/tasks.md | 2 +- .../druid/indexing/kafka/KafkaIndexTask.java | 2 +- .../indexing/kinesis/KinesisIndexTask.java | 3 +- extensions-core/orc-extensions/pom.xml | 11 +- .../parquet/simple/ParquetGroupConverter.java | 2 +- .../indexer/HadoopDruidIndexerConfig.java | 5 +- .../druid/indexer/IndexGeneratorJob.java | 22 +- .../druid/indexing/common/TaskToolbox.java | 2 +- .../AppenderatorDriverRealtimeIndexTask.java | 8 +- .../indexing/common/task/CompactionTask.java | 2 + .../druid/indexing/common/task/IndexTask.java | 165 +++++++--- ...ocessor.java => InputSourceProcessor.java} | 36 ++- .../common/task/RealtimeIndexTask.java | 7 +- .../batch/parallel/ParallelIndexIOConfig.java | 13 +- .../parallel/ParallelIndexIngestionSpec.java | 14 + .../parallel/ParallelIndexPhaseRunner.java | 23 +- .../parallel/ParallelIndexSupervisorTask.java | 29 +- ...egmentGenerateParallelIndexTaskRunner.java | 26 +- .../parallel/PartialSegmentGenerateTask.java | 27 +- .../parallel/PartialSegmentMergeTask.java | 2 +- .../SinglePhaseParallelIndexTaskRunner.java | 22 +- .../batch/parallel/SinglePhaseSubTask.java | 36 ++- .../common/task/CompactionTaskRunTest.java | 12 +- .../common/task/CompactionTaskTest.java | 12 +- .../common/task/IndexIngestionSpecTest.java | 142 +++++++++ .../indexing/common/task/IndexTaskTest.java | 295 +++++++++--------- ...stractParallelIndexSupervisorTaskTest.java | 19 +- .../MultiPhaseParallelIndexingTest.java | 117 +++---- .../ParallelIndexSupervisorTaskKillTest.java | 71 +++-- ...rallelIndexSupervisorTaskResourceTest.java | 93 ++++-- .../ParallelIndexSupervisorTaskSerdeTest.java | 42 +-- .../SinglePhaseParallelIndexingTest.java | 173 +++++----- .../indexing/overlord/TaskLifecycleTest.java | 92 ++++-- .../incremental/IncrementalIndexSchema.java | 7 +- .../segment/transform/TransformSpec.java | 8 +- .../druid/segment/transform/Transformer.java | 31 +- .../segment/transform/TransformingReader.java | 51 +++ .../druid/segment/indexing/BatchIOConfig.java | 35 +++ .../druid/segment/indexing/DataSchema.java | 241 +++++++++----- .../segment/realtime/FireDepartment.java | 3 +- .../firehose/HttpFirehoseFactory.java | 20 +- .../realtime/firehose/SqlFirehoseFactory.java | 4 +- .../druid/segment/realtime/plumber/Sink.java | 4 +- .../segment/indexing/DataSchemaTest.java | 6 +- .../indexing/TestModifiedDataSchema.java | 20 +- .../segment/realtime/plumber/SinkTest.java | 15 +- 91 files changed, 4236 insertions(+), 706 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java create mode 100644 core/src/main/java/org/apache/druid/data/input/InputSource.java create mode 100644 core/src/main/java/org/apache/druid/data/input/InputSourceReader.java create mode 100644 core/src/main/java/org/apache/druid/data/input/SplitReader.java create mode 100644 core/src/main/java/org/apache/druid/data/input/SplitSource.java create mode 100644 core/src/main/java/org/apache/druid/data/input/TextReader.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/FileSource.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/InputFormat.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/SplitIteratingReader.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java create mode 100644 core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java create mode 100644 core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java create mode 100644 core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java create mode 100644 core/src/test/java/org/apache/druid/data/input/impl/HttpInputSourceTest.java create mode 100644 core/src/test/java/org/apache/druid/data/input/impl/JsonInputFormatTest.java create mode 100644 core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java create mode 100644 core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java create mode 100644 core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java create mode 100644 core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java create mode 100644 core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java create mode 100644 core/src/test/java/org/apache/druid/data/input/impl/SplitIteratingReaderTest.java rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/{FiniteFirehoseProcessor.java => InputSourceProcessor.java} (87%) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java create mode 100644 processing/src/main/java/org/apache/druid/segment/transform/TransformingReader.java create mode 100644 server/src/main/java/org/apache/druid/segment/indexing/BatchIOConfig.java diff --git a/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java b/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java index 09f943bf3674..5bfc5f85d565 100644 --- a/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java +++ b/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java @@ -33,6 +33,7 @@ * @param parser type * @param input split type */ +@Deprecated public interface FiniteFirehoseFactory extends FirehoseFactory { /** diff --git a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java new file mode 100644 index 000000000000..e3a066beb726 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.FirehoseToInputSourceReaderAdaptor; +import org.apache.druid.data.input.impl.InputFormat; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.data.input.impl.TimestampSpec; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.stream.Stream; + +public class FirehoseFactoryToInputSourceAdaptor implements SplittableInputSource +{ + private final FirehoseFactory firehoseFactory; + private final InputRowParser inputRowParser; + + public FirehoseFactoryToInputSourceAdaptor(FirehoseFactory firehoseFactory, InputRowParser inputRowParser) + { + this.firehoseFactory = firehoseFactory; + this.inputRowParser = Preconditions.checkNotNull(inputRowParser, "inputRowParser"); + } + + @Override + public boolean isSplittable() + { + return firehoseFactory.isSplittable(); + } + + @Override + public Stream createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + throws IOException + { + if (firehoseFactory.isSplittable()) { + return ((FiniteFirehoseFactory) firehoseFactory).getSplits(splitHintSpec); + } else { + throw new UnsupportedOperationException(); + } + } + + @Override + public int getNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException + { + if (firehoseFactory.isSplittable()) { + return ((FiniteFirehoseFactory) firehoseFactory).getNumSplits(splitHintSpec); + } else { + throw new UnsupportedOperationException(); + } + } + + @Override + public SplittableInputSource withSplit(InputSplit split) + { + if (firehoseFactory.isSplittable()) { + return new FirehoseFactoryToInputSourceAdaptor( + ((FiniteFirehoseFactory) firehoseFactory).withSplit(split), + inputRowParser + ); + } else { + throw new UnsupportedOperationException(); + } + } + + @Override + public InputSourceReader reader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + @Nullable InputFormat inputFormat, // inputFormat will be ignored + @Nullable File temporaryDirectory + ) + { + return new FirehoseToInputSourceReaderAdaptor(firehoseFactory, inputRowParser, temporaryDirectory); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/InputSource.java b/core/src/main/java/org/apache/druid/data/input/InputSource.java new file mode 100644 index 000000000000..117a8aad9144 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/InputSource.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonSubTypes.Type; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.HttpInputSource; +import org.apache.druid.data.input.impl.InputFormat; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.guice.annotations.ExtensionPoint; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; + +/** + * InputSource abstracts the storage system where input data is stored. + * It creates an {@link InputSourceReader} to read data from the given input source. + * The most common use case would be: + * + *
{@code
+ *   InputSourceReader reader = inputSource.reader();
+ *   try (CloseableIterator iterator = reader.read()) {
+ *     while (iterator.hasNext()) {
+ *       InputRow row = iterator.next();
+ *       processRow(row);
+ *     }
+ *   }
+ * }
+ */ +@ExtensionPoint +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @Type(name = "local", value = LocalInputSource.class), + @Type(name = "http", value = HttpInputSource.class) +}) +public interface InputSource +{ + /** + * Returns true if this inputSource can be processed in parallel using ParallelIndexSupervisorTask. + */ + default boolean isSplittable() + { + return false; + } + + InputSourceReader reader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) throws IOException; +} diff --git a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java new file mode 100644 index 000000000000..f29ccf53b743 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import java.io.IOException; + +/** + * InputSourceReader reads data from {@link InputSource} and returns a {@link CloseableIterator} of + * {@link InputRow}. See {@link InputSource} for an example usage. + * + * Implementations of this class can use {@link SplitSource} and {@link SplitReader}. + * + * See {@link org.apache.druid.data.input.impl.SplitIteratingReader} as an example. + */ +@ExtensionPoint +public interface InputSourceReader +{ + CloseableIterator read() throws IOException; + + CloseableIterator sample() throws IOException; +} diff --git a/core/src/main/java/org/apache/druid/data/input/SplitHintSpec.java b/core/src/main/java/org/apache/druid/data/input/SplitHintSpec.java index 69042a74d92f..5e22160c4526 100644 --- a/core/src/main/java/org/apache/druid/data/input/SplitHintSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/SplitHintSpec.java @@ -31,6 +31,8 @@ * * @see FiniteFirehoseFactory#getSplits(SplitHintSpec) * @see FiniteFirehoseFactory#getNumSplits(SplitHintSpec) + * @see org.apache.druid.data.input.impl.SplittableInputSource#createSplits + * @see org.apache.druid.data.input.impl.SplittableInputSource#getNumSplits */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { diff --git a/core/src/main/java/org/apache/druid/data/input/SplitReader.java b/core/src/main/java/org/apache/druid/data/input/SplitReader.java new file mode 100644 index 000000000000..cf0f3128225b --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/SplitReader.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import java.io.File; +import java.io.IOException; + +/** + * SplitReader knows how to parse data into {@link InputRow}. + * This class is stateful and a new SplitReader should be created per {@link InputSplit}. + * + * @see TextReader for text format readers + */ +@ExtensionPoint +public interface SplitReader +{ + CloseableIterator read(SplitSource source, File temporaryDirectory) throws IOException; + + CloseableIterator sample(SplitSource source, File temporaryDirectory) throws IOException; +} diff --git a/core/src/main/java/org/apache/druid/data/input/SplitSource.java b/core/src/main/java/org/apache/druid/data/input/SplitSource.java new file mode 100644 index 000000000000..0ae0ccce9744 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/SplitSource.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import com.google.common.base.Predicate; +import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; + +/** + * SplitSource abstracts an {@link InputSplit} and knows how to read bytes from the given split. + */ +@ExtensionPoint +public interface SplitSource +{ + Logger LOG = new Logger(SplitSource.class); + + int DEFAULT_FETCH_BUFFER_SIZE = 4 * 1024; // 4 KB + int DEFAULT_MAX_FETCH_RETRY = 2; // 3 tries including the initial try + + /** + * CleanableFile is the result type of {@link #fetch}. + * It should clean up any temporary resource on {@link #close()}. + */ + interface CleanableFile extends Closeable + { + File file(); + } + + InputSplit getSplit(); + + /** + * Opens an {@link InputStream} on the split directly. + * This is the basic way to read the given split. + * + * @see #fetch as an alternative way to read data. + */ + InputStream open() throws IOException; + + /** + * Fetches the split into the local storage. + * This method might be preferred instead of {@link #open()}, for example + * + * - {@link org.apache.druid.data.input.impl.InputFormat} requires expensive random access on remote storage. + * - Holding a connection until you consume the entire InputStream is expensive. + * + * @param temporaryDirectory to store temp data. This directory will be removed automatically once + * the task finishes. + * @param fetchBuffer is used to fetch remote split into local storage. + * + * @see FileUtils#copyLarge + */ + default CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) throws IOException + { + final File tempFile = File.createTempFile("druid-split", ".tmp", temporaryDirectory); + LOG.debug("Fetching split into file[%s]", tempFile.getAbsolutePath()); + FileUtils.copyLarge( + open(), + tempFile, + fetchBuffer, + getRetryCondition(), + DEFAULT_MAX_FETCH_RETRY, + StringUtils.format("Failed to fetch into [%s]", tempFile.getAbsolutePath()) + ); + + return new CleanableFile() + { + @Override + public File file() + { + return tempFile; + } + + @Override + public void close() + { + if (!tempFile.delete()) { + LOG.warn("Failed to remove file[%s]", tempFile.getAbsolutePath()); + } + } + }; + } + + /** + * {@link #fetch} will retry during the fetch if it sees an exception mathing to the returned predicate. + */ + Predicate getRetryCondition(); +} diff --git a/core/src/main/java/org/apache/druid/data/input/TextReader.java b/core/src/main/java/org/apache/druid/data/input/TextReader.java new file mode 100644 index 000000000000..23a5cdbeb063 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/TextReader.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import org.apache.commons.io.LineIterator; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.ParseException; + +import java.io.File; +import java.io.IOException; +import java.io.InputStreamReader; + +/** + * Abstract {@link SplitReader} for text format readers such as CSV or JSON. + */ +public abstract class TextReader implements SplitReader +{ + private final TimestampSpec timestampSpec; + private final DimensionsSpec dimensionsSpec; + + public TextReader(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec) + { + this.timestampSpec = timestampSpec; + this.dimensionsSpec = dimensionsSpec; + } + + public TimestampSpec getTimestampSpec() + { + return timestampSpec; + } + + public DimensionsSpec getDimensionsSpec() + { + return dimensionsSpec; + } + + @Override + public CloseableIterator read(SplitSource source, File temporaryDirectory) throws IOException + { + return lineIterator(source).map(line -> { + try { + return readLine(line); + } + catch (IOException e) { + throw new ParseException(e, "Unable to parse row [%s]", line); + } + }); + } + + @Override + public CloseableIterator sample(SplitSource source, File temporaryDirectory) + throws IOException + { + return lineIterator(source).map(line -> { + try { + return InputRowPlusRaw.of(readLine(line), StringUtils.toUtf8(line)); + } + catch (ParseException e) { + return InputRowPlusRaw.of(StringUtils.toUtf8(line), e); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + private CloseableIterator lineIterator(SplitSource source) throws IOException + { + final LineIterator delegate = new LineIterator( + new InputStreamReader(source.open(), StringUtils.UTF8_STRING) + ); + final int numHeaderLines = getNumHeaderLines(); + for (int i = 0; i < numHeaderLines && delegate.hasNext(); i++) { + processHeaderLine(delegate.nextLine()); + } + + return new CloseableIterator() + { + @Override + public boolean hasNext() + { + return delegate.hasNext(); + } + + @Override + public String next() + { + return delegate.nextLine(); + } + + @Override + public void close() throws IOException + { + delegate.close(); + } + }; + } + + /** + * Parses the given line into {@link InputRow}. + */ + public abstract InputRow readLine(String line) throws IOException, ParseException; + + /** + * Returns the number of header lines to skip. + * {@link #processHeaderLine} will be called as many times as the returned number. + */ + public abstract int getNumHeaderLines(); + + /** + * Processes a header line. This will be called as many times as {@link #getNumHeaderLines()}. + */ + public abstract void processHeaderLine(String line) throws IOException; +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java b/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java new file mode 100644 index 000000000000..f828b566318f --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitSource; +import org.apache.druid.io.ByteBufferInputStream; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +public class ByteSource implements SplitSource +{ + private final InputSplit split; + + public ByteSource(ByteBuffer buffer) + { + this.split = new InputSplit<>(buffer.duplicate()); + } + + public ByteSource(byte[] bytes) + { + this(ByteBuffer.wrap(bytes)); + } + + @Override + public InputSplit getSplit() + { + return split; + } + + @Override + public InputStream open() throws IOException + { + return new ByteBufferInputStream(split.get()); + } + + @Override + public Predicate getRetryCondition() + { + return Predicates.alwaysFalse(); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java index 61eca2e091e0..24f11e3e08d0 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java @@ -105,9 +105,7 @@ public int getSkipHeaderRows() @Override public void verify(List usedCols) { - for (String columnName : usedCols) { - Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName); - } + CsvReader.verify(columns, usedCols); } @Override @@ -116,6 +114,12 @@ public Parser makeParser() return new CSVParser(listDelimiter, columns, hasHeaderRow, skipHeaderRows); } + @Override + public InputFormat toInputFormat() + { + return new CsvInputFormat(columns, listDelimiter, hasHeaderRow, skipHeaderRows); + } + @Override public ParseSpec withTimestampSpec(TimestampSpec spec) { @@ -127,5 +131,4 @@ public ParseSpec withDimensionsSpec(DimensionsSpec spec) { return new CSVParseSpec(getTimestampSpec(), spec, listDelimiter, columns, hasHeaderRow, skipHeaderRows); } - } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java new file mode 100644 index 000000000000..f4b2ace7b3c1 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.SplitReader; +import org.apache.druid.indexer.Checks; +import org.apache.druid.indexer.Property; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +public class CsvInputFormat implements InputFormat +{ + private final String listDelimiter; + private final List columns; + private final boolean findColumnsFromHeader; + private final int skipHeaderRows; + + @JsonCreator + public CsvInputFormat( + @JsonProperty("columns") @Nullable List columns, + @JsonProperty("listDelimiter") String listDelimiter, + @Deprecated @JsonProperty("hasHeaderRow") @Nullable Boolean hasHeaderRow, + @JsonProperty("findColumnsFromHeader") @Nullable Boolean findColumnsFromHeader, + @JsonProperty("skipHeaderRows") int skipHeaderRows + ) + { + this.listDelimiter = listDelimiter; + this.columns = columns == null ? Collections.emptyList() : columns; + //noinspection ConstantConditions + this.findColumnsFromHeader = Checks.checkOneNotNullOrEmpty( + ImmutableList.of( + new Property<>("hasHeaderRow", hasHeaderRow), + new Property<>("findColumnsFromHeader", findColumnsFromHeader) + ) + ).getValue(); + this.skipHeaderRows = skipHeaderRows; + + if (!this.columns.isEmpty()) { + for (String column : this.columns) { + Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column); + } + } else { + Preconditions.checkArgument( + this.findColumnsFromHeader, + "If columns field is not set, the first row of your data must have your header" + + " and hasHeaderRow must be set to true." + ); + } + } + + @VisibleForTesting + public CsvInputFormat( + List columns, + String listDelimiter, + boolean findColumnsFromHeader, + int skipHeaderRows + ) + { + this(columns, listDelimiter, null, findColumnsFromHeader, skipHeaderRows); + } + + @JsonProperty + public List getColumns() + { + return columns; + } + + @JsonProperty + public String getListDelimiter() + { + return listDelimiter; + } + + @JsonProperty + public boolean isFindColumnsFromHeader() + { + return findColumnsFromHeader; + } + + @JsonProperty + public int getSkipHeaderRows() + { + return skipHeaderRows; + } + + @Override + public boolean isSplittable() + { + return true; + } + + @Override + public SplitReader createReader(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec) + { + return new CsvReader(timestampSpec, dimensionsSpec, listDelimiter, columns, findColumnsFromHeader, skipHeaderRows); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CsvInputFormat format = (CsvInputFormat) o; + return findColumnsFromHeader == format.findColumnsFromHeader && + skipHeaderRows == format.skipHeaderRows && + Objects.equals(listDelimiter, format.listDelimiter) && + Objects.equals(columns, format.columns); + } + + @Override + public int hashCode() + { + return Objects.hash(listDelimiter, columns, findColumnsFromHeader, skipHeaderRows); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java new file mode 100644 index 000000000000..5f868c63cfb6 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.base.Splitter; +import com.google.common.base.Strings; +import com.google.common.collect.Iterables; +import com.opencsv.CSVParser; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.TextReader; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.collect.Utils; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.java.util.common.parsers.ParserUtils; +import org.apache.druid.java.util.common.parsers.Parsers; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +public class CsvReader extends TextReader +{ + private final CSVParser parser = new CSVParser(); + private final boolean hasHeaderRow; + private final int skipHeaderRows; + private final Function multiValueFunction; + @Nullable + private List columns; + + CsvReader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + String listDelimiter, + @Nullable List columns, + boolean hasHeaderRow, + int skipHeaderRows + ) + { + super(timestampSpec, dimensionsSpec); + this.hasHeaderRow = hasHeaderRow; + this.skipHeaderRows = skipHeaderRows; + final String finalListDelimeter = listDelimiter == null ? Parsers.DEFAULT_LIST_DELIMITER : listDelimiter; + this.multiValueFunction = ParserUtils.getMultiValueFunction(finalListDelimeter, Splitter.on(finalListDelimeter)); + this.columns = hasHeaderRow ? null : columns; // columns will be overriden by header row + + if (this.columns != null) { + for (String column : this.columns) { + Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column); + } + verify(this.columns, dimensionsSpec.getDimensionNames()); + } else { + Preconditions.checkArgument( + hasHeaderRow, + "If columns field is not set, the first row of your data must have your header" + + " and hasHeaderRow must be set to true." + ); + } + } + + @Override + public InputRow readLine(String line) throws IOException, ParseException + { + final String[] parsed = parser.parseLine(line); + final Map zipped = Utils.zipMapPartial( + Preconditions.checkNotNull(columns, "columns"), + Iterables.transform(Arrays.asList(parsed), multiValueFunction) + ); + return MapInputRowParser.parse(getTimestampSpec(), getDimensionsSpec(), zipped); + } + + @Override + public int getNumHeaderLines() + { + return (hasHeaderRow ? 1 : 0) + skipHeaderRows; + } + + @Override + public void processHeaderLine(String line) throws IOException + { + if (hasHeaderRow && (columns == null || columns.isEmpty())) { + columns = findOrCreateColumnNames(Arrays.asList(parser.parseLine(line))); + } + if (columns == null || columns.isEmpty()) { + throw new ISE("Empty columns"); + } + } + + public static void verify(List columns, List dimensionNames) + { + for (String columnName : dimensionNames) { + Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName); + } + } + + public static List findOrCreateColumnNames(List parsedLine) + { + final List columns = new ArrayList<>(parsedLine.size()); + for (int i = 0; i < parsedLine.size(); i++) { + if (Strings.isNullOrEmpty(parsedLine.get(i))) { + columns.add(ParserUtils.getDefaultColumnName(i)); + } else { + columns.add(parsedLine.get(i)); + } + } + if (columns.isEmpty()) { + return ParserUtils.generateFieldNames(parsedLine.size()); + } else { + ParserUtils.validateFields(columns); + return columns; + } + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java b/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java new file mode 100644 index 000000000000..d77f0eabd6f9 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitSource; +import org.apache.druid.utils.CompressionUtils; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.RandomAccessFile; +import java.nio.channels.Channels; +import java.nio.channels.FileChannel; + +public class FileSource implements SplitSource +{ + private final InputSplit split; + private final FileChannel channel; + + FileSource(InputSplit split) throws FileNotFoundException + { + this.split = split; + final RandomAccessFile file = new RandomAccessFile(split.get(), "r"); + this.channel = file.getChannel(); + } + + @Override + public CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) + { + return new CleanableFile() + { + @Override + public File file() + { + return split.get(); + } + + @Override + public void close() + { + // do nothing + } + }; + } + + @Override + public InputSplit getSplit() + { + return split; + } + + @Override + public InputStream open() throws IOException + { + return CompressionUtils.decompress(Channels.newInputStream(channel), split.get().getName()); + } + + @Override + public Predicate getRetryCondition() + { + return Predicates.alwaysFalse(); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java new file mode 100644 index 000000000000..f62efc44aecd --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import java.io.File; +import java.io.IOException; + +public class FirehoseToInputSourceReaderAdaptor implements InputSourceReader +{ + private final FirehoseFactory firehoseFactory; + private final InputRowParser inputRowParser; + private final File temporaryDirectory; + + public FirehoseToInputSourceReaderAdaptor( + FirehoseFactory firehoseFactory, + InputRowParser inputRowPlusRaw, + File temporaryDirectory + ) + { + this.firehoseFactory = firehoseFactory; + this.inputRowParser = inputRowPlusRaw; + this.temporaryDirectory = temporaryDirectory; + } + + @Override + public CloseableIterator read() throws IOException + { + return new CloseableIterator() + { + final Firehose firehose = firehoseFactory.connect(inputRowParser, temporaryDirectory); + + @Override + public boolean hasNext() + { + try { + return firehose.hasMore(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public InputRow next() + { + try { + return firehose.nextRow(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() throws IOException + { + firehose.close(); + } + }; + } + + @Override + public CloseableIterator sample() throws IOException + { + return new CloseableIterator() + { + final Firehose firehose = firehoseFactory.connectForSampler(inputRowParser, temporaryDirectory); + + @Override + public boolean hasNext() + { + try { + return firehose.hasMore(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public InputRowPlusRaw next() + { + try { + return firehose.nextRowWithRaw(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() throws IOException + { + firehose.close(); + } + }; + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java new file mode 100644 index 000000000000..ac0b41f5afbf --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; +import org.apache.druid.metadata.PasswordProvider; + +import javax.annotation.Nullable; +import java.io.File; +import java.net.URI; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.stream.Stream; + +public class HttpInputSource implements SplittableInputSource +{ + private final List uris; + @Nullable + private final String httpAuthenticationUsername; + @Nullable + private final PasswordProvider httpAuthenticationPasswordProvider; + + @JsonCreator + public HttpInputSource( + @JsonProperty("uris") List uris, + @JsonProperty("httpAuthenticationUsername") @Nullable String httpAuthenticationUsername, + @JsonProperty("httpAuthenticationPassword") @Nullable PasswordProvider httpAuthenticationPasswordProvider + ) + { + Preconditions.checkArgument(uris != null && !uris.isEmpty(), "Empty URIs"); + this.uris = uris; + this.httpAuthenticationUsername = httpAuthenticationUsername; + this.httpAuthenticationPasswordProvider = httpAuthenticationPasswordProvider; + } + + @JsonProperty + public List getUris() + { + return uris; + } + + @Nullable + @JsonProperty + public String getHttpAuthenticationUsername() + { + return httpAuthenticationUsername; + } + + @Nullable + @JsonProperty("httpAuthenticationPassword") + public PasswordProvider getHttpAuthenticationPasswordProvider() + { + return httpAuthenticationPasswordProvider; + } + + @Override + public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + { + return uris.stream().map(InputSplit::new); + } + + @Override + public int getNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + { + return uris.size(); + } + + @Override + public SplittableInputSource withSplit(InputSplit split) + { + return new HttpInputSource( + Collections.singletonList(split.get()), + httpAuthenticationUsername, + httpAuthenticationPasswordProvider + ); + } + + @Override + public InputSourceReader reader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) + { + return new SplitIteratingReader<>( + timestampSpec, + dimensionsSpec, + inputFormat, + createSplits(inputFormat, null).map(split -> new HttpSource( + split, + httpAuthenticationUsername, + httpAuthenticationPasswordProvider + )), + temporaryDirectory + ); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + HttpInputSource source = (HttpInputSource) o; + return Objects.equals(uris, source.uris) && + Objects.equals(httpAuthenticationUsername, source.httpAuthenticationUsername) && + Objects.equals(httpAuthenticationPasswordProvider, source.httpAuthenticationPasswordProvider); + } + + @Override + public int hashCode() + { + return Objects.hash(uris, httpAuthenticationUsername, httpAuthenticationPasswordProvider); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java new file mode 100644 index 000000000000..899f4069028c --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.google.common.base.Predicate; +import com.google.common.base.Strings; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitSource; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.metadata.PasswordProvider; +import org.apache.druid.utils.CompressionUtils; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.net.URLConnection; +import java.util.Base64; + +public class HttpSource implements SplitSource +{ + private final InputSplit split; + @Nullable + private final String httpAuthenticationUsername; + @Nullable + private final PasswordProvider httpAuthenticationPasswordProvider; + + HttpSource( + InputSplit split, + @Nullable String httpAuthenticationUsername, + @Nullable PasswordProvider httpAuthenticationPasswordProvider + ) + { + this.split = split; + this.httpAuthenticationUsername = httpAuthenticationUsername; + this.httpAuthenticationPasswordProvider = httpAuthenticationPasswordProvider; + } + + @Override + public InputSplit getSplit() + { + return split; + } + + @Override + public InputStream open() throws IOException + { + return CompressionUtils.decompress(openURLConnection( + split.get(), + httpAuthenticationUsername, + httpAuthenticationPasswordProvider + ).getInputStream(), split.get().toString()); + } + + @Override + public Predicate getRetryCondition() + { + return t -> t instanceof IOException; + } + + public static URLConnection openURLConnection(URI object, String userName, PasswordProvider passwordProvider) + throws IOException + { + URLConnection urlConnection = object.toURL().openConnection(); + if (!Strings.isNullOrEmpty(userName) && passwordProvider != null) { + String userPass = userName + ":" + passwordProvider.getPassword(); + String basicAuthString = "Basic " + Base64.getEncoder().encodeToString(StringUtils.toUtf8(userPass)); + urlConnection.setRequestProperty("Authorization", basicAuthString); + } + return urlConnection; + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/InputFormat.java new file mode 100644 index 000000000000..496f159b51f4 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputFormat.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonSubTypes.Type; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.data.input.SplitReader; +import org.apache.druid.guice.annotations.ExtensionPoint; + +/** + * InputFormat abstracts the file format of input data. + * It creates a {@link SplitReader} to read data and parse it into {@link org.apache.druid.data.input.InputRow}. + * The created SplitReader is used by {@link org.apache.druid.data.input.InputSourceReader}. + * + * @see NestedInputFormat for nested input formats such as JSON. + */ +@ExtensionPoint +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @Type(name = "csv", value = CsvInputFormat.class), + @Type(name = "json", value = JsonInputFormat.class) +}) +public interface InputFormat +{ + /** + * Trait to indicate that a file can be split into multiple {@link org.apache.druid.data.input.InputSplit}s. + * + * This method is not being used anywhere for now, but should be considered + * in {@link SplittableInputSource#createSplits}. + */ + @JsonIgnore + boolean isSplittable(); + + SplitReader createReader(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec); +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputRowParser.java b/core/src/main/java/org/apache/druid/data/input/impl/InputRowParser.java index 52a0ac48c7a5..b29a59cc3e9d 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InputRowParser.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputRowParser.java @@ -29,6 +29,7 @@ import javax.validation.constraints.NotNull; import java.util.List; +@Deprecated @ExtensionPoint @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class) @JsonSubTypes(value = { diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JSONParseSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/JSONParseSpec.java index bf33d5136ec2..d2e96f60b650 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JSONParseSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JSONParseSpec.java @@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.java.util.common.parsers.Parser; +import javax.annotation.Nullable; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -73,6 +74,13 @@ public Parser makeParser() return new JSONPathParser(getFlattenSpec(), objectMapper); } + @Nullable + @Override + public InputFormat toInputFormat() + { + return new JsonInputFormat(getFlattenSpec(), getFeatureSpec()); + } + @Override public ParseSpec withTimestampSpec(TimestampSpec spec) { diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java new file mode 100644 index 000000000000..661aee7f3c52 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonParser.Feature; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.SplitReader; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; + +public class JsonInputFormat extends NestedInputFormat +{ + private final Map featureSpec; + private final ObjectMapper objectMapper; + + @JsonCreator + public JsonInputFormat( + @JsonProperty("flattenSpec") JSONPathSpec flattenSpec, + @JsonProperty("featureSpec") @Nullable Map featureSpec + ) + { + super(flattenSpec); + this.featureSpec = featureSpec == null ? Collections.emptyMap() : featureSpec; + this.objectMapper = new ObjectMapper(); + for (Entry entry : this.featureSpec.entrySet()) { + Feature feature = Feature.valueOf(entry.getKey()); + objectMapper.configure(feature, entry.getValue()); + } + } + + @JsonProperty + public Map getFeatureSpec() + { + return featureSpec; + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public SplitReader createReader(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec) + { + return new JsonReader(timestampSpec, dimensionsSpec, getFlattenSpec(), objectMapper); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + JsonInputFormat that = (JsonInputFormat) o; + return Objects.equals(featureSpec, that.featureSpec); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), featureSpec); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java new file mode 100644 index 000000000000..34020516a0e3 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.TextReader; +import org.apache.druid.java.util.common.parsers.JSONFlattenerMaker; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.java.util.common.parsers.ObjectFlattener; +import org.apache.druid.java.util.common.parsers.ObjectFlatteners; +import org.apache.druid.java.util.common.parsers.ParseException; + +import java.io.IOException; +import java.util.Map; + +public class JsonReader extends TextReader +{ + private final ObjectFlattener flattener; + private final ObjectMapper mapper; + + public JsonReader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + JSONPathSpec flattenSpec, + ObjectMapper mapper + ) + { + super(timestampSpec, dimensionsSpec); + this.flattener = ObjectFlatteners.create(flattenSpec, new JSONFlattenerMaker()); + this.mapper = mapper; + } + + @Override + public InputRow readLine(String line) throws IOException, ParseException + { + final JsonNode document = mapper.readValue(line, JsonNode.class); + final Map flattened = flattener.flatten(document); + return MapInputRowParser.parse( + getTimestampSpec(), + getDimensionsSpec(), + flattened + ); + } + + @Override + public int getNumHeaderLines() + { + return 0; + } + + @Override + public void processHeaderLine(String line) + { + // do nothing + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java new file mode 100644 index 000000000000..32126214480d --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.filefilter.TrueFileFilter; +import org.apache.commons.io.filefilter.WildcardFileFilter; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.FileNotFoundException; +import java.util.Iterator; +import java.util.Objects; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +public class LocalInputSource implements SplittableInputSource +{ + private final File baseDir; + private final String filter; + + @JsonCreator + public LocalInputSource( + @JsonProperty("baseDir") File baseDir, + @JsonProperty("filter") String filter + ) + { + this.baseDir = baseDir; + this.filter = filter; + } + + @JsonProperty + public File getBaseDir() + { + return baseDir; + } + + @JsonProperty + public String getFilter() + { + return filter; + } + + @Override + public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + { + return StreamSupport.stream(Spliterators.spliteratorUnknownSize(getFileIterator(), Spliterator.DISTINCT), false) + .map(InputSplit::new); + } + + @Override + public int getNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + { + final Iterator fileIterator = getFileIterator(); + int num = 0; + while (fileIterator.hasNext()) { + fileIterator.next(); + num++; + } + return num; + } + + private Iterator getFileIterator() + { + return FileUtils.iterateFiles( + Preconditions.checkNotNull(baseDir).getAbsoluteFile(), + new WildcardFileFilter(filter), + TrueFileFilter.INSTANCE + ); + } + + @Override + public SplittableInputSource withSplit(InputSplit split) + { + final File file = split.get(); + return new LocalInputSource(file.getParentFile(), file.getName()); + } + + @Override + public InputSourceReader reader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) + { + return new SplitIteratingReader<>( + timestampSpec, + dimensionsSpec, + inputFormat, + // reader() is supposed to be called in each task that creates segments. + // The task should already have only one split in parallel indexing, + // while there's no need to make splits using splitHintSpec in sequential indexing. + createSplits(inputFormat, null).map(split -> { + try { + return new FileSource(split); + } + catch (FileNotFoundException e) { + throw new RuntimeException(e); + } + }), + temporaryDirectory + ); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LocalInputSource source = (LocalInputSource) o; + return Objects.equals(baseDir, source.baseDir) && + Objects.equals(filter, source.filter); + } + + @Override + public int hashCode() + { + return Objects.hash(baseDir, filter); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java b/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java index 80db2c439658..9f605c383033 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; @@ -30,8 +29,10 @@ import org.apache.druid.java.util.common.parsers.ParseException; import org.joda.time.DateTime; +import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Set; public class MapInputRowParser implements InputRowParser> { @@ -50,18 +51,42 @@ public MapInputRowParser( @Override public List parseBatch(Map theMap) { - final List dimensions; - if (!this.dimensions.isEmpty()) { - dimensions = this.dimensions; + return ImmutableList.of( + parse( + parseSpec.getTimestampSpec(), + dimensions, + parseSpec.getDimensionsSpec().getDimensionExclusions(), + theMap + ) + ); + } + + public static InputRow parse( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + Map theMap + ) + { + return parse(timestampSpec, dimensionsSpec.getDimensionNames(), dimensionsSpec.getDimensionExclusions(), theMap); + } + + public static InputRow parse( + TimestampSpec timestampSpec, + List dimensions, + Set dimensionExclusions, + Map theMap + ) + { + final List dimensionsToUse; + if (!dimensions.isEmpty()) { + dimensionsToUse = dimensions; } else { - dimensions = Lists.newArrayList( - Sets.difference(theMap.keySet(), parseSpec.getDimensionsSpec().getDimensionExclusions()) - ); + dimensionsToUse = new ArrayList<>(Sets.difference(theMap.keySet(), dimensionExclusions)); } final DateTime timestamp; try { - timestamp = parseSpec.getTimestampSpec().extractTimestamp(theMap); + timestamp = timestampSpec.extractTimestamp(theMap); if (timestamp == null) { final String input = theMap.toString(); throw new NullPointerException( @@ -76,7 +101,7 @@ public List parseBatch(Map theMap) throw new ParseException(e, "Unparseable timestamp found! Event: %s", theMap); } - return ImmutableList.of(new MapBasedInputRow(timestamp, dimensions, theMap)); + return new MapBasedInputRow(timestamp, dimensionsToUse, theMap); } @JsonProperty diff --git a/core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java new file mode 100644 index 000000000000..d63c3629667f --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Abstract class for nested file formats such as JSON, ORC, etc. + * It has {@link JSONPathSpec}, which is internall called {@code flattenSpec}, to flatten the nested data structure. + */ +public abstract class NestedInputFormat implements InputFormat +{ + private final JSONPathSpec flattenSpec; + + protected NestedInputFormat(@Nullable JSONPathSpec flattenSpec) + { + this.flattenSpec = flattenSpec == null ? JSONPathSpec.DEFAULT : flattenSpec; + } + + @JsonProperty("flattenSpec") + public JSONPathSpec getFlattenSpec() + { + return flattenSpec; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + NestedInputFormat that = (NestedInputFormat) o; + return Objects.equals(flattenSpec, that.flattenSpec); + } + + @Override + public int hashCode() + { + return Objects.hash(flattenSpec); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/ParseSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/ParseSpec.java index b872219e7c8f..c8783c214f3e 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/ParseSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/ParseSpec.java @@ -27,8 +27,10 @@ import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.parsers.Parser; +import javax.annotation.Nullable; import java.util.List; +@Deprecated @ExtensionPoint @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "format") @JsonSubTypes(value = { @@ -75,6 +77,16 @@ public Parser makeParser() return null; } + /** + * Returns null if it's not implemented yet. + * This method (and maybe this class) will be removed in favor of {@link InputFormat} in the future. + */ + @Nullable + public InputFormat toInputFormat() + { + return null; + } + @PublicApi public ParseSpec withTimestampSpec(TimestampSpec spec) { diff --git a/core/src/main/java/org/apache/druid/data/input/impl/SplitIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/SplitIteratingReader.java new file mode 100644 index 000000000000..d9732df2daac --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/SplitIteratingReader.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.SplitReader; +import org.apache.druid.data.input.SplitSource; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.function.Function; +import java.util.stream.Stream; + +/** + * InputSourceReader iterating multiple {@link SplitSource}s. + */ +public class SplitIteratingReader implements InputSourceReader +{ + private final TimestampSpec timestampSpec; + private final DimensionsSpec dimensionsSpec; + private final InputFormat inputFormat; + private final Iterator> sourceIterator; + private final File temporaryDirectory; + + public SplitIteratingReader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + InputFormat inputFormat, + Stream> sourceStream, + File temporaryDirectory + ) + { + this.timestampSpec = timestampSpec; + this.dimensionsSpec = dimensionsSpec; + this.inputFormat = inputFormat; + this.sourceIterator = sourceStream.iterator(); + this.temporaryDirectory = temporaryDirectory; + } + + @Override + public CloseableIterator read() + { + return createIterator(reader -> { + try { + return reader.read(sourceIterator.next(), temporaryDirectory); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + @Override + public CloseableIterator sample() + { + return createIterator(reader -> { + try { + return reader.sample(sourceIterator.next(), temporaryDirectory); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + private CloseableIterator createIterator(Function> rowPopulator) + { + return new CloseableIterator() + { + CloseableIterator rowIterator = null; + + @Override + public boolean hasNext() + { + checkRowIterator(); + return rowIterator != null && rowIterator.hasNext(); + } + + @Override + public R next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return rowIterator.next(); + } + + private void checkRowIterator() + { + if (rowIterator == null || !rowIterator.hasNext()) { + try { + if (rowIterator != null) { + rowIterator.close(); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + if (sourceIterator.hasNext()) { + // SplitSampler is stateful and so a new one should be created per split. + final SplitReader splitReader = inputFormat.createReader(timestampSpec, dimensionsSpec); + rowIterator = rowPopulator.apply(splitReader); + } + } + } + + @Override + public void close() throws IOException + { + if (rowIterator != null) { + rowIterator.close(); + } + } + }; + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java new file mode 100644 index 000000000000..06bf3b68ffb0 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.stream.Stream; + +/** + * Splittable InputSource. ParallelIndexSupervisorTask can process {@link InputSplit}s in parallel. + */ +public interface SplittableInputSource extends InputSource +{ + @JsonIgnore + @Override + default boolean isSplittable() + { + return true; + } + + /** + * Creates a {@link Stream} of {@link InputSplit}s. The returned stream is supposed to be evaluated lazily to avoid + * consuming too much memory. + * Note that this interface also has {@link #getNumSplits} which is related to this method. The implementations + * should be careful to NOT cache the created splits in memory. + * + * Implementations can consider {@link InputFormat#isSplittable()} and {@link SplitHintSpec} to create splits + * in the same way with {@link #getNumSplits}. + */ + Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException; + + /** + * Returns the total number of splits to be created via {@link #createSplits}. + * This method can be expensive since it needs to iterate all directories or whatever substructure + * to find all input objects. + * + * Implementations can consider {@link InputFormat#isSplittable()} and {@link SplitHintSpec} to find splits + * in the same way with {@link #createSplits}. + */ + int getNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException; + + /** + * Helper method for ParallelIndexSupervisorTask. + * Most of implementations can simply create a new instance with the given split. + */ + SplittableInputSource withSplit(InputSplit split); +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java index 3f848d4d4aec..08db77572b49 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.parsers.TimestampParser; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.util.List; import java.util.Map; import java.util.Objects; @@ -57,10 +58,10 @@ private static class ParseCtx @JsonCreator public TimestampSpec( - @JsonProperty("column") String timestampColumn, - @JsonProperty("format") String format, + @JsonProperty("column") @Nullable String timestampColumn, + @JsonProperty("format") @Nullable String format, // this value should never be set for production data; the data loader uses it before a timestamp column is chosen - @JsonProperty("missingValue") DateTime missingValue + @JsonProperty("missingValue") @Nullable DateTime missingValue ) { this.timestampColumn = (timestampColumn == null) ? DEFAULT_COLUMN : timestampColumn; diff --git a/core/src/main/java/org/apache/druid/indexer/Checks.java b/core/src/main/java/org/apache/druid/indexer/Checks.java index 424ca6e79ac8..7153b7476371 100644 --- a/core/src/main/java/org/apache/druid/indexer/Checks.java +++ b/core/src/main/java/org/apache/druid/indexer/Checks.java @@ -41,7 +41,7 @@ public static Property checkOneNotNullOrEmpty(List> propertie } } if (nonNullProperty == null) { - throw new IAE("At most one of %s must be present", properties); + throw new IAE("At least one of %s must be present", properties); } return nonNullProperty; } diff --git a/core/src/main/java/org/apache/druid/indexer/Property.java b/core/src/main/java/org/apache/druid/indexer/Property.java index 9f9467ff9741..8c0f49481ee0 100644 --- a/core/src/main/java/org/apache/druid/indexer/Property.java +++ b/core/src/main/java/org/apache/druid/indexer/Property.java @@ -19,6 +19,7 @@ package org.apache.druid.indexer; +import javax.annotation.Nullable; import java.util.Collection; import java.util.Objects; @@ -28,9 +29,10 @@ public class Property { private final String name; + @Nullable private final T value; - public Property(String name, T value) + public Property(String name, @Nullable T value) { this.name = name; this.value = value; @@ -41,6 +43,7 @@ public String getName() return name; } + @Nullable public T getValue() { return value; diff --git a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java index e0d8898af8a2..aedf67166b0e 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java @@ -306,6 +306,33 @@ public static long copyLarge( } } + public static long copyLarge( + InputStream inputStream, + File outFile, + byte[] fetchBuffer, + Predicate retryCondition, + int numRetries, + String messageOnRetry + ) throws IOException + { + try { + return RetryUtils.retry( + () -> { + try (OutputStream out = new FileOutputStream(outFile)) { + return IOUtils.copyLarge(inputStream, out, fetchBuffer); + } + }, + retryCondition, + outFile::delete, + numRetries, + messageOnRetry + ); + } + catch (Exception e) { + throw new IOException(e); + } + } + public interface OutputStreamConsumer { T apply(OutputStream outputStream) throws IOException; diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/AbstractFlatTextFormatParser.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/AbstractFlatTextFormatParser.java index 9039106ff9d6..3d54443333de 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/AbstractFlatTextFormatParser.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/AbstractFlatTextFormatParser.java @@ -21,14 +21,13 @@ import com.google.common.base.Function; import com.google.common.base.Splitter; -import com.google.common.base.Strings; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import org.apache.druid.data.input.impl.CsvReader; import org.apache.druid.java.util.common.collect.Utils; import javax.annotation.Nullable; import java.io.IOException; -import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -53,7 +52,6 @@ public String getDefaultDelimiter() } private final String listDelimiter; - private final Splitter listSplitter; private final Function valueFunction; private final boolean hasHeaderRow; private final int maxSkipHeaderRows; @@ -70,8 +68,7 @@ public AbstractFlatTextFormatParser( ) { this.listDelimiter = listDelimiter != null ? listDelimiter : Parsers.DEFAULT_LIST_DELIMITER; - this.listSplitter = Splitter.on(this.listDelimiter); - this.valueFunction = ParserUtils.getMultiValueFunction(this.listDelimiter, this.listSplitter); + this.valueFunction = ParserUtils.getMultiValueFunction(this.listDelimiter, Splitter.on(this.listDelimiter)); this.hasHeaderRow = hasHeaderRow; this.maxSkipHeaderRows = maxSkipHeaderRows; @@ -103,16 +100,7 @@ public List getFieldNames() public void setFieldNames(final Iterable fieldNames) { if (fieldNames != null) { - final List fieldsList = Lists.newArrayList(fieldNames); - this.fieldNames = new ArrayList<>(fieldsList.size()); - for (int i = 0; i < fieldsList.size(); i++) { - if (Strings.isNullOrEmpty(fieldsList.get(i))) { - this.fieldNames.add(ParserUtils.getDefaultColumnName(i)); - } else { - this.fieldNames.add(fieldsList.get(i)); - } - } - ParserUtils.validateFields(this.fieldNames); + this.fieldNames = CsvReader.findOrCreateColumnNames(Lists.newArrayList(fieldNames)); } } diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java index b9164b2b3d4e..56b5816f4bb4 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java @@ -20,10 +20,41 @@ package org.apache.druid.java.util.common.parsers; import java.io.Closeable; +import java.io.IOException; import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.function.Function; /** */ public interface CloseableIterator extends Iterator, Closeable { + default CloseableIterator map(Function mapFunction) + { + final CloseableIterator delegate = this; + + return new CloseableIterator() + { + @Override + public boolean hasNext() + { + return delegate.hasNext(); + } + + @Override + public R next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return mapFunction.apply(delegate.next()); + } + + @Override + public void close() throws IOException + { + delegate.close(); + } + }; + } } diff --git a/core/src/main/java/org/apache/druid/timeline/DataSegment.java b/core/src/main/java/org/apache/druid/timeline/DataSegment.java index 6475dc49c3f7..e580b9754481 100644 --- a/core/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/core/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -102,6 +102,7 @@ public static class PruneSpecsHolder private final CompactionState lastCompactionState; private final long size; + @VisibleForTesting public DataSegment( SegmentId segmentId, Map loadSpec, diff --git a/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java b/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java new file mode 100644 index 000000000000..f9e9ed1700c2 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import com.google.common.collect.Iterables; +import org.apache.druid.data.input.impl.CSVParseSpec; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputFormat; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +public class FirehoseFactoryToInputSourceAdaptorTest +{ + @Test + public void testUnimplementedInputFormat() throws IOException + { + final List lines = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + lines.add(StringUtils.format("%d,name_%d,%d", 20190101 + i, i, i + 100)); + } + final TestFirehoseFactory firehoseFactory = new TestFirehoseFactory(lines); + final StringInputRowParser inputRowParser = new StringInputRowParser( + new TestCsvParseSpec( + new TimestampSpec(null, "yyyyMMdd", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "name", "score"))), + ",", + Arrays.asList("timestamp", "name", "score"), + false, + 0 + ), + StringUtils.UTF8_STRING + ); + final FirehoseFactoryToInputSourceAdaptor inputSourceAdaptor = new FirehoseFactoryToInputSourceAdaptor( + firehoseFactory, + inputRowParser + ); + final InputSourceReader reader = inputSourceAdaptor.reader( + inputRowParser.getParseSpec().getTimestampSpec(), + inputRowParser.getParseSpec().getDimensionsSpec(), + null, + null + ); + final List result = new ArrayList<>(); + try (CloseableIterator iterator = reader.read()) { + while (iterator.hasNext()) { + result.add(iterator.next()); + } + } + Assert.assertEquals(10, result.size()); + for (int i = 0; i < 10; i++) { + Assert.assertEquals(DateTimes.of(StringUtils.format("2019-01-%02d", 1 + i)), result.get(i).getTimestamp()); + Assert.assertEquals( + StringUtils.format("name_%d", i), + Iterables.getOnlyElement(result.get(i).getDimension("name")) + ); + Assert.assertEquals( + StringUtils.format("%d", i + 100), + Iterables.getOnlyElement(result.get(i).getDimension("score")) + ); + } + } + + private static class TestCsvParseSpec extends CSVParseSpec + { + private TestCsvParseSpec( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + String listDelimiter, + List columns, + boolean hasHeaderRow, + int skipHeaderRows + ) + { + super(timestampSpec, dimensionsSpec, listDelimiter, columns, hasHeaderRow, skipHeaderRows); + } + + @Nullable + @Override + public InputFormat toInputFormat() + { + return null; + } + } + + private static class TestFirehoseFactory implements FirehoseFactory + { + private final List lines; + + private TestFirehoseFactory(List lines) + { + this.lines = lines; + } + + @Override + public Firehose connect(StringInputRowParser parser, @Nullable File temporaryDirectory) throws ParseException + { + return new Firehose() + { + final Iterator iterator = lines.iterator(); + + @Override + public boolean hasMore() + { + return iterator.hasNext(); + } + + @Override + public InputRow nextRow() + { + return parser.parse(iterator.next()); + } + + @Override + public void close() + { + // do nothing + } + }; + } + } +} diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java new file mode 100644 index 000000000000..abae7fa21467 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; + +public class CsvInputFormatTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testSerde() throws IOException + { + final ObjectMapper mapper = new ObjectMapper(); + final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), "|", true, 10); + final byte[] bytes = mapper.writeValueAsBytes(format); + final CsvInputFormat fromJson = (CsvInputFormat) mapper.readValue(bytes, InputFormat.class); + Assert.assertEquals(format, fromJson); + } + + @Test + public void testColumnMissing() + { + final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), ",", false, 0); + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("column[b] not in columns"); + format.createReader( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b"))) + ); + } + + @Test + public void testComma() + { + final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), ",", false, 0); + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("column[a,] not in columns"); + format.createReader( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("a,", "b"))) + ); + } +} diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java new file mode 100644 index 000000000000..a28ff16e1e99 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.SplitReader; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +public class CsvReaderTest +{ + private static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec("ts", "auto", null); + private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "name")) + ); + + @Test + public void testWithoutHeaders() throws IOException + { + final ByteSource source = writeData( + ImmutableList.of( + "2019-01-01T00:00:10Z,name_1,5", + "2019-01-01T00:00:20Z,name_2,10", + "2019-01-01T00:00:30Z,name_3,15" + ) + ); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "score"), null, false, 0); + assertResult(source, format); + } + + @Test + public void testFindColumn() throws IOException + { + final ByteSource source = writeData( + ImmutableList.of( + "ts,name,score", + "2019-01-01T00:00:10Z,name_1,5", + "2019-01-01T00:00:20Z,name_2,10", + "2019-01-01T00:00:30Z,name_3,15" + ) + ); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), null, true, 0); + assertResult(source, format); + } + + @Test + public void testSkipHeaders() throws IOException + { + final ByteSource source = writeData( + ImmutableList.of( + "this,is,a,row,to,skip", + "2019-01-01T00:00:10Z,name_1,5", + "2019-01-01T00:00:20Z,name_2,10", + "2019-01-01T00:00:30Z,name_3,15" + ) + ); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "score"), null, false, 1); + assertResult(source, format); + } + + @Test + public void testFindColumnAndSkipHeaders() throws IOException + { + final ByteSource source = writeData( + ImmutableList.of( + "ts,name,score", + "this,is,a,row,to,skip", + "2019-01-01T00:00:10Z,name_1,5", + "2019-01-01T00:00:20Z,name_2,10", + "2019-01-01T00:00:30Z,name_3,15" + ) + ); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), null, true, 1); + assertResult(source, format); + } + + @Test + public void testMultiValues() throws IOException + { + final ByteSource source = writeData( + ImmutableList.of( + "ts,name,score", + "2019-01-01T00:00:10Z,name_1,5|1", + "2019-01-01T00:00:20Z,name_2,10|2", + "2019-01-01T00:00:30Z,name_3,15|3" + ) + ); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), "|", true, 0); + final SplitReader reader = format.createReader(TIMESTAMP_SPEC, DIMENSIONS_SPEC); + int numResults = 0; + try (CloseableIterator iterator = reader.read(source, null)) { + while (iterator.hasNext()) { + final InputRow row = iterator.next(); + Assert.assertEquals( + DateTimes.of(StringUtils.format("2019-01-01T00:00:%02dZ", (numResults + 1) * 10)), + row.getTimestamp() + ); + Assert.assertEquals( + StringUtils.format("name_%d", numResults + 1), + Iterables.getOnlyElement(row.getDimension("name")) + ); + Assert.assertEquals( + ImmutableList.of(Integer.toString((numResults + 1) * 5), Integer.toString(numResults + 1)), + row.getDimension("score") + ); + numResults++; + } + Assert.assertEquals(3, numResults); + } + } + + private ByteSource writeData(List lines) throws IOException + { + final List byteLines = lines.stream() + .map(line -> StringUtils.toUtf8(line + "\n")) + .collect(Collectors.toList()); + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream( + byteLines.stream().mapToInt(bytes -> bytes.length).sum() + ); + for (byte[] bytes : byteLines) { + outputStream.write(bytes); + } + return new ByteSource(outputStream.toByteArray()); + } + + private void assertResult(ByteSource source, CsvInputFormat format) throws IOException + { + final SplitReader reader = format.createReader(TIMESTAMP_SPEC, DIMENSIONS_SPEC); + int numResults = 0; + try (CloseableIterator iterator = reader.read(source, null)) { + while (iterator.hasNext()) { + final InputRow row = iterator.next(); + Assert.assertEquals( + DateTimes.of(StringUtils.format("2019-01-01T00:00:%02dZ", (numResults + 1) * 10)), + row.getTimestamp() + ); + Assert.assertEquals( + StringUtils.format("name_%d", numResults + 1), + Iterables.getOnlyElement(row.getDimension("name")) + ); + Assert.assertEquals( + Integer.toString((numResults + 1) * 5), + Iterables.getOnlyElement(row.getDimension("score")) + ); + numResults++; + } + Assert.assertEquals(3, numResults); + } + } +} diff --git a/core/src/test/java/org/apache/druid/data/input/impl/HttpInputSourceTest.java b/core/src/test/java/org/apache/druid/data/input/impl/HttpInputSourceTest.java new file mode 100644 index 000000000000..61be2cc1b206 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/HttpInputSourceTest.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.metadata.DefaultPasswordProvider; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.net.URI; + +public class HttpInputSourceTest +{ + @Test + public void testSerde() throws IOException + { + final ObjectMapper mapper = new ObjectMapper(); + final HttpInputSource source = new HttpInputSource( + ImmutableList.of(URI.create("http://test.com/http-test")), + "myName", + new DefaultPasswordProvider("myPassword") + ); + final byte[] json = mapper.writeValueAsBytes(source); + final HttpInputSource fromJson = (HttpInputSource) mapper.readValue(json, InputSource.class); + Assert.assertEquals(source, fromJson); + } +} diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JsonInputFormatTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JsonInputFormatTest.java new file mode 100644 index 000000000000..239ebc4b4ba7 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/JsonInputFormatTest.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.core.JsonParser.Feature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; +import org.apache.druid.java.util.common.parsers.JSONPathFieldType; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class JsonInputFormatTest +{ + @Test + public void testSerde() throws IOException + { + final ObjectMapper mapper = new ObjectMapper(); + final JsonInputFormat format = new JsonInputFormat( + new JSONPathSpec( + false, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + ImmutableMap.of(Feature.ALLOW_COMMENTS.name(), true, Feature.ALLOW_UNQUOTED_FIELD_NAMES.name(), false) + ); + final byte[] bytes = mapper.writeValueAsBytes(format); + final JsonInputFormat fromJson = (JsonInputFormat) mapper.readValue(bytes, InputFormat.class); + Assert.assertEquals(format, fromJson); + } +} diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java new file mode 100644 index 000000000000..c074bcfd1029 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.SplitReader; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; +import org.apache.druid.java.util.common.parsers.JSONPathFieldType; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; + +public class JsonReaderTest +{ + @Test + public void testParseRow() throws IOException + { + final JsonInputFormat format = new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"), + new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2") + ) + ), + null + ); + + final ByteSource source = new ByteSource( + StringUtils.toUtf8("{\"timestamp\":\"2019-01-01\",\"bar\":null,\"foo\":\"x\",\"baz\":4,\"o\":{\"mg\":1}}") + ); + + final SplitReader reader = format.createReader( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))) + ); + try (CloseableIterator iterator = reader.read(source, null)) { + while (iterator.hasNext()) { + final InputRow row = iterator.next(); + Assert.assertEquals(DateTimes.of("2019-01-01"), row.getTimestamp()); + Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz"))); + Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg"))); + Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg"))); + + Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); + Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); + Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + } + } + } + + @Test + public void testParseRowWithConditional() throws IOException + { + final JsonInputFormat format = new JsonInputFormat( + new JSONPathSpec( + true, + ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.PATH, "foo", "$.[?(@.maybe_object)].maybe_object.foo.test"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "baz", "$.maybe_object_2.foo.test"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "bar", "$.[?(@.something_else)].something_else.foo") + ) + ), + null + ); + + final ByteSource source = new ByteSource( + StringUtils.toUtf8("{\"timestamp\":\"2019-01-01\",\"something_else\": {\"foo\": \"test\"}}") + ); + + final SplitReader reader = format.createReader( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))) + ); + + try (CloseableIterator iterator = reader.read(source, null)) { + while (iterator.hasNext()) { + final InputRow row = iterator.next(); + Assert.assertEquals("test", Iterables.getOnlyElement(row.getDimension("bar"))); + Assert.assertEquals(Collections.emptyList(), row.getDimension("foo")); + Assert.assertTrue(row.getDimension("baz").isEmpty()); + } + } + } +} diff --git a/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java b/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java new file mode 100644 index 000000000000..3852d10f08e4 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputSource; +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; + +public class LocalInputSourceTest +{ + @Test + public void testSerde() throws IOException + { + final ObjectMapper mapper = new ObjectMapper(); + final LocalInputSource source = new LocalInputSource( + new File("myFile").getAbsoluteFile(), + "myFilter" + ); + final byte[] json = mapper.writeValueAsBytes(source); + final LocalInputSource fromJson = (LocalInputSource) mapper.readValue(json, InputSource.class); + Assert.assertEquals(source, fromJson); + } +} diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java new file mode 100644 index 000000000000..8b7653ffb91f --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import org.apache.druid.data.input.FirehoseFactory; + +public class NoopFirehoseFactory implements FirehoseFactory +{ + @Override + public String toString() + { + return "NoopFirehoseFactory{}"; + } +} diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java new file mode 100644 index 000000000000..5e69e1e77c57 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import org.apache.druid.data.input.SplitReader; + +public class NoopInputFormat implements InputFormat +{ + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public SplitReader createReader(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec) + { + throw new UnsupportedOperationException(); + } + + @Override + public String toString() + { + return "NoopInputFormat{}"; + } +} diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java new file mode 100644 index 000000000000..1f6bd82dc05c --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; + +import javax.annotation.Nullable; +import java.io.File; + +public class NoopInputSource implements InputSource +{ + @Override + public InputSourceReader reader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) + { + return null; + } + + @Override + public String toString() + { + return "NoopInputSource{}"; + } +} diff --git a/core/src/test/java/org/apache/druid/data/input/impl/SplitIteratingReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/SplitIteratingReaderTest.java new file mode 100644 index 000000000000..41f417c3e746 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/SplitIteratingReaderTest.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.impl; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.io.Writer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; + +public class SplitIteratingReaderTest +{ + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Test + public void test() throws IOException + { + final List files = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + final File file = temporaryFolder.newFile("test_" + i); + files.add(file); + try (Writer writer = Files.newBufferedWriter(file.toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("%d,%s,%d\n", 20190101 + i, "name_" + i, i)); + writer.write(StringUtils.format("%d,%s,%d", 20190102 + i, "name_" + (i + 1), i + 1)); + } + } + final SplitIteratingReader firehose = new SplitIteratingReader<>( + new TimestampSpec("time", "yyyyMMdd", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "name", "score")) + ), + new CsvInputFormat( + ImmutableList.of("time", "name", "score"), + null, + false, + 0 + ), + files.stream().flatMap(file -> { + try { + return ImmutableList.of(new FileSource(new InputSplit<>(file))).stream(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }), + temporaryFolder.newFolder() + ); + + try (CloseableIterator iterator = firehose.read()) { + int i = 0; + while (iterator.hasNext()) { + InputRow row = iterator.next(); + Assert.assertEquals(DateTimes.of(StringUtils.format("2019-01-%02d", i + 1)), row.getTimestamp()); + Assert.assertEquals(StringUtils.format("name_%d", i), Iterables.getOnlyElement(row.getDimension("name"))); + Assert.assertEquals(Integer.toString(i), Iterables.getOnlyElement(row.getDimension("score"))); + + Assert.assertTrue(iterator.hasNext()); + row = iterator.next(); + Assert.assertEquals(DateTimes.of(StringUtils.format("2019-01-%02d", i + 2)), row.getTimestamp()); + Assert.assertEquals(StringUtils.format("name_%d", i + 1), Iterables.getOnlyElement(row.getDimension("name"))); + Assert.assertEquals(Integer.toString(i + 1), Iterables.getOnlyElement(row.getDimension("score"))); + i++; + } + } + } +} diff --git a/docs/ingestion/tasks.md b/docs/ingestion/tasks.md index fa120a74ad42..e678a61c67dd 100644 --- a/docs/ingestion/tasks.md +++ b/docs/ingestion/tasks.md @@ -164,7 +164,7 @@ Only batch tasks have the DETERMINE_PARTITIONS phase. Realtime tasks such as tho the `rowStats` map contains information about row counts. There is one entry for each ingestion phase. The definitions of the different row counts are shown below: * `processed`: Number of rows successfully ingested without parsing errors * `processedWithError`: Number of rows that were ingested, but contained a parsing error within one or more columns. This typically occurs where input rows have a parseable structure but invalid types for columns, such as passing in a non-numeric String value for a numeric column. -* `thrownAway`: Number of rows skipped. This includes rows with timestamps that were outside of the ingestion task's defined time interval and rows that were filtered out with a [`transformSpec`](index.md#transformspec). +* `thrownAway`: Number of rows skipped. This includes rows with timestamps that were outside of the ingestion task's defined time interval and rows that were filtered out with a [`transformSpec`](index.md#transformspec), but doesn't include the rows skipped by explicit user configurations. For example, the rows skipped by `skipHeaderRows` or `hasHeaderRow` in the CSV format are not counted. * `unparseable`: Number of rows that could not be parsed at all and were discarded. This tracks input rows without a parseable structure, such as passing in non-JSON data when using a JSON parser. The `errorMsg` field shows a message describing the error that caused a task to fail. It will be null if the task was successful. diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index a1b14a070577..1e959266c945 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -138,7 +138,7 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() //noinspection unchecked return new IncrementalPublishingKafkaIndexTaskRunner( this, - dataSchema.getParser(), + Preconditions.checkNotNull(dataSchema.getParser(), "inputRowParser"), authorizerMapper, chatHandlerProvider, savedParseExceptions, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index b2209eaf11c8..a53111bf37e3 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -22,6 +22,7 @@ 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 org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.TaskResource; @@ -77,7 +78,7 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() //noinspection unchecked return new KinesisIndexTaskRunner( this, - dataSchema.getParser(), + Preconditions.checkNotNull(dataSchema.getParser(), "inputRowParser"), authorizerMapper, chatHandlerProvider, savedParseExceptions, diff --git a/extensions-core/orc-extensions/pom.xml b/extensions-core/orc-extensions/pom.xml index c739bedba4ce..ec8c8a752ea8 100644 --- a/extensions-core/orc-extensions/pom.xml +++ b/extensions-core/orc-extensions/pom.xml @@ -52,7 +52,7 @@ org.apache.hadoop hadoop-client ${hadoop.compile.version} - provided + compile org.apache.orc @@ -178,12 +178,17 @@ org.apache.hadoop hadoop-mapreduce-client-core - provided + compile + + + org.apache.hadoop + hadoop-hdfs-client + compile org.apache.hadoop hadoop-common - provided + compile com.google.inject diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java index 304384ca8986..24d5f556475c 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java @@ -467,7 +467,7 @@ static boolean isWrappedListPrimitive(Object o) return false; } - private boolean binaryAsString; + private final boolean binaryAsString; ParquetGroupConverter(boolean binaryAsString) { diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java index b5777e917bbc..80a44676f94f 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java @@ -346,7 +346,7 @@ public boolean isCombineText() public InputRowParser getParser() { - return schema.getDataSchema().getParser(); + return Preconditions.checkNotNull(schema.getDataSchema().getParser(), "inputRowParser"); } public HadoopyShardSpec getShardSpec(Bucket bucket) @@ -579,8 +579,9 @@ public void intoConfiguration(Job job) public void verify() { Preconditions.checkNotNull(schema.getDataSchema().getDataSource(), "dataSource"); + Preconditions.checkNotNull(schema.getDataSchema().getParser(), "inputRowParser"); Preconditions.checkNotNull(schema.getDataSchema().getParser().getParseSpec(), "parseSpec"); - Preconditions.checkNotNull(schema.getDataSchema().getParser().getParseSpec().getTimestampSpec(), "timestampSpec"); + Preconditions.checkNotNull(schema.getDataSchema().getNonNullTimestampSpec(), "timestampSpec"); Preconditions.checkNotNull(schema.getDataSchema().getGranularitySpec(), "granularitySpec"); Preconditions.checkNotNull(pathSpec, "inputSpec"); Preconditions.checkNotNull(schema.getTuningConfig().getWorkingPath(), "workingPath"); diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java index 5ee8d5e11de8..f3ff85df5295 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java @@ -294,8 +294,8 @@ private static IncrementalIndex makeIncrementalIndex( final HadoopTuningConfig tuningConfig = config.getSchema().getTuningConfig(); final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder() .withMinTimestamp(theBucket.time.getMillis()) - .withTimestampSpec(config.getSchema().getDataSchema().getParser().getParseSpec().getTimestampSpec()) - .withDimensionsSpec(config.getSchema().getDataSchema().getParser()) + .withTimestampSpec(config.getSchema().getDataSchema().getNonNullTimestampSpec()) + .withDimensionsSpec(config.getSchema().getDataSchema().getNonNullDimensionsSpec()) .withQueryGranularity(config.getSchema().getDataSchema().getGranularitySpec().getQueryGranularity()) .withMetrics(aggs) .withRollup(config.getSchema().getDataSchema().getGranularitySpec().isRollup()) @@ -341,11 +341,7 @@ protected void setup(Context context) aggsForSerializingSegmentInputRow[i] = aggregators[i].getCombiningFactory(); } } - typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema() - .getDataSchema() - .getParser() - .getParseSpec() - .getDimensionsSpec()); + typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema().getDataSchema().getNonNullDimensionsSpec()); } @Override @@ -431,11 +427,7 @@ protected void setup(Context context) for (int i = 0; i < aggregators.length; ++i) { combiningAggs[i] = aggregators[i].getCombiningFactory(); } - typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema() - .getDataSchema() - .getParser() - .getParseSpec() - .getDimensionsSpec()); + typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema().getDataSchema().getNonNullDimensionsSpec()); } @Override @@ -631,11 +623,7 @@ protected void setup(Context context) metricNames.add(aggregators[i].getName()); combiningAggs[i] = aggregators[i].getCombiningFactory(); } - typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema() - .getDataSchema() - .getParser() - .getParseSpec() - .getDimensionsSpec()); + typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema().getDataSchema().getNonNullDimensionsSpec()); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index 3287925104b5..1bd29dd6d195 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -294,7 +294,7 @@ public IndexMergerV9 getIndexMergerV9() return indexMergerV9; } - public File getFirehoseTemporaryDir() + public File getIndexingTmpDir() { return new File(taskWorkDir, "firehose"); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index d6850fe30e0f..5cbede11e561 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; +import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -273,7 +274,7 @@ public TaskStatus run(final TaskToolbox toolbox) this.metrics = fireDepartmentForMetrics.getMetrics(); final Supplier committerSupplier = Committers.nilSupplier(); - final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); + final File firehoseTempDir = toolbox.getIndexingTmpDir(); DiscoveryDruidNode discoveryDruidNode = createDiscoveryDruidNode(toolbox); @@ -350,7 +351,10 @@ public TaskStatus run(final TaskToolbox toolbox) // Skip connecting firehose if we've been stopped before we got started. synchronized (this) { if (!gracefullyStopped) { - firehose = firehoseFactory.connect(spec.getDataSchema().getParser(), firehoseTempDir); + firehose = firehoseFactory.connect( + Preconditions.checkNotNull(spec.getDataSchema().getParser(), "inputRowParser"), + firehoseTempDir + ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 115ff8103b08..826bff025b18 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -542,6 +542,8 @@ private static ParallelIndexIOConfig createIoConfig( segmentLoaderFactory, retryPolicyFactory ), + null, + null, false ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 51b44f18d117..d9fc9de6c130 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -22,22 +22,32 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; 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.Lists; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.Rows; +import org.apache.druid.data.input.impl.InputFormat; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.hll.HyperLogLogCollector; +import org.apache.druid.indexer.Checks; import org.apache.druid.indexer.IngestionState; +import org.apache.druid.indexer.Property; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -53,6 +63,7 @@ import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; @@ -61,10 +72,11 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.IOConfig; import org.apache.druid.segment.indexing.IngestionSpec; import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.TuningConfig; @@ -461,11 +473,13 @@ public TaskStatus runTask(final TaskToolbox toolbox) .bucketIntervals() .isPresent(); - final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); + final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource( + ingestionSchema.getDataSchema().getParser() + ); - final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); - // Firehose temporary directory is automatically removed when this IndexTask completes. - FileUtils.forceMkdir(firehoseTempDir); + final File tmpDir = toolbox.getIndexingTmpDir(); + // Temporary directory is automatically removed when this IndexTask completes. + FileUtils.forceMkdir(tmpDir); ingestionState = IngestionState.DETERMINE_PARTITIONS; @@ -474,8 +488,8 @@ public TaskStatus runTask(final TaskToolbox toolbox) final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec(); final Map> allocateSpec = determineShardSpecs( toolbox, - firehoseFactory, - firehoseTempDir, + inputSource, + tmpDir, partitionsSpec ); final List allocateIntervals = new ArrayList<>(allocateSpec.keySet()); @@ -498,8 +512,8 @@ public TaskStatus runTask(final TaskToolbox toolbox) toolbox, dataSchema, allocateSpec, - firehoseFactory, - firehoseTempDir, + inputSource, + tmpDir, partitionsSpec ); } @@ -585,8 +599,8 @@ private Map getTaskCompletionRowStats() */ private Map> determineShardSpecs( final TaskToolbox toolbox, - final FirehoseFactory firehoseFactory, - final File firehoseTempDir, + final InputSource inputSource, + final File tmpDir, final PartitionsSpec nonNullPartitionsSpec ) throws IOException { @@ -616,8 +630,8 @@ private Map> determineShardSpecs( return createShardSpecsFromInput( jsonMapper, ingestionSchema, - firehoseFactory, - firehoseTempDir, + inputSource, + tmpDir, granularitySpec, nonNullPartitionsSpec, determineIntervals @@ -628,8 +642,8 @@ private Map> determineShardSpecs( private Map> createShardSpecsFromInput( ObjectMapper jsonMapper, IndexIngestionSpec ingestionSchema, - FirehoseFactory firehoseFactory, - File firehoseTempDir, + InputSource inputSource, + File tmpDir, GranularitySpec granularitySpec, PartitionsSpec nonNullPartitionsSpec, boolean determineIntervals @@ -641,8 +655,8 @@ private Map> createShardSpecsFromInput final Map> hllCollectors = collectIntervalsAndShardSpecs( jsonMapper, ingestionSchema, - firehoseFactory, - firehoseTempDir, + inputSource, + tmpDir, granularitySpec, nonNullPartitionsSpec, determineIntervals @@ -693,8 +707,8 @@ private Map> createShardSpecsFromInput private Map> collectIntervalsAndShardSpecs( ObjectMapper jsonMapper, IndexIngestionSpec ingestionSchema, - FirehoseFactory firehoseFactory, - File firehoseTempDir, + InputSource inputSource, + File tmpDir, GranularitySpec granularitySpec, PartitionsSpec nonNullPartitionsSpec, boolean determineIntervals @@ -704,14 +718,19 @@ private Map> collectIntervalsAndShardSp Comparators.intervalsByStartThenEnd() ); final Granularity queryGranularity = granularitySpec.getQueryGranularity(); + final InputSourceReader inputSourceReader = ingestionSchema.getDataSchema().getTransformSpec().decorate( + inputSource.reader( + ingestionSchema.getDataSchema().getNonNullTimestampSpec(), + ingestionSchema.getDataSchema().getNonNullDimensionsSpec(), + getInputFormat(ingestionSchema), + tmpDir + ) + ); - try ( - final Firehose firehose = firehoseFactory.connect(ingestionSchema.getDataSchema().getParser(), firehoseTempDir) - ) { - - while (firehose.hasMore()) { + try (final CloseableIterator inputRowIterator = inputSourceReader.read()) { + while (inputRowIterator.hasNext()) { try { - final InputRow inputRow = firehose.nextRow(); + final InputRow inputRow = inputRowIterator.next(); // The null inputRow means the caller must skip this row. if (inputRow == null) { @@ -831,8 +850,8 @@ private TaskStatus generateAndPublishSegments( final TaskToolbox toolbox, final DataSchema dataSchema, final Map> allocateSpec, - final FirehoseFactory firehoseFactory, - final File firehoseTempDir, + final InputSource inputSource, + final File tmpDir, final PartitionsSpec partitionsSpec ) throws IOException, InterruptedException { @@ -880,19 +899,20 @@ private TaskStatus generateAndPublishSegments( try (final BatchAppenderatorDriver driver = BatchAppenderators.newDriver(appenderator, toolbox, segmentAllocator)) { driver.startJob(); - final FiniteFirehoseProcessor firehoseProcessor = new FiniteFirehoseProcessor( + final InputSourceProcessor inputSourceProcessor = new InputSourceProcessor( buildSegmentsMeters, buildSegmentsSavedParseExceptions, tuningConfig.isLogParseExceptions(), tuningConfig.getMaxParseExceptions(), pushTimeout ); - firehoseProcessor.process( + inputSourceProcessor.process( dataSchema, driver, partitionsSpec, - firehoseFactory, - firehoseTempDir, + inputSource, + getInputFormat(ingestionSchema), + tmpDir, segmentAllocator ); @@ -1000,6 +1020,14 @@ ShardSpec getShardSpec(Interval interval, InputRow row) } } + private static InputFormat getInputFormat(IndexIngestionSpec ingestionSchema) + { + final InputRowParser parser = ingestionSchema.getDataSchema().getParser(); + return ingestionSchema.getIOConfig().getNonNullInputFormat( + parser == null ? null : parser.getParseSpec() + ); + } + public static class IndexIngestionSpec extends IngestionSpec { private final DataSchema dataSchema; @@ -1015,6 +1043,16 @@ public IndexIngestionSpec( { super(dataSchema, ioConfig, tuningConfig); + Checks.checkOneNotNullOrEmpty( + ImmutableList.of( + new Property<>("parser", dataSchema.getParserMap()), + new Property<>("inputFormat", ioConfig.getInputFormat()) + ) + ); + if (dataSchema.getParserMap() != null && ioConfig.getInputSource() != null) { + throw new IAE("Cannot use parser and inputSource together. Try use inputFormat instead of parser."); + } + this.dataSchema = dataSchema; this.ioConfig = ioConfig; this.tuningConfig = tuningConfig == null ? new IndexTuningConfig() : tuningConfig; @@ -1043,30 +1081,87 @@ public IndexTuningConfig getTuningConfig() } @JsonTypeName("index") - public static class IndexIOConfig implements IOConfig + public static class IndexIOConfig implements BatchIOConfig { private static final boolean DEFAULT_APPEND_TO_EXISTING = false; private final FirehoseFactory firehoseFactory; + private final InputSource inputSource; + private final InputFormat inputFormat; private final boolean appendToExisting; @JsonCreator public IndexIOConfig( - @JsonProperty("firehose") FirehoseFactory firehoseFactory, + @Deprecated @JsonProperty("firehose") @Nullable FirehoseFactory firehoseFactory, + @JsonProperty("inputSource") @Nullable InputSource inputSource, + @JsonProperty("inputFormat") @Nullable InputFormat inputFormat, @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting ) { + Checks.checkOneNotNullOrEmpty( + ImmutableList.of(new Property<>("firehose", firehoseFactory), new Property<>("inputSource", inputSource)) + ); + if (firehoseFactory != null && inputFormat != null) { + throw new IAE("Cannot use firehose and inputFormat together. Try use inputSource instead of firehose."); + } this.firehoseFactory = firehoseFactory; + this.inputSource = inputSource; + this.inputFormat = inputFormat; this.appendToExisting = appendToExisting == null ? DEFAULT_APPEND_TO_EXISTING : appendToExisting; } + // old constructor for backward compatibility + @Deprecated + public IndexIOConfig(FirehoseFactory firehoseFactory, @Nullable Boolean appendToExisting) + { + this(firehoseFactory, null, null, appendToExisting); + } + + @Nullable @JsonProperty("firehose") + @JsonInclude(Include.NON_NULL) + @Deprecated public FirehoseFactory getFirehoseFactory() { return firehoseFactory; } - @JsonProperty("appendToExisting") + @Nullable + @Override + @JsonProperty + public InputSource getInputSource() + { + return inputSource; + } + + @Nullable + @Override + @JsonProperty + public InputFormat getInputFormat() + { + return inputFormat; + } + + public InputSource getNonNullInputSource(@Nullable InputRowParser inputRowParser) + { + if (inputSource == null) { + return new FirehoseFactoryToInputSourceAdaptor(firehoseFactory, inputRowParser); + } else { + return inputSource; + } + } + + public InputFormat getNonNullInputFormat(@Nullable ParseSpec parseSpec) + { + if (inputFormat == null) { + return Preconditions.checkNotNull(parseSpec, "parseSpec").toInputFormat(); + } else { + return inputFormat; + } + } + + @Override + @JsonProperty public boolean isAppendToExisting() { return appendToExisting; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FiniteFirehoseProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java similarity index 87% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/FiniteFirehoseProcessor.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java index a3463f41ecc0..bece70379861 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FiniteFirehoseProcessor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java @@ -20,9 +20,10 @@ package org.apache.druid.indexing.common.task; import com.google.common.base.Optional; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.stats.RowIngestionMeters; @@ -30,6 +31,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; @@ -45,9 +47,9 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; -public class FiniteFirehoseProcessor +public class InputSourceProcessor { - private static final Logger LOG = new Logger(FiniteFirehoseProcessor.class); + private static final Logger LOG = new Logger(InputSourceProcessor.class); private final RowIngestionMeters buildSegmentsMeters; @Nullable @@ -56,7 +58,7 @@ public class FiniteFirehoseProcessor private final int maxParseExceptions; private final long pushTimeout; - public FiniteFirehoseProcessor( + public InputSourceProcessor( RowIngestionMeters buildSegmentsMeters, @Nullable CircularBuffer buildSegmentsSavedParseExceptions, boolean logParseExceptions, @@ -72,7 +74,7 @@ public FiniteFirehoseProcessor( } /** - * This method connects the given {@link FirehoseFactory} and processes data from the connected {@link Firehose}. + * This method opens the given {@link InputSource} and processes data via {@link InputSourceReader}. * All read data is consumed by {@link BatchAppenderatorDriver} which creates new segments. * All created segments are pushed when all input data is processed successfully. * @@ -82,8 +84,9 @@ public SegmentsAndMetadata process( DataSchema dataSchema, BatchAppenderatorDriver driver, PartitionsSpec partitionsSpec, - FirehoseFactory firehoseFactory, - File firehoseTempDir, + InputSource inputSource, + InputFormat inputFormat, + File tmpDir, IndexTaskSegmentAllocator segmentAllocator ) throws IOException, InterruptedException, ExecutionException, TimeoutException { @@ -92,12 +95,18 @@ public SegmentsAndMetadata process( ? (DynamicPartitionsSpec) partitionsSpec : null; final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); - try ( - final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) - ) { - while (firehose.hasMore()) { + final InputSourceReader inputSourceReader = dataSchema.getTransformSpec().decorate( + inputSource.reader( + dataSchema.getNonNullTimestampSpec(), + dataSchema.getNonNullDimensionsSpec(), + inputFormat, + tmpDir + ) + ); + try (final CloseableIterator inputRowIterator = inputSourceReader.read()) { + while (inputRowIterator.hasNext()) { try { - final InputRow inputRow = firehose.nextRow(); + final InputRow inputRow = inputRowIterator.next(); if (inputRow == null) { buildSegmentsMeters.incrementThrownAway(); @@ -123,7 +132,6 @@ public SegmentsAndMetadata process( final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName); if (addResult.isOk()) { - // incremental segment publishment is allowed only when rollup doesn't have to be perfect. if (dynamicPartitionsSpec != null) { final boolean isPushRequired = addResult.isPushRequired( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java index 5b8731edf581..dc07d2b22b18 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java @@ -352,7 +352,7 @@ public String getVersion(final Interval interval) this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, metrics); final Supplier committerSupplier = Committers.nilSupplier(); - final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); + final File firehoseTempDir = toolbox.getIndexingTmpDir(); LookupNodeService lookupNodeService = getContextValue(CTX_KEY_LOOKUP_TIER) == null ? toolbox.getLookupNodeService() : @@ -386,7 +386,10 @@ public String getVersion(final Interval interval) // Skip connecting firehose if we've been stopped before we got started. synchronized (this) { if (!gracefullyStopped) { - firehose = firehoseFactory.connect(spec.getDataSchema().getParser(), firehoseTempDir); + firehose = firehoseFactory.connect( + Preconditions.checkNotNull(spec.getDataSchema().getParser(), "inputRowParser"), + firehoseTempDir + ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java index 2e4ea8df5d31..ef540bbfec0c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java @@ -23,6 +23,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; import javax.annotation.Nullable; @@ -36,9 +38,18 @@ public class ParallelIndexIOConfig extends IndexIOConfig @JsonCreator public ParallelIndexIOConfig( @JsonProperty("firehose") FirehoseFactory firehoseFactory, + @JsonProperty("inputSource") @Nullable InputSource inputSource, + @JsonProperty("inputFormat") @Nullable InputFormat inputFormat, @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting ) { - super(firehoseFactory, appendToExisting); + super(firehoseFactory, inputSource, inputFormat, appendToExisting); + } + + // old constructor for backward compatibility + @Deprecated + public ParallelIndexIOConfig(FirehoseFactory firehoseFactory, @Nullable Boolean appendToExisting) + { + this(firehoseFactory, null, null, appendToExisting); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java index 22354ed4106f..5f5fdb2d1d0f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java @@ -21,6 +21,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import org.apache.druid.indexer.Checks; +import org.apache.druid.indexer.Property; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.IngestionSpec; @@ -39,6 +43,16 @@ public ParallelIndexIngestionSpec( { super(dataSchema, ioConfig, tuningConfig); + Checks.checkOneNotNullOrEmpty( + ImmutableList.of( + new Property<>("parser", dataSchema.getParserMap()), + new Property<>("inputFormat", ioConfig.getInputFormat()) + ) + ); + if (dataSchema.getParserMap() != null && ioConfig.getInputSource() != null) { + throw new IAE("Cannot use parser and inputSource together. Try use inputFormat instead of parser."); + } + this.dataSchema = dataSchema; this.ioConfig = ioConfig; this.tuningConfig = tuningConfig == null ? ParallelIndexTuningConfig.defaultConfig() : tuningConfig; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java index e3a7632b48a2..a454ef8d9eeb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java @@ -25,10 +25,12 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SplitHintSpec; +import org.apache.druid.data.input.impl.InputFormat; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskToolbox; @@ -195,10 +197,14 @@ public TaskState run() throws Exception } else { final SinglePhaseSubTaskSpec spec = (SinglePhaseSubTaskSpec) taskCompleteEvent.getSpec(); + final InputRowParser inputRowParser = spec.getIngestionSpec().getDataSchema().getParser(); LOG.error( "Failed to run sub tasks for inputSplits[%s]", getSplitsIfSplittable( - spec.getIngestionSpec().getIOConfig().getFirehoseFactory(), + spec.getIngestionSpec().getIOConfig().getNonNullInputSource(inputRowParser), + spec.getIngestionSpec().getIOConfig().getNonNullInputFormat( + inputRowParser == null ? null : inputRowParser.getParseSpec() + ), tuningConfig.getSplitHintSpec() ) ); @@ -255,15 +261,16 @@ public void onFailure(Throwable t) } private static List getSplitsIfSplittable( - FirehoseFactory firehoseFactory, + InputSource inputSource, + InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec ) throws IOException { - if (firehoseFactory instanceof FiniteFirehoseFactory) { - final FiniteFirehoseFactory finiteFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory; - return finiteFirehoseFactory.getSplits(splitHintSpec).collect(Collectors.toList()); + if (inputSource instanceof SplittableInputSource) { + final SplittableInputSource splittableInputSource = (SplittableInputSource) inputSource; + return splittableInputSource.createSplits(inputFormat, splitHintSpec).collect(Collectors.toList()); } else { - throw new ISE("firehoseFactory[%s] is not splittable", firehoseFactory.getClass().getSimpleName()); + throw new ISE("inputSource[%s] is not splittable", inputSource.getClass().getSimpleName()); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 265d38dd6053..c7df03c1c25b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -29,7 +29,9 @@ import com.google.common.base.Throwables; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.impl.InputFormat; +import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; @@ -117,7 +119,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen private static final Logger LOG = new Logger(ParallelIndexSupervisorTask.class); private final ParallelIndexIngestionSpec ingestionSchema; - private final FiniteFirehoseFactory baseFirehoseFactory; + private final InputSource baseInputSource; private final IndexingServiceClient indexingServiceClient; private final ChatHandlerProvider chatHandlerProvider; private final AuthorizerMapper authorizerMapper; @@ -179,11 +181,6 @@ public ParallelIndexSupervisorTask( this.ingestionSchema = ingestionSchema; - final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); - if (!(firehoseFactory instanceof FiniteFirehoseFactory)) { - throw new IAE("[%s] should implement FiniteFirehoseFactory", firehoseFactory.getClass().getSimpleName()); - } - if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) { if (ingestionSchema.getTuningConfig().getNumShards() == null) { throw new ISE("forceGuaranteedRollup is set but numShards is missing in partitionsSpec"); @@ -194,7 +191,9 @@ public ParallelIndexSupervisorTask( } } - this.baseFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory; + this.baseInputSource = ingestionSchema.getIOConfig().getNonNullInputSource( + ingestionSchema.getDataSchema().getParser() + ); this.indexingServiceClient = indexingServiceClient; this.chatHandlerProvider = chatHandlerProvider; this.authorizerMapper = authorizerMapper; @@ -392,10 +391,10 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception return runSinglePhaseParallel(toolbox); } } else { - if (!baseFirehoseFactory.isSplittable()) { + if (!baseInputSource.isSplittable()) { LOG.warn( "firehoseFactory[%s] is not splittable. Running sequentially.", - baseFirehoseFactory.getClass().getSimpleName() + baseInputSource.getClass().getSimpleName() ); } else if (ingestionSchema.getTuningConfig().getMaxNumConcurrentSubTasks() <= 1) { LOG.warn( @@ -434,7 +433,7 @@ private void initializeSubTaskCleaner() private boolean isParallelMode() { - return baseFirehoseFactory.isSplittable() && ingestionSchema.getTuningConfig().getMaxNumConcurrentSubTasks() > 1; + return baseInputSource.isSplittable() && ingestionSchema.getTuningConfig().getMaxNumConcurrentSubTasks() > 1; } /** @@ -778,6 +777,14 @@ private static String findVersion(Map versions, Interval inter .orElse(null); } + static InputFormat getInputFormat(ParallelIndexIngestionSpec ingestionSchema) + { + final InputRowParser parser = ingestionSchema.getDataSchema().getParser(); + return ingestionSchema.getIOConfig().getNonNullInputFormat( + parser == null ? null : parser.getParseSpec() + ); + } + /** * {@link SinglePhaseSubTask}s call this API to report the segments they generated and pushed. */ diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java index edb00aa100d3..81925b9ed02d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java @@ -21,8 +21,8 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexing.common.TaskToolbox; import java.io.IOException; @@ -38,7 +38,7 @@ class PartialSegmentGenerateParallelIndexTaskRunner extends ParallelIndexPhaseRunner { private final ParallelIndexIngestionSpec ingestionSchema; - private final FiniteFirehoseFactory baseFirehoseFactory; + private final SplittableInputSource baseInputSource; PartialSegmentGenerateParallelIndexTaskRunner( TaskToolbox toolbox, @@ -58,7 +58,9 @@ class PartialSegmentGenerateParallelIndexTaskRunner indexingServiceClient ); this.ingestionSchema = ingestionSchema; - this.baseFirehoseFactory = (FiniteFirehoseFactory) ingestionSchema.getIOConfig().getFirehoseFactory(); + this.baseInputSource = (SplittableInputSource) ingestionSchema.getIOConfig().getNonNullInputSource( + ingestionSchema.getDataSchema().getParser() + ); } @Override @@ -70,13 +72,19 @@ public String getName() @Override Iterator> subTaskSpecIterator() throws IOException { - return baseFirehoseFactory.getSplits(getTuningConfig().getSplitHintSpec()).map(this::newTaskSpec).iterator(); + return baseInputSource.createSplits( + ingestionSchema.getIOConfig().getInputFormat(), + getTuningConfig().getSplitHintSpec() + ).map(this::newTaskSpec).iterator(); } @Override int getTotalNumSubTasks() throws IOException { - return baseFirehoseFactory.getNumSplits(getTuningConfig().getSplitHintSpec()); + return baseInputSource.getNumSplits( + ingestionSchema.getIOConfig().getInputFormat(), + getTuningConfig().getSplitHintSpec() + ); } @VisibleForTesting @@ -86,9 +94,9 @@ ParallelIndexIngestionSpec getIngestionSchema() } @VisibleForTesting - FiniteFirehoseFactory getBaseFirehoseFactory() + SplittableInputSource getBaseInputSource() { - return baseFirehoseFactory; + return baseInputSource; } SubTaskSpec newTaskSpec(InputSplit split) @@ -96,7 +104,9 @@ SubTaskSpec newTaskSpec(InputSplit split) final ParallelIndexIngestionSpec subTaskIngestionSpec = new ParallelIndexIngestionSpec( ingestionSchema.getDataSchema(), new ParallelIndexIOConfig( - baseFirehoseFactory.withSplit(split), + null, + baseInputSource.withSplit(split), + ingestionSchema.getIOConfig().getInputFormat(), ingestionSchema.getIOConfig().isAppendToExisting() ), ingestionSchema.getTuningConfig() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index dd35a43b5006..af1772fc3dc8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -25,7 +25,7 @@ import com.google.common.base.Preconditions; import org.apache.commons.io.FileUtils; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.InputSource; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; @@ -36,9 +36,9 @@ import org.apache.druid.indexing.common.task.BatchAppenderators; import org.apache.druid.indexing.common.task.CachingLocalSegmentAllocator; import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; -import org.apache.druid.indexing.common.task.FiniteFirehoseProcessor; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator; +import org.apache.druid.indexing.common.task.InputSourceProcessor; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher; @@ -206,11 +206,13 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception @Override public TaskStatus runTask(TaskToolbox toolbox) throws Exception { - final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); + final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource( + ingestionSchema.getDataSchema().getParser() + ); - final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); + final File tmpDir = toolbox.getIndexingTmpDir(); // Firehose temporary directory is automatically removed when this IndexTask completes. - FileUtils.forceMkdir(firehoseTempDir); + FileUtils.forceMkdir(tmpDir); final ParallelIndexSupervisorTaskClient taskClient = taskClientFactory.build( new ClientBasedTaskInfoProvider(indexingServiceClient), @@ -220,7 +222,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception ingestionSchema.getTuningConfig().getChatHandlerNumRetries() ); - final List segments = generateSegments(toolbox, firehoseFactory, firehoseTempDir); + final List segments = generateSegments(toolbox, inputSource, tmpDir); final List partitionStats = segments .stream() .map(segment -> new PartitionStat( @@ -240,8 +242,8 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception private List generateSegments( final TaskToolbox toolbox, - final FirehoseFactory firehoseFactory, - final File firehoseTempDir + final InputSource inputSource, + final File tmpDir ) throws IOException, InterruptedException, ExecutionException, TimeoutException { final DataSchema dataSchema = ingestionSchema.getDataSchema(); @@ -295,19 +297,20 @@ private List generateSegments( try (final BatchAppenderatorDriver driver = BatchAppenderators.newDriver(appenderator, toolbox, segmentAllocator)) { driver.startJob(); - final FiniteFirehoseProcessor firehoseProcessor = new FiniteFirehoseProcessor( + final InputSourceProcessor inputSourceProcessor = new InputSourceProcessor( buildSegmentsMeters, null, tuningConfig.isLogParseExceptions(), tuningConfig.getMaxParseExceptions(), pushTimeout ); - final SegmentsAndMetadata pushed = firehoseProcessor.process( + final SegmentsAndMetadata pushed = inputSourceProcessor.process( dataSchema, driver, partitionsSpec, - firehoseFactory, - firehoseTempDir, + inputSource, + ParallelIndexSupervisorTask.getInputFormat(ingestionSchema), + tmpDir, segmentAllocator ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java index 2a3953468da9..58dba1310f41 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java @@ -284,7 +284,7 @@ private Map>> fetchSegmentFiles( Map>> intervalToPartitions ) throws IOException { - final File tempDir = toolbox.getFirehoseTemporaryDir(); + final File tempDir = toolbox.getIndexingTmpDir(); FileUtils.deleteQuietly(tempDir); FileUtils.forceMkdir(tempDir); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index 302f3fa6db1f..9ddb10722a3a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -21,8 +21,8 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexing.common.TaskToolbox; import java.io.IOException; @@ -40,7 +40,7 @@ class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner { private final ParallelIndexIngestionSpec ingestionSchema; - private final FiniteFirehoseFactory baseFirehoseFactory; + private final SplittableInputSource baseInputSource; SinglePhaseParallelIndexTaskRunner( TaskToolbox toolbox, @@ -60,7 +60,9 @@ class SinglePhaseParallelIndexTaskRunner indexingServiceClient ); this.ingestionSchema = ingestionSchema; - this.baseFirehoseFactory = (FiniteFirehoseFactory) ingestionSchema.getIOConfig().getFirehoseFactory(); + this.baseInputSource = (SplittableInputSource) ingestionSchema.getIOConfig().getNonNullInputSource( + ingestionSchema.getDataSchema().getParser() + ); } @Override @@ -79,13 +81,19 @@ ParallelIndexIngestionSpec getIngestionSchema() @Override Iterator> subTaskSpecIterator() throws IOException { - return baseFirehoseFactory.getSplits(getTuningConfig().getSplitHintSpec()).map(this::newTaskSpec).iterator(); + return baseInputSource.createSplits( + ingestionSchema.getIOConfig().getInputFormat(), + getTuningConfig().getSplitHintSpec() + ).map(this::newTaskSpec).iterator(); } @Override int getTotalNumSubTasks() throws IOException { - return baseFirehoseFactory.getNumSplits(getTuningConfig().getSplitHintSpec()); + return baseInputSource.getNumSplits( + ingestionSchema.getIOConfig().getInputFormat(), + getTuningConfig().getSplitHintSpec() + ); } @VisibleForTesting @@ -98,7 +106,9 @@ SubTaskSpec newTaskSpec(InputSplit split) new ParallelIndexIngestionSpec( ingestionSchema.getDataSchema(), new ParallelIndexIOConfig( - baseFirehoseFactory.withSplit(split), + null, + baseInputSource.withSplit(split), + ingestionSchema.getIOConfig().getInputFormat(), ingestionSchema.getIOConfig().isAppendToExisting() ), ingestionSchema.getTuningConfig() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 7f9590a11cd7..57e305e0a529 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -26,9 +26,9 @@ import com.google.common.base.Optional; import org.apache.commons.io.FileUtils; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; @@ -51,6 +51,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.indexing.DataSchema; @@ -208,11 +209,13 @@ public TaskStatus runTask(final TaskToolbox toolbox) throws Exception + "Forced to use timeChunk lock." ); } - final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); + final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource( + ingestionSchema.getDataSchema().getParser() + ); - final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); + final File tmpDir = toolbox.getIndexingTmpDir(); // Firehose temporary directory is automatically removed when this IndexTask completes. - FileUtils.forceMkdir(firehoseTempDir); + FileUtils.forceMkdir(tmpDir); final ParallelIndexSupervisorTaskClient taskClient = taskClientFactory.build( new ClientBasedTaskInfoProvider(indexingServiceClient), @@ -224,8 +227,8 @@ public TaskStatus runTask(final TaskToolbox toolbox) throws Exception final Set pushedSegments = generateAndPushSegments( toolbox, taskClient, - firehoseFactory, - firehoseTempDir + inputSource, + tmpDir ); // Find inputSegments overshadowed by pushedSegments @@ -386,8 +389,8 @@ private SegmentAllocator createSegmentAllocator() private Set generateAndPushSegments( final TaskToolbox toolbox, final ParallelIndexSupervisorTaskClient taskClient, - final FirehoseFactory firehoseFactory, - final File firehoseTempDir + final InputSource inputSource, + final File tmpDir ) throws IOException, InterruptedException { final DataSchema dataSchema = ingestionSchema.getDataSchema(); @@ -420,18 +423,27 @@ private Set generateAndPushSegments( tuningConfig, getContextValue(Tasks.STORE_COMPACTION_STATE_KEY, Tasks.DEFAULT_STORE_COMPACTION_STATE) ); + final InputSourceReader inputSourceReader = dataSchema.getTransformSpec().decorate( + inputSource.reader( + ingestionSchema.getDataSchema().getNonNullTimestampSpec(), + ingestionSchema.getDataSchema().getNonNullDimensionsSpec(), + ParallelIndexSupervisorTask.getInputFormat(ingestionSchema), + tmpDir + ) + ); + boolean exceptionOccurred = false; try ( final BatchAppenderatorDriver driver = BatchAppenderators.newDriver(appenderator, toolbox, segmentAllocator); - final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) + final CloseableIterator inputRowIterator = inputSourceReader.read() ) { driver.startJob(); final Set pushedSegments = new HashSet<>(); - while (firehose.hasMore()) { + while (inputRowIterator.hasNext()) { try { - final InputRow inputRow = firehose.nextRow(); + final InputRow inputRow = inputRowIterator.next(); if (inputRow == null) { fireDepartmentMetrics.incrementThrownAway(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 9a7fa51bad8e..79104efb762c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -101,16 +101,8 @@ public class CompactionTaskRunTest extends IngestionTestBase public ExpectedException expectedException = ExpectedException.none(); public static final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec( - new TimestampSpec( - "ts", - "auto", - null - ), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")), - Collections.emptyList(), - Collections.emptyList() - ), + new TimestampSpec("ts", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim"))), null, Arrays.asList("ts", "dim", "val"), false, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 3253fdc4f672..89fb241007aa 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -36,11 +36,9 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.DoubleDimensionSchema; import org.apache.druid.data.input.impl.FloatDimensionSchema; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.LongDimensionSchema; -import org.apache.druid.data.input.impl.NoopInputRowParser; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.GuiceAnnotationIntrospector; import org.apache.druid.guice.GuiceInjectableValues; import org.apache.druid.guice.GuiceInjectors; @@ -103,7 +101,6 @@ import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.selector.settable.SettableColumnValueSelector; -import org.apache.druid.segment.transform.TransformingInputRowParser; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthorizerMapper; @@ -1060,13 +1057,10 @@ private void assertIngestionSchema( final DataSchema dataSchema = ingestionSchema.getDataSchema(); Assert.assertEquals(DATA_SOURCE, dataSchema.getDataSource()); - final InputRowParser parser = OBJECT_MAPPER.convertValue(dataSchema.getParser(), InputRowParser.class); - Assert.assertTrue(parser instanceof TransformingInputRowParser); - Assert.assertTrue(((TransformingInputRowParser) parser).getParser() instanceof NoopInputRowParser); - Assert.assertTrue(parser.getParseSpec() instanceof TimeAndDimsParseSpec); + Assert.assertEquals(new TimestampSpec(null, null, null), dataSchema.getTimestampSpec()); Assert.assertEquals( new HashSet<>(expectedDimensionsSpec.getDimensions()), - new HashSet<>(parser.getParseSpec().getDimensionsSpec().getDimensions()) + new HashSet<>(dataSchema.getDimensionsSpec().getDimensions()) ); // metrics diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java new file mode 100644 index 000000000000..c700ad361321 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.NoopFirehoseFactory; +import org.apache.druid.data.input.impl.NoopInputFormat; +import org.apache.druid.data.input.impl.NoopInputSource; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; +import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class IndexIngestionSpecTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testParserAndInputFormat() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "At most one of [Property{name='parser', value={fake=parser map}}, Property{name='inputFormat'," + ); + final IndexIngestionSpec spec = new IndexIngestionSpec( + new DataSchema( + "dataSource", + ImmutableMap.of("fake", "parser map"), + new AggregatorFactory[0], + new ArbitraryGranularitySpec(Granularities.NONE, null), + null, + null + ), + new IndexIOConfig( + null, + new NoopInputSource(), + new NoopInputFormat(), + null + ), + null + ); + } + + @Test + public void testParserAndInputSource() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Cannot use parser and inputSource together."); + final IndexIngestionSpec spec = new IndexIngestionSpec( + new DataSchema( + "dataSource", + ImmutableMap.of("fake", "parser map"), + new AggregatorFactory[0], + new ArbitraryGranularitySpec(Granularities.NONE, null), + null, + null + ), + new IndexIOConfig( + null, + new NoopInputSource(), + null, + null + ), + null + ); + } + + @Test + public void testFirehoseAndInputSource() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "At most one of [Property{name='firehose', value=NoopFirehoseFactory{}}, Property{name='inputSource'" + ); + final IndexIngestionSpec spec = new IndexIngestionSpec( + new DataSchema( + "dataSource", + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, + new AggregatorFactory[0], + new ArbitraryGranularitySpec(Granularities.NONE, null), + null + ), + new IndexIOConfig( + new NoopFirehoseFactory(), + new NoopInputSource(), + null, + null + ), + null + ); + } + + @Test + public void testFirehoseAndInputFormat() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Cannot use firehose and inputFormat together."); + final IndexIngestionSpec spec = new IndexIngestionSpec( + new DataSchema( + "dataSource", + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, + new AggregatorFactory[0], + new ArbitraryGranularitySpec(Granularities.NONE, null), + null + ), + new IndexIOConfig( + new NoopFirehoseFactory(), + null, + new NoopInputFormat(), + null + ), + null + ); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 8cb99452a80b..8b5c2b46dc27 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -29,7 +29,9 @@ import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; +import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.StringDimensionSchema; @@ -45,6 +47,7 @@ import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.java.util.common.Intervals; @@ -115,47 +118,49 @@ public class IndexTaskTest extends IngestionTestBase @Rule public ExpectedException expectedException = ExpectedException.none(); + private static final TimestampSpec DEFAULT_TIMESTAMP_SPEC = new TimestampSpec("ts", "auto", null); + private static final DimensionsSpec DEFAULT_DIMENSIONS_SPEC = new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")) + ); private static final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec( - new TimestampSpec( - "ts", - "auto", - null - ), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")), - new ArrayList<>(), - new ArrayList<>() - ), + DEFAULT_TIMESTAMP_SPEC, + DEFAULT_DIMENSIONS_SPEC, null, Arrays.asList("ts", "dim", "val"), false, 0 ); + private static final InputFormat DEFAULT_INPUT_FORMAT = DEFAULT_PARSE_SPEC.toInputFormat(); - @Parameterized.Parameters(name = "{0}") + @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}") public static Iterable constructorFeeder() { return ImmutableList.of( - new Object[]{LockGranularity.TIME_CHUNK}, - new Object[]{LockGranularity.SEGMENT} + new Object[]{LockGranularity.TIME_CHUNK, false}, + new Object[]{LockGranularity.TIME_CHUNK, true}, + new Object[]{LockGranularity.SEGMENT, false}, + new Object[]{LockGranularity.SEGMENT, true} ); } private static final IndexSpec INDEX_SPEC = new IndexSpec(); private final ObjectMapper jsonMapper; - private AppenderatorsManager appenderatorsManager; private final IndexIO indexIO; private final RowIngestionMetersFactory rowIngestionMetersFactory; private final LockGranularity lockGranularity; + private final boolean useInputFormatApi; + + private AppenderatorsManager appenderatorsManager; private SegmentLoader segmentLoader; private TestTaskRunner taskRunner; - public IndexTaskTest(LockGranularity lockGranularity) + public IndexTaskTest(LockGranularity lockGranularity, boolean useInputFormatApi) { this.jsonMapper = getObjectMapper(); this.indexIO = getIndexIO(); this.rowIngestionMetersFactory = getRowIngestionMetersFactory(); this.lockGranularity = lockGranularity; + this.useInputFormatApi = useInputFormatApi; } @Before @@ -198,10 +203,12 @@ public void testDeterminePartitions() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, null, + null, createTuningConfigWithMaxRowsPerSegment(2, true), false ), @@ -246,6 +253,7 @@ public void testTransformSpec() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -295,6 +303,7 @@ public void testWithArbitraryGranularity() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -333,6 +342,7 @@ public void testIntervalBucketing() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -372,10 +382,12 @@ public void testNumShardsProvided() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, null, + null, createTuningConfigWithNumShards(1, null, true), false ), @@ -412,10 +424,12 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, null, + null, createTuningConfigWithNumShards(2, ImmutableList.of("dim"), true), false ), @@ -488,10 +502,12 @@ public void testAppendToExisting() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, null, + null, createTuningConfigWithMaxRowsPerSegment(2, false), true ), @@ -536,6 +552,7 @@ public void testIntervalNotSpecified() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -590,19 +607,12 @@ public void testCSVFileWithHeader() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, new CSVParseSpec( - new TimestampSpec( - "time", - "auto", - null - ), - new DimensionsSpec( - null, - new ArrayList<>(), - new ArrayList<>() - ), + new TimestampSpec("time", "auto", null), + DimensionsSpec.EMPTY, null, null, true, @@ -644,19 +654,12 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, new CSVParseSpec( - new TimestampSpec( - "time", - "auto", - null - ), - new DimensionsSpec( - null, - new ArrayList<>(), - new ArrayList<>() - ), + new TimestampSpec("time", "auto", null), + DimensionsSpec.EMPTY, null, Arrays.asList("time", "dim", "val"), true, @@ -704,6 +707,7 @@ public void testWithSmallMaxTotalRows() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -750,6 +754,7 @@ public void testPerfectRollup() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -796,6 +801,7 @@ public void testBestEffortRollup() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -861,19 +867,12 @@ public void testIgnoreParseException() throws Exception // GranularitySpec.intervals and numShards must be null to verify reportParseException=false is respected both in // IndexTask.determineShardSpecs() and IndexTask.generateAndPublishSegments() final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, new CSVParseSpec( - new TimestampSpec( - "time", - "auto", - null - ), - new DimensionsSpec( - null, - new ArrayList<>(), - new ArrayList<>() - ), + new TimestampSpec("time", "auto", null), + DimensionsSpec.EMPTY, null, Arrays.asList("time", "dim", "val"), true, @@ -916,19 +915,12 @@ public void testReportParseException() throws Exception } final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, new CSVParseSpec( - new TimestampSpec( - "time", - "auto", - null - ), - new DimensionsSpec( - null, - new ArrayList<>(), - new ArrayList<>() - ), + new TimestampSpec("time", "auto", null), + DimensionsSpec.EMPTY, null, Arrays.asList("time", "dim", "val"), true, @@ -987,7 +979,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception writer.write("this is not JSON\n"); // invalid JSON } - final IndexTask.IndexTuningConfig tuningConfig = new IndexTask.IndexTuningConfig( + final IndexTuningConfig tuningConfig = new IndexTuningConfig( null, null, null, @@ -1011,22 +1003,17 @@ public void testMultipleParseExceptionsSuccess() throws Exception ); final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, new JSONParseSpec( - new TimestampSpec( - "time", - "auto", - null - ), + new TimestampSpec("time", "auto", null), new DimensionsSpec( Arrays.asList( new StringDimensionSchema("dim"), new LongDimensionSchema("dimLong"), new FloatDimensionSchema("dimFloat") - ), - new ArrayList<>(), - new ArrayList<>() + ) ), null, null @@ -1112,7 +1099,7 @@ public void testMultipleParseExceptionsFailure() throws Exception } // Allow up to 3 parse exceptions, and save up to 2 parse exceptions - final IndexTask.IndexTuningConfig tuningConfig = new IndexTask.IndexTuningConfig( + final IndexTuningConfig tuningConfig = new IndexTuningConfig( null, null, null, @@ -1136,22 +1123,17 @@ public void testMultipleParseExceptionsFailure() throws Exception ); final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, new CSVParseSpec( - new TimestampSpec( - "time", - "auto", - null - ), + new TimestampSpec("time", "auto", null), new DimensionsSpec( Arrays.asList( new StringDimensionSchema("dim"), new LongDimensionSchema("dimLong"), new FloatDimensionSchema("dimFloat") - ), - new ArrayList<>(), - new ArrayList<>() + ) ), null, Arrays.asList("time", "dim", "dimLong", "dimFloat", "val"), @@ -1193,7 +1175,7 @@ public void testMultipleParseExceptionsFailure() throws Exception RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.PROCESSED, 1, RowIngestionMeters.UNPARSEABLE, 3, - RowIngestionMeters.THROWN_AWAY, 2 + RowIngestionMeters.THROWN_AWAY, useInputFormatApi ? 1 : 2 ) ); @@ -1230,7 +1212,7 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc } // Allow up to 3 parse exceptions, and save up to 2 parse exceptions - final IndexTask.IndexTuningConfig tuningConfig = new IndexTask.IndexTuningConfig( + final IndexTuningConfig tuningConfig = new IndexTuningConfig( null, null, null, @@ -1254,22 +1236,17 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc ); final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, new CSVParseSpec( - new TimestampSpec( - "time", - "auto", - null - ), + new TimestampSpec("time", "auto", null), new DimensionsSpec( Arrays.asList( new StringDimensionSchema("dim"), new LongDimensionSchema("dimLong"), new FloatDimensionSchema("dimFloat") - ), - new ArrayList<>(), - new ArrayList<>() + ) ), null, Arrays.asList("time", "dim", "dimLong", "dimFloat", "val"), @@ -1304,7 +1281,7 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.PROCESSED, 1, RowIngestionMeters.UNPARSEABLE, 3, - RowIngestionMeters.THROWN_AWAY, 2 + RowIngestionMeters.THROWN_AWAY, useInputFormatApi ? 1 : 2 ), RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( @@ -1331,7 +1308,6 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc Assert.assertEquals(expectedUnparseables, reportData.getUnparseableEvents()); } - @Test public void testCsvWithHeaderOfEmptyColumns() throws Exception { @@ -1359,19 +1335,12 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception } final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, new CSVParseSpec( - new TimestampSpec( - "time", - "auto", - null - ), - new DimensionsSpec( - null, - null, - null - ), + new TimestampSpec("time", "auto", null), + DimensionsSpec.EMPTY, null, null, true, @@ -1431,19 +1400,12 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception } final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, new CSVParseSpec( - new TimestampSpec( - "time", - "auto", - null - ), - new DimensionsSpec( - null, - new ArrayList<>(), - new ArrayList<>() - ), + new TimestampSpec("time", "auto", null), + DimensionsSpec.EMPTY, null, Arrays.asList("time", "", ""), true, @@ -1495,6 +1457,7 @@ public void testOverwriteWithSameSegmentGranularity() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -1563,6 +1526,7 @@ public void testOverwriteWithDifferentSegmentGranularity() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -1659,7 +1623,7 @@ static IndexTuningConfig createTuningConfig( boolean reportParseException ) { - return new IndexTask.IndexTuningConfig( + return new IndexTuningConfig( null, maxRowsPerSegment, maxRowsInMemory, @@ -1696,16 +1660,38 @@ private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOExcep ); } - public static IndexTask.IndexIngestionSpec createIngestionSpec( + static IndexIngestionSpec createIngestionSpec( ObjectMapper objectMapper, File baseDir, - ParseSpec parseSpec, + @Nullable ParseSpec parseSpec, GranularitySpec granularitySpec, IndexTuningConfig tuningConfig, boolean appendToExisting ) { return createIngestionSpec( + false, + objectMapper, + baseDir, + parseSpec, + granularitySpec, + tuningConfig, + appendToExisting + ); + } + + static IndexIngestionSpec createIngestionSpec( + boolean useInputFormatApi, + ObjectMapper objectMapper, + File baseDir, + @Nullable ParseSpec parseSpec, + GranularitySpec granularitySpec, + IndexTuningConfig tuningConfig, + boolean appendToExisting + ) + { + return createIngestionSpec( + useInputFormatApi, objectMapper, baseDir, parseSpec, @@ -1716,46 +1702,73 @@ public static IndexTask.IndexIngestionSpec createIngestionSpec( ); } - public static IndexTask.IndexIngestionSpec createIngestionSpec( + private static IndexIngestionSpec createIngestionSpec( + boolean useInputFormatApi, ObjectMapper objectMapper, File baseDir, - ParseSpec parseSpec, + @Nullable ParseSpec parseSpec, TransformSpec transformSpec, GranularitySpec granularitySpec, IndexTuningConfig tuningConfig, boolean appendToExisting ) { - return new IndexTask.IndexIngestionSpec( - new DataSchema( - "test", - objectMapper.convertValue( - new StringInputRowParser( - parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC, - null - ), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - granularitySpec != null ? granularitySpec : new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/2015")) - ), - transformSpec, - objectMapper - ), - new IndexTask.IndexIOConfig( - new LocalFirehoseFactory( - baseDir, - "druid*", - null - ), - appendToExisting - ), - tuningConfig - ); + if (useInputFormatApi) { + return new IndexIngestionSpec( + new DataSchema( + "test", + parseSpec == null ? DEFAULT_TIMESTAMP_SPEC : parseSpec.getTimestampSpec(), + parseSpec == null ? DEFAULT_DIMENSIONS_SPEC : parseSpec.getDimensionsSpec(), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + granularitySpec != null ? granularitySpec : new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2014/2015")) + ), + transformSpec + ), + new IndexIOConfig( + null, + new LocalInputSource(baseDir, "druid*"), + parseSpec == null ? DEFAULT_INPUT_FORMAT : parseSpec.toInputFormat(), + appendToExisting + ), + tuningConfig + ); + } else { + return new IndexIngestionSpec( + new DataSchema( + "test", + objectMapper.convertValue( + new StringInputRowParser( + parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC, + null + ), + Map.class + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + granularitySpec != null ? granularitySpec : new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2014/2015")) + ), + transformSpec, + objectMapper + ), + new IndexIOConfig( + new LocalFirehoseFactory( + baseDir, + "druid*", + null + ), + appendToExisting + ), + tuningConfig + ); + } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 974cfb3687db..5e40b6b1a844 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -30,6 +30,7 @@ import org.apache.druid.client.indexing.TaskStatusResponse; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.RunnerTaskState; @@ -71,7 +72,6 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; import java.util.Map; @@ -82,22 +82,19 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase { + static final TimestampSpec DEFAULT_TIMESTAMP_SPEC = new TimestampSpec("ts", "auto", null); + static final DimensionsSpec DEFAULT_DIMENSIONS_SPEC = new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")) + ); static final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec( - new TimestampSpec( - "ts", - "auto", - null - ), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")), - new ArrayList<>(), - new ArrayList<>() - ), + DEFAULT_TIMESTAMP_SPEC, + DEFAULT_DIMENSIONS_SPEC, null, Arrays.asList("ts", "dim", "val"), false, 0 ); + static final InputFormat DEFAULT_INPUT_FORMAT = DEFAULT_PARSE_SPEC.toInputFormat(); protected TestLocalTaskActionClient actionClient; protected LocalIndexingServiceClient indexingServiceClient; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java index 6501479e0f5b..a7bc506fe9bf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java @@ -24,6 +24,7 @@ import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; @@ -90,12 +91,14 @@ @RunWith(Parameterized.class) public class MultiPhaseParallelIndexingTest extends AbstractParallelIndexSupervisorTaskTest { - @Parameterized.Parameters(name = "{0}") + @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}") public static Iterable constructorFeeder() { return ImmutableList.of( - new Object[]{LockGranularity.TIME_CHUNK}, - new Object[]{LockGranularity.SEGMENT} + new Object[]{LockGranularity.TIME_CHUNK, false}, + new Object[]{LockGranularity.TIME_CHUNK, true}, + new Object[]{LockGranularity.SEGMENT, false}, + new Object[]{LockGranularity.SEGMENT, true} ); } @@ -103,11 +106,14 @@ public static Iterable constructorFeeder() public ExpectedException expectedException = ExpectedException.none(); private final LockGranularity lockGranularity; + private final boolean useInputFormatApi; + private File inputDir; - public MultiPhaseParallelIndexingTest(LockGranularity lockGranularity) + public MultiPhaseParallelIndexingTest(LockGranularity lockGranularity, boolean useInputFormatApi) { this.lockGranularity = lockGranularity; + this.useInputFormatApi = useInputFormatApi; } @Before @@ -156,14 +162,7 @@ public void testRun() throws Exception private Set runTestTask(Interval interval, HashedPartitionsSpec partitionsSpec) throws Exception { - final ParallelIndexSupervisorTask task = newTask( - interval, - new ParallelIndexIOConfig( - new LocalFirehoseFactory(inputDir, "test_*", null), - false - ), - partitionsSpec - ); + final ParallelIndexSupervisorTask task = newTask(interval, partitionsSpec); actionClient = createActionClient(task); toolbox = createTaskToolbox(task); @@ -177,14 +176,12 @@ private Set runTestTask(Interval interval, HashedPartitionsSpec par private ParallelIndexSupervisorTask newTask( Interval interval, - ParallelIndexIOConfig ioConfig, HashedPartitionsSpec partitionsSpec ) { return newTask( interval, Granularities.DAY, - ioConfig, new ParallelIndexTuningConfig( null, null, @@ -219,53 +216,65 @@ private ParallelIndexSupervisorTask newTask( private ParallelIndexSupervisorTask newTask( Interval interval, Granularity segmentGranularity, - ParallelIndexIOConfig ioConfig, ParallelIndexTuningConfig tuningConfig ) { - // set up ingestion spec final ParseSpec parseSpec = new CSVParseSpec( - new TimestampSpec( - "ts", - "auto", - null - ), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim1", "dim2")), - new ArrayList<>(), - new ArrayList<>() - ), + new TimestampSpec("ts", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim1", "dim2"))), null, Arrays.asList("ts", "dim1", "dim2", "val"), false, 0 ); - - //noinspection unchecked - final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - getObjectMapper().convertValue( - new StringInputRowParser( - parseSpec, - null - ), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - segmentGranularity, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null, - getObjectMapper() - ), - ioConfig, - tuningConfig - ); + final ParallelIndexIngestionSpec ingestionSpec; + if (useInputFormatApi) { + ingestionSpec = new ParallelIndexIngestionSpec( + new DataSchema( + "dataSource", + parseSpec.getTimestampSpec(), + parseSpec.getDimensionsSpec(), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + new UniformGranularitySpec( + segmentGranularity, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ), + null + ), + new ParallelIndexIOConfig( + null, + new LocalInputSource(inputDir, "test_*"), + parseSpec.toInputFormat(), + false + ), + tuningConfig + ); + } else { + ingestionSpec = new ParallelIndexIngestionSpec( + new DataSchema( + "dataSource", + getObjectMapper().convertValue( + new StringInputRowParser(parseSpec, null), + Map.class + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + new UniformGranularitySpec( + segmentGranularity, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ), + null, + getObjectMapper() + ), + new ParallelIndexIOConfig(new LocalFirehoseFactory(inputDir, "test_*", null), false), + tuningConfig + ); + } // set up test tools return new TestSupervisorTask( @@ -414,7 +423,9 @@ SubTaskSpec newTaskSpec(InputSplit split) final ParallelIndexIngestionSpec subTaskIngestionSpec = new ParallelIndexIngestionSpec( getIngestionSchema().getDataSchema(), new ParallelIndexIOConfig( - getBaseFirehoseFactory().withSplit(split), + null, + getBaseInputSource().withSplit(split), + getIngestionSchema().getIOConfig().getInputFormat(), getIngestionSchema().getIOConfig().isAppendToExisting() ), getIngestionSchema().getTuningConfig() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index 463ef4db5e67..d845b73aa5d9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -21,10 +21,14 @@ import com.google.common.collect.Iterables; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.FiniteFirehoseFactory; +import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputFormat; +import org.apache.druid.data.input.impl.NoopInputFormat; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; @@ -51,6 +55,7 @@ import org.junit.rules.ExpectedException; import javax.annotation.Nullable; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -90,8 +95,10 @@ public void testStopGracefully() throws Exception final ParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), new ParallelIndexIOConfig( + null, // Sub tasks would run forever - new TestFirehoseFactory(Pair.of(new TestInput(Integer.MAX_VALUE, TaskState.SUCCESS), 4)), + new TestInputSource(Pair.of(new TestInput(Integer.MAX_VALUE, TaskState.SUCCESS), 4)), + new NoopInputFormat(), false ) ); @@ -124,10 +131,12 @@ public void testSubTaskFail() throws Exception final ParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), new ParallelIndexIOConfig( - new TestFirehoseFactory( + null, + new TestInputSource( Pair.of(new TestInput(10L, TaskState.FAILED), 1), Pair.of(new TestInput(Integer.MAX_VALUE, TaskState.FAILED), 3) ), + new NoopInputFormat(), false ) ); @@ -161,19 +170,14 @@ private ParallelIndexSupervisorTask newTask( ParallelIndexIOConfig ioConfig ) { - final TestFirehoseFactory firehoseFactory = (TestFirehoseFactory) ioConfig.getFirehoseFactory(); - final int numTotalSubTasks = firehoseFactory.getNumSplits(null); + final TestInputSource inputSource = (TestInputSource) ioConfig.getInputSource(); + final int numTotalSubTasks = inputSource.getNumSplits(new NoopInputFormat(), null); // set up ingestion spec final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( new DataSchema( "dataSource", - getObjectMapper().convertValue( - new StringInputRowParser( - DEFAULT_PARSE_SPEC, - null - ), - Map.class - ), + DEFAULT_TIMESTAMP_SPEC, + DEFAULT_DIMENSIONS_SPEC, new AggregatorFactory[]{ new LongSumAggregatorFactory("val", "val") }, @@ -182,8 +186,7 @@ private ParallelIndexSupervisorTask newTask( Granularities.MINUTE, interval == null ? null : Collections.singletonList(interval) ), - null, - getObjectMapper() + null ), ioConfig, new ParallelIndexTuningConfig( @@ -236,12 +239,12 @@ private TestInput(long runTime, TaskState finalState) } } - private static class TestFirehoseFactory implements FiniteFirehoseFactory + private static class TestInputSource implements SplittableInputSource { private final List> splits; @SafeVarargs - private TestFirehoseFactory(Pair... inputSpecs) + private TestInputSource(Pair... inputSpecs) { splits = new ArrayList<>(); for (Pair inputSpec : inputSpecs) { @@ -252,27 +255,38 @@ private TestFirehoseFactory(Pair... inputSpecs) } } - private TestFirehoseFactory(InputSplit split) + private TestInputSource(InputSplit split) { this.splits = Collections.singletonList(split); } @Override - public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec) + public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) { return splits.stream(); } @Override - public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) + public int getNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) { return splits.size(); } @Override - public FiniteFirehoseFactory withSplit(InputSplit split) + public SplittableInputSource withSplit(InputSplit split) + { + return new TestInputSource(split); + } + + @Override + public InputSourceReader reader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) { - return new TestFirehoseFactory(split); + throw new UnsupportedOperationException(); } } @@ -331,9 +345,9 @@ private TestRunner( @Override SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) { - final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() + final SplittableInputSource baseInputSource = (SplittableInputSource) getIngestionSchema() .getIOConfig() - .getFirehoseFactory(); + .getInputSource(); return new TestSinglePhaseSubTaskSpec( supervisorTask.getId() + "_" + getAndIncrementNextSpecId(), supervisorTask.getGroupId(), @@ -341,7 +355,9 @@ SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) new ParallelIndexIngestionSpec( getIngestionSchema().getDataSchema(), new ParallelIndexIOConfig( - baseFirehoseFactory.withSplit(split), + null, + baseInputSource.withSplit(split), + getIngestionSchema().getIOConfig().getInputFormat(), getIngestionSchema().getIOConfig().isAppendToExisting() ), getIngestionSchema().getTuningConfig() @@ -423,9 +439,8 @@ public boolean isReady(TaskActionClient taskActionClient) @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { - final TestFirehoseFactory firehoseFactory = (TestFirehoseFactory) getIngestionSchema().getIOConfig() - .getFirehoseFactory(); - final TestInput testInput = Iterables.getOnlyElement(firehoseFactory.splits).get(); + final TestInputSource inputSource = (TestInputSource) getIngestionSchema().getIOConfig().getInputSource(); + final TestInput testInput = Iterables.getOnlyElement(inputSource.splits).get(); Thread.sleep(testInput.runTime); return TaskStatus.fromCode(getId(), testInput.finalState); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index d55ce70a61b1..dc2c871e9f8c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -22,11 +22,15 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.FiniteFirehoseFactory; +import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputFormat; +import org.apache.druid.data.input.impl.NoopInputFormat; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; @@ -62,6 +66,7 @@ import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -85,19 +90,29 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd { private static final int NUM_SUB_TASKS = 10; - /** specId -> spec */ + /** + * specId -> spec + */ private final ConcurrentMap subTaskSpecs = new ConcurrentHashMap<>(); - /** specId -> taskStatusPlus */ + /** + * specId -> taskStatusPlus + */ private final ConcurrentMap runningSpecs = new ConcurrentHashMap<>(); - /** specId -> taskStatusPlus list */ + /** + * specId -> taskStatusPlus list + */ private final ConcurrentHashMap> taskHistories = new ConcurrentHashMap<>(); - /** taskId -> subTaskSpec */ + /** + * taskId -> subTaskSpec + */ private final ConcurrentMap taskIdToSpec = new ConcurrentHashMap<>(); - /** taskId -> task */ + /** + * taskId -> task + */ private final CopyOnWriteArrayList runningTasks = new CopyOnWriteArrayList<>(); private ExecutorService service; @@ -126,7 +141,9 @@ public void testAPIs() throws Exception task = newTask( Intervals.of("2017/2018"), new ParallelIndexIOConfig( - new TestFirehose(IntStream.range(0, NUM_SUB_TASKS).boxed().collect(Collectors.toList())), + null, + new TestInputSource(IntStream.range(0, NUM_SUB_TASKS).boxed().collect(Collectors.toList())), + new NoopInputFormat(), false ) ); @@ -149,7 +166,10 @@ public void testAPIs() throws Exception // test expectedNumSucceededTasks response = task.getProgress(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(NUM_SUB_TASKS, ((SinglePhaseParallelIndexingProgress) response.getEntity()).getExpectedSucceeded()); + Assert.assertEquals( + NUM_SUB_TASKS, + ((SinglePhaseParallelIndexingProgress) response.getEntity()).getExpectedSucceeded() + ); // Since taskMonitor works based on polling, it's hard to use a fancier way to check its state. // We use polling to check the state of taskMonitor in this test. @@ -188,7 +208,8 @@ public void testAPIs() throws Exception } // Wait for new tasks to be started - while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getFailed) < failedTasks || runningTasks.size() < NUM_SUB_TASKS - succeededTasks) { + while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getFailed) < failedTasks + || runningTasks.size() < NUM_SUB_TASKS - succeededTasks) { Thread.sleep(100); } @@ -364,7 +385,8 @@ private void checkState( .filter(entry -> { final TaskStatusPlus currentStatus = entry.getValue().getCurrentStatus(); return currentStatus != null && - (currentStatus.getStatusCode() == TaskState.SUCCESS || currentStatus.getStatusCode() == TaskState.FAILED); + (currentStatus.getStatusCode() == TaskState.SUCCESS + || currentStatus.getStatusCode() == TaskState.FAILED); }) .map(Entry::getKey) .findFirst() @@ -398,13 +420,8 @@ private TestSupervisorTask newTask( final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( new DataSchema( "dataSource", - getObjectMapper().convertValue( - new StringInputRowParser( - DEFAULT_PARSE_SPEC, - null - ), - Map.class - ), + DEFAULT_TIMESTAMP_SPEC, + DEFAULT_DIMENSIONS_SPEC, new AggregatorFactory[]{ new LongSumAggregatorFactory("val", "val") }, @@ -413,8 +430,7 @@ private TestSupervisorTask newTask( Granularities.MINUTE, interval == null ? null : Collections.singletonList(interval) ), - null, - getObjectMapper() + null ), ioConfig, new ParallelIndexTuningConfig( @@ -457,31 +473,42 @@ private TestSupervisorTask newTask( ); } - private static class TestFirehose implements FiniteFirehoseFactory + private static class TestInputSource implements SplittableInputSource { private final List ids; - TestFirehose(List ids) + TestInputSource(List ids) { this.ids = ids; } @Override - public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec) + public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) { return ids.stream().map(InputSplit::new); } @Override - public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) + public int getNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) { return ids.size(); } @Override - public FiniteFirehoseFactory withSplit(InputSplit split) + public SplittableInputSource withSplit(InputSplit split) + { + return new TestInputSource(Collections.singletonList(split.get())); + } + + @Override + public InputSourceReader reader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) { - return new TestFirehose(Collections.singletonList(split.get())); + throw new UnsupportedOperationException(); } } @@ -539,9 +566,9 @@ private class TestRunner extends TestSinglePhaseParallelIndexTaskRunner @Override SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) { - final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() + final TestInputSource baseInputSource = (TestInputSource) getIngestionSchema() .getIOConfig() - .getFirehoseFactory(); + .getInputSource(); final TestSubTaskSpec spec = new TestSubTaskSpec( supervisorTask.getId() + "_" + getAndIncrementNextSpecId(), supervisorTask.getGroupId(), @@ -549,7 +576,9 @@ SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) new ParallelIndexIngestionSpec( getIngestionSchema().getDataSchema(), new ParallelIndexIOConfig( - baseFirehoseFactory.withSplit(split), + null, + baseInputSource.withSplit(split), + getIngestionSchema().getIOConfig().getInputFormat(), getIngestionSchema().getIOConfig().isAppendToExisting() ), getIngestionSchema().getTuningConfig() @@ -597,8 +626,10 @@ public SinglePhaseSubTask newSubTask(int numAttempts) getContext(), new LocalParallelIndexTaskClientFactory(supervisorTask) ); - final TestFirehose firehose = (TestFirehose) getIngestionSpec().getIOConfig().getFirehoseFactory(); - final InputSplit split = firehose.getSplits(null).findFirst().orElse(null); + final TestInputSource inputSource = (TestInputSource) getIngestionSpec().getIOConfig().getInputSource(); + final InputSplit split = inputSource.createSplits(getIngestionSpec().getIOConfig().getInputFormat(), null) + .findFirst() + .orElse(null); if (split == null) { throw new ISE("Split is null"); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index c8857aaf105b..21ad0a4632c5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -19,15 +19,13 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient; -import org.apache.druid.data.input.impl.CSVParseSpec; +import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -57,7 +55,6 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -171,29 +168,15 @@ ParallelIndexSupervisorTask build() private static class ParallelIndexIngestionSpecBuilder { - private static final ParseSpec PARSE_SPEC = new CSVParseSpec( - new TimestampSpec( - "ts", - "auto", - null - ), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")), - new ArrayList<>(), - new ArrayList<>() - ), - null, - Arrays.asList("ts", "dim", "val"), - false, - 0 + private static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec("ts", "auto", null); + private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")) ); - private static final TypeReference> PARSER_TYPE = new TypeReference>() - { - }; - private final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - new LocalFirehoseFactory(new File("tmp"), "test_*", null), + null, + new LocalInputSource(new File("tmp"), "test_*"), + new CsvInputFormat(Arrays.asList("ts", "dim", "val"), null, false, 0), false ); @@ -230,16 +213,13 @@ ParallelIndexIngestionSpec build() { DataSchema dataSchema = new DataSchema( "dataSource", - OBJECT_MAPPER.convertValue( - new StringInputRowParser(PARSE_SPEC, null), - PARSER_TYPE - ), + TIMESTAMP_SPEC, + DIMENSIONS_SPEC, new AggregatorFactory[]{ new LongSumAggregatorFactory("val", "val") }, new UniformGranularitySpec(Granularities.DAY, Granularities.MINUTE, inputIntervals), - null, - OBJECT_MAPPER + null ); ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index 0c17b8c77abf..c2efc2aed18b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -21,8 +21,9 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.common.LockGranularity; @@ -67,21 +68,26 @@ @RunWith(Parameterized.class) public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSupervisorTaskTest { - @Parameterized.Parameters(name = "{0}") + @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}") public static Iterable constructorFeeder() { return ImmutableList.of( - new Object[]{LockGranularity.TIME_CHUNK}, - new Object[]{LockGranularity.SEGMENT} + new Object[]{LockGranularity.TIME_CHUNK, false}, + new Object[]{LockGranularity.TIME_CHUNK, true}, + new Object[]{LockGranularity.SEGMENT, false}, + new Object[]{LockGranularity.SEGMENT, true} ); } private final LockGranularity lockGranularity; + private final boolean useInputFormatApi; + private File inputDir; - public SinglePhaseParallelIndexingTest(LockGranularity lockGranularity) + public SinglePhaseParallelIndexingTest(LockGranularity lockGranularity, boolean useInputFormatApi) { this.lockGranularity = lockGranularity; + this.useInputFormatApi = useInputFormatApi; } @Before @@ -118,13 +124,7 @@ public void teardown() @Test public void testIsReady() throws Exception { - final ParallelIndexSupervisorTask task = newTask( - Intervals.of("2017/2018"), - new ParallelIndexIOConfig( - new LocalFirehoseFactory(inputDir, "test_*", null), - false - ) - ); + final ParallelIndexSupervisorTask task = newTask(Intervals.of("2017/2018"), false, true); actionClient = createActionClient(task); toolbox = createTaskToolbox(task); @@ -157,14 +157,7 @@ public void testIsReady() throws Exception private void runTestTask(@Nullable Interval interval, Granularity segmentGranularity, boolean appendToExisting) throws Exception { - final ParallelIndexSupervisorTask task = newTask( - interval, - segmentGranularity, - new ParallelIndexIOConfig( - new LocalFirehoseFactory(inputDir, "test_*", null), - appendToExisting - ) - ); + final ParallelIndexSupervisorTask task = newTask(interval, segmentGranularity, appendToExisting, true); actionClient = createActionClient(task); toolbox = createTaskToolbox(task); @@ -233,20 +226,7 @@ public void testRunInParallelWithDifferentSegmentGranularity() throws Exception @Test public void testRunInSequential() throws Exception { - final ParallelIndexSupervisorTask task = newTask( - Intervals.of("2017/2018"), - new ParallelIndexIOConfig( - new LocalFirehoseFactory(inputDir, "test_*", null) - { - @Override - public boolean isSplittable() - { - return false; - } - }, - false - ) - ); + final ParallelIndexSupervisorTask task = newTask(Intervals.of("2017/2018"), false, false); actionClient = createActionClient(task); toolbox = createTaskToolbox(task); @@ -259,13 +239,7 @@ public boolean isSplittable() @Test public void testPublishEmptySegments() throws Exception { - final ParallelIndexSupervisorTask task = newTask( - Intervals.of("2020/2021"), - new ParallelIndexIOConfig( - new LocalFirehoseFactory(inputDir, "test_*", null), - false - ) - ); + final ParallelIndexSupervisorTask task = newTask(Intervals.of("2020/2021"), false, true); actionClient = createActionClient(task); toolbox = createTaskToolbox(task); @@ -281,10 +255,8 @@ public void testWith1MaxNumConcurrentSubTasks() throws Exception final ParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), Granularities.DAY, - new ParallelIndexIOConfig( - new LocalFirehoseFactory(inputDir, "test_*", null), - false - ), + false, + true, new ParallelIndexTuningConfig( null, null, @@ -343,21 +315,27 @@ public void testAppendToExisting() throws Exception Assert.assertEquals(newSegments, visibles); } - private ParallelIndexSupervisorTask newTask(@Nullable Interval interval, ParallelIndexIOConfig ioConfig) + private ParallelIndexSupervisorTask newTask( + @Nullable Interval interval, + boolean appendToExisting, + boolean splittableInputSource + ) { - return newTask(interval, Granularities.DAY, ioConfig); + return newTask(interval, Granularities.DAY, appendToExisting, splittableInputSource); } private ParallelIndexSupervisorTask newTask( @Nullable Interval interval, Granularity segmentGranularity, - ParallelIndexIOConfig ioConfig + boolean appendToExisting, + boolean splittableInputSource ) { return newTask( interval, segmentGranularity, - ioConfig, + appendToExisting, + splittableInputSource, new ParallelIndexTuningConfig( null, null, @@ -392,36 +370,73 @@ private ParallelIndexSupervisorTask newTask( private ParallelIndexSupervisorTask newTask( @Nullable Interval interval, Granularity segmentGranularity, - ParallelIndexIOConfig ioConfig, + boolean appendToExisting, + boolean splittableInputSource, ParallelIndexTuningConfig tuningConfig ) { // set up ingestion spec - //noinspection unchecked - final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - getObjectMapper().convertValue( - new StringInputRowParser( - DEFAULT_PARSE_SPEC, - null - ), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - segmentGranularity, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null, - getObjectMapper() - ), - ioConfig, - tuningConfig - ); + final ParallelIndexIngestionSpec ingestionSpec; + if (useInputFormatApi) { + ingestionSpec = new ParallelIndexIngestionSpec( + new DataSchema( + "dataSource", + DEFAULT_TIMESTAMP_SPEC, + DEFAULT_DIMENSIONS_SPEC, + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + new UniformGranularitySpec( + segmentGranularity, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ), + null + ), + new ParallelIndexIOConfig( + null, + new LocalInputSource(inputDir, "test_*") + { + @Override + public boolean isSplittable() + { + return splittableInputSource; + } + }, + DEFAULT_INPUT_FORMAT, + appendToExisting + ), + tuningConfig + ); + } else { + ingestionSpec = new ParallelIndexIngestionSpec( + new DataSchema( + "dataSource", + getObjectMapper().convertValue( + new StringInputRowParser( + DEFAULT_PARSE_SPEC, + null + ), + Map.class + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + new UniformGranularitySpec( + segmentGranularity, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ), + null, + getObjectMapper() + ), + new ParallelIndexIOConfig( + new LocalFirehoseFactory(inputDir, "test_*", null), + appendToExisting + ), + tuningConfig + ); + } // set up test tools return new TestSupervisorTask( @@ -477,9 +492,9 @@ public static class TestSinglePhaseRunner extends TestSinglePhaseParallelIndexTa @Override SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) { - final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() + final SplittableInputSource baseInputSource = (SplittableInputSource) getIngestionSchema() .getIOConfig() - .getFirehoseFactory(); + .getNonNullInputSource(getIngestionSchema().getDataSchema().getParser()); return new TestSinglePhaseSubTaskSpec( supervisorTask.getId() + "_" + getAndIncrementNextSpecId(), supervisorTask.getGroupId(), @@ -487,7 +502,9 @@ SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) new ParallelIndexIngestionSpec( getIngestionSchema().getDataSchema(), new ParallelIndexIOConfig( - baseFirehoseFactory.withSplit(split), + null, + baseInputSource.withSplit(split), + getIngestionSchema().getIOConfig().getInputFormat(), getIngestionSchema().getIOConfig().isAppendToExisting() ), getIngestionSchema().getTuningConfig() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 74c61d563eef..e8f49803078b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -19,8 +19,6 @@ package org.apache.druid.indexing.overlord; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Function; @@ -38,8 +36,17 @@ import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.impl.NoopInputFormat; +import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DruidNodeAnnouncer; import org.apache.druid.discovery.LookupNodeService; @@ -79,6 +86,7 @@ import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -88,6 +96,8 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Comparators; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.Monitor; @@ -102,6 +112,7 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.RealtimeTuningConfig; @@ -299,23 +310,40 @@ public void close() } } - private static class MockFirehoseFactory implements FirehoseFactory + private static class MockInputSource implements InputSource { - @JsonProperty - private boolean usedByRealtimeIdxTask; - - @JsonCreator - public MockFirehoseFactory(@JsonProperty("usedByRealtimeIdxTask") boolean usedByRealtimeIdxTask) + @Override + public InputSourceReader reader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) { - this.usedByRealtimeIdxTask = usedByRealtimeIdxTask; + return new InputSourceReader() + { + @Override + public CloseableIterator read() + { + final Iterator inputRowIterator = IDX_TASK_INPUT_ROWS.iterator(); + return CloseableIterators.withEmptyBaggage(inputRowIterator); + } + + @Override + public CloseableIterator sample() + { + throw new UnsupportedOperationException(); + } + }; } + } + private static class MockFirehoseFactory implements FirehoseFactory + { @Override public Firehose connect(InputRowParser parser, File temporaryDirectory) { - final Iterator inputRowIterator = usedByRealtimeIdxTask - ? REALTIME_IDX_TASK_INPUT_ROWS.iterator() - : IDX_TASK_INPUT_ROWS.iterator(); + final Iterator inputRowIterator = REALTIME_IDX_TASK_INPUT_ROWS.iterator(); return new Firehose() { @@ -397,7 +425,9 @@ private TaskStorage setUpTaskStorage() TestDerbyConnector testDerbyConnector = derbyConnectorRule.getConnector(); mapper.registerSubtypes( new NamedType(MockExceptionalFirehoseFactory.class, "mockExcepFirehoseFactory"), - new NamedType(MockFirehoseFactory.class, "mockFirehoseFactory") + new NamedType(MockFirehoseFactory.class, "mockFirehoseFactory"), + new NamedType(MockInputSource.class, "mockInputSource"), + new NamedType(NoopInputFormat.class, "noopInputFormat") ); testDerbyConnector.createTaskTables(); testDerbyConnector.createSegmentTable(); @@ -667,17 +697,17 @@ public void testIndexTask() throws Exception new IndexIngestionSpec( new DataSchema( "foo", - null, + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, new UniformGranularitySpec( Granularities.DAY, null, ImmutableList.of(Intervals.of("2010-01-01/P2D")) ), - null, - mapper + null ), - new IndexIOConfig(new MockFirehoseFactory(false), false), + new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false), new IndexTuningConfig( null, 10000, @@ -1173,17 +1203,17 @@ public void testResumeTasks() throws Exception new IndexIngestionSpec( new DataSchema( "foo", - null, + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, new UniformGranularitySpec( Granularities.DAY, null, ImmutableList.of(Intervals.of("2010-01-01/P2D")) ), - null, - mapper + null ), - new IndexIOConfig(new MockFirehoseFactory(false), false), + new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false), new IndexTuningConfig( null, 10000, @@ -1279,17 +1309,17 @@ public void testUnifiedAppenderatorsManagerCleanup() throws Exception new IndexIngestionSpec( new DataSchema( "foo", - null, + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, new UniformGranularitySpec( Granularities.DAY, null, ImmutableList.of(Intervals.of("2010-01-01/P2D")) ), - null, - mapper + null ), - new IndexIOConfig(new MockFirehoseFactory(false), false), + new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false), new IndexTuningConfig( null, 10000, @@ -1387,14 +1417,22 @@ private RealtimeIndexTask newRealtimeIndexTask() String taskId = StringUtils.format("rt_task_%s", System.currentTimeMillis()); DataSchema dataSchema = new DataSchema( "test_ds", - null, + TestHelper.makeJsonMapper().convertValue( + new MapInputRowParser( + new TimeAndDimsParseSpec( + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY + ) + ), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ), new AggregatorFactory[]{new LongSumAggregatorFactory("count", "rows")}, new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), null, mapper ); RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig( - new MockFirehoseFactory(true), + new MockFirehoseFactory(), null // PlumberSchool - Realtime Index Task always uses RealtimePlumber which is hardcoded in RealtimeIndexTask class ); diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java index 87d75f8c7e2e..4b96cc4257ee 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java @@ -27,6 +27,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.VirtualColumns; +import javax.annotation.Nullable; + /** */ public class IncrementalIndexSchema @@ -149,13 +151,14 @@ public Builder withVirtualColumns(VirtualColumns virtualColumns) return this; } - public Builder withDimensionsSpec(DimensionsSpec dimensionsSpec) + public Builder withDimensionsSpec(@Nullable DimensionsSpec dimensionsSpec) { this.dimensionsSpec = dimensionsSpec == null ? DimensionsSpec.EMPTY : dimensionsSpec; return this; } - public Builder withDimensionsSpec(InputRowParser parser) + @Deprecated + public Builder withDimensionsSpec(@Nullable InputRowParser parser) { if (parser != null && parser.getParseSpec() != null diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java index 159187788f6f..c5418609edca 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.java.util.common.ISE; @@ -111,13 +112,18 @@ public InputRowParser decorate(final InputRowParser parser) } } + public InputSourceReader decorate(InputSourceReader reader) + { + return new TransformingReader(reader, toTransformer()); + } + /** * Create a {@link Transformer} from this TransformSpec, when the rows to be transformed do not have a known * signature. */ public Transformer toTransformer() { - return new Transformer(this, null); + return new Transformer(this); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java index cb1a8b68569e..c18e0ddcb54f 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java @@ -20,13 +20,13 @@ package org.apache.druid.segment.transform; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowPlusRaw; import org.apache.druid.data.input.Row; import org.apache.druid.data.input.Rows; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.groupby.RowBasedColumnSelectorFactory; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -44,7 +44,7 @@ public class Transformer private final ThreadLocal rowSupplierForValueMatcher = new ThreadLocal<>(); private final ValueMatcher valueMatcher; - Transformer(final TransformSpec transformSpec, final Map rowSignature) + Transformer(final TransformSpec transformSpec) { for (final Transform transform : transformSpec.getTransforms()) { transforms.put(transform.getName(), transform.getRowFunction()); @@ -55,7 +55,7 @@ public class Transformer .makeMatcher( RowBasedColumnSelectorFactory.create( rowSupplierForValueMatcher::get, - rowSignature + null ) ); } else { @@ -93,6 +93,31 @@ public InputRow transform(@Nullable final InputRow row) return transformedRow; } + @Nullable + public InputRowPlusRaw transform(@Nullable final InputRowPlusRaw row) + { + if (row == null) { + return null; + } + + final InputRowPlusRaw transformedRow; + + if (transforms.isEmpty()) { + transformedRow = row; + } else { + transformedRow = InputRowPlusRaw.of(new TransformedInputRow(row.getInputRow(), transforms), row.getRaw()); + } + + if (valueMatcher != null) { + rowSupplierForValueMatcher.set(transformedRow.getInputRow()); + if (!valueMatcher.matches()) { + return null; + } + } + + return transformedRow; + } + public static class TransformedInputRow implements InputRow { private final InputRow row; diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformingReader.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformingReader.java new file mode 100644 index 000000000000..87f441534ca9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformingReader.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.transform; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import java.io.IOException; + +public class TransformingReader implements InputSourceReader +{ + private final InputSourceReader delegate; + private final Transformer transformer; + + TransformingReader(InputSourceReader delegate, Transformer transformer) + { + this.delegate = delegate; + this.transformer = transformer; + } + + @Override + public CloseableIterator read() throws IOException + { + return delegate.read().map(transformer::transform); + } + + @Override + public CloseableIterator sample() throws IOException + { + return delegate.sample().map(transformer::transform); + } +} diff --git a/server/src/main/java/org/apache/druid/segment/indexing/BatchIOConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/BatchIOConfig.java new file mode 100644 index 000000000000..f53f22acdece --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/indexing/BatchIOConfig.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.indexing; + +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.impl.InputFormat; + +/** + * IOConfig for all batch tasks except compactionTask. + */ +public interface BatchIOConfig extends IOConfig +{ + InputSource getInputSource(); + + InputFormat getInputFormat(); + + boolean isAppendToExisting(); +} diff --git a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java index 6ce4f0d985cf..43968b77b1ed 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java @@ -21,7 +21,8 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; @@ -29,6 +30,7 @@ import com.google.common.collect.Sets; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.logger.Logger; @@ -37,12 +39,14 @@ import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.transform.TransformSpec; +import javax.annotation.Nullable; import java.util.Arrays; import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; /** @@ -53,38 +57,53 @@ public class DataSchema private static final Logger log = new Logger(DataSchema.class); private static final Pattern INVALIDCHARS = Pattern.compile("(?s).*[^\\S ].*"); private final String dataSource; - private final Map parser; private final AggregatorFactory[] aggregators; private final GranularitySpec granularitySpec; private final TransformSpec transformSpec; + private final Map parserMap; + private final ObjectMapper objectMapper; - private final ObjectMapper jsonMapper; + // The below fields can be initialized lazily from parser for backward compatibility. + private TimestampSpec timestampSpec; + private DimensionsSpec dimensionsSpec; - private InputRowParser cachedParser; + // This is used for backward compatibility + private InputRowParser inputRowParser; @JsonCreator public DataSchema( @JsonProperty("dataSource") String dataSource, - @JsonProperty("parser") Map parser, + @JsonProperty("timestampSpec") @Nullable TimestampSpec timestampSpec, // can be null in old task spec + @JsonProperty("dimensionsSpec") @Nullable DimensionsSpec dimensionsSpec, // can be null in old task spec @JsonProperty("metricsSpec") AggregatorFactory[] aggregators, @JsonProperty("granularitySpec") GranularitySpec granularitySpec, @JsonProperty("transformSpec") TransformSpec transformSpec, - @JacksonInject ObjectMapper jsonMapper + @Deprecated @JsonProperty("parser") @Nullable Map parserMap, + @JacksonInject ObjectMapper objectMapper ) { - this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "null ObjectMapper."); - this.parser = parser; - this.transformSpec = transformSpec == null ? TransformSpec.NONE : transformSpec; - validateDatasourceName(dataSource); this.dataSource = dataSource; + this.timestampSpec = timestampSpec; + this.dimensionsSpec = dimensionsSpec == null + ? null + : computeDimensionsSpec( + Preconditions.checkNotNull(timestampSpec, "timestampSpec"), + dimensionsSpec, + aggregators + ); + + this.aggregators = aggregators == null ? new AggregatorFactory[]{} : aggregators; if (granularitySpec == null) { log.warn("No granularitySpec has been specified. Using UniformGranularitySpec as default."); this.granularitySpec = new UniformGranularitySpec(null, null, null); } else { this.granularitySpec = granularitySpec; } + this.transformSpec = transformSpec == null ? TransformSpec.NONE : transformSpec; + this.parserMap = parserMap; + this.objectMapper = objectMapper; if (aggregators != null && aggregators.length != 0) { // validate for no duplication @@ -97,11 +116,35 @@ public DataSchema( } else if (this.granularitySpec.isRollup()) { log.warn("No metricsSpec has been specified. Are you sure this is what you want?"); } + } - this.aggregators = aggregators == null ? new AggregatorFactory[]{} : aggregators; + public DataSchema( + String dataSource, + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + AggregatorFactory[] aggregators, + GranularitySpec granularitySpec, + TransformSpec transformSpec + ) + { + this(dataSource, timestampSpec, dimensionsSpec, aggregators, granularitySpec, transformSpec, null, null); } - static void validateDatasourceName(String dataSource) + // old constructor for backward compatibility + @Deprecated + public DataSchema( + String dataSource, + Map parserMap, + AggregatorFactory[] aggregators, + GranularitySpec granularitySpec, + TransformSpec transformSpec, + ObjectMapper objectMapper + ) + { + this(dataSource, null, null, aggregators, granularitySpec, transformSpec, parserMap, objectMapper); + } + + private static void validateDatasourceName(String dataSource) { Preconditions.checkArgument( !Strings.isNullOrEmpty(dataSource), @@ -115,83 +158,75 @@ static void validateDatasourceName(String dataSource) Preconditions.checkArgument(!dataSource.contains("/"), "dataSource cannot contain the '/' character."); } + private static DimensionsSpec computeDimensionsSpec( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + AggregatorFactory[] aggregators + ) + { + final Set dimensionExclusions = new HashSet<>(); + + final String timestampColumn = timestampSpec.getTimestampColumn(); + if (!(dimensionsSpec.hasCustomDimensions() && dimensionsSpec.getDimensionNames().contains(timestampColumn))) { + dimensionExclusions.add(timestampColumn); + } + + for (AggregatorFactory aggregator : aggregators) { + dimensionExclusions.addAll(aggregator.requiredFields()); + dimensionExclusions.add(aggregator.getName()); + } + + final Set metSet = Arrays.stream(aggregators).map(AggregatorFactory::getName).collect(Collectors.toSet()); + final Set dimSet = new HashSet<>(dimensionsSpec.getDimensionNames()); + final Set overlap = Sets.intersection(metSet, dimSet); + if (!overlap.isEmpty()) { + throw new IAE( + "Cannot have overlapping dimensions and metrics of the same name. Please change the name of the metric. Overlap: %s", + overlap + ); + } + + return dimensionsSpec.withDimensionExclusions(Sets.difference(dimensionExclusions, dimSet)); + } + @JsonProperty public String getDataSource() { return dataSource; } - @JsonProperty("parser") - public Map getParserMap() + @Nullable + @JsonProperty + public TimestampSpec getTimestampSpec() { - return parser; + return timestampSpec; } - @JsonIgnore - public InputRowParser getParser() + public TimestampSpec getNonNullTimestampSpec() { - if (parser == null) { - log.warn("No parser has been specified"); - return null; - } - - if (cachedParser != null) { - return cachedParser; - } - - final InputRowParser inputRowParser = transformSpec.decorate( - jsonMapper.convertValue(this.parser, InputRowParser.class) - ); - - final Set dimensionExclusions = new HashSet<>(); - for (AggregatorFactory aggregator : aggregators) { - dimensionExclusions.addAll(aggregator.requiredFields()); - dimensionExclusions.add(aggregator.getName()); + if (timestampSpec == null) { + timestampSpec = Preconditions.checkNotNull(getParser(), "inputRowParser").getParseSpec().getTimestampSpec(); } + return timestampSpec; + } - if (inputRowParser.getParseSpec() != null) { - final DimensionsSpec dimensionsSpec = inputRowParser.getParseSpec().getDimensionsSpec(); - final TimestampSpec timestampSpec = inputRowParser.getParseSpec().getTimestampSpec(); - - // exclude timestamp from dimensions by default, unless explicitly included in the list of dimensions - if (timestampSpec != null) { - final String timestampColumn = timestampSpec.getTimestampColumn(); - if (!(dimensionsSpec.hasCustomDimensions() && dimensionsSpec.getDimensionNames().contains(timestampColumn))) { - dimensionExclusions.add(timestampColumn); - } - } - if (dimensionsSpec != null) { - final Set metSet = new HashSet<>(); - for (AggregatorFactory aggregator : aggregators) { - metSet.add(aggregator.getName()); - } - final Set dimSet = Sets.newHashSet(dimensionsSpec.getDimensionNames()); - final Set overlap = Sets.intersection(metSet, dimSet); - if (!overlap.isEmpty()) { - throw new IAE( - "Cannot have overlapping dimensions and metrics of the same name. Please change the name of the metric. Overlap: %s", - overlap - ); - } + @Nullable + @JsonProperty + public DimensionsSpec getDimensionsSpec() + { + return dimensionsSpec; + } - cachedParser = inputRowParser.withParseSpec( - inputRowParser.getParseSpec() - .withDimensionsSpec( - dimensionsSpec - .withDimensionExclusions( - Sets.difference(dimensionExclusions, dimSet) - ) - ) - ); - } else { - cachedParser = inputRowParser; - } - } else { - log.warn("No parseSpec in parser has been specified."); - cachedParser = inputRowParser; + public DimensionsSpec getNonNullDimensionsSpec() + { + if (dimensionsSpec == null) { + dimensionsSpec = computeDimensionsSpec( + getNonNullTimestampSpec(), + Preconditions.checkNotNull(getParser(), "inputRowParser").getParseSpec().getDimensionsSpec(), + aggregators + ); } - - return cachedParser; + return dimensionsSpec; } @JsonProperty("metricsSpec") @@ -212,14 +247,62 @@ public TransformSpec getTransformSpec() return transformSpec; } + @Deprecated + @JsonProperty("parser") + @Nullable + @JsonInclude(Include.NON_NULL) + public Map getParserMap() + { + return parserMap; + } + + @Nullable + public InputRowParser getParser() + { + if (inputRowParser == null) { + if (parserMap == null) { + return null; + } + //noinspection unchecked + inputRowParser = transformSpec.decorate(objectMapper.convertValue(this.parserMap, InputRowParser.class)); + ParseSpec parseSpec = inputRowParser.getParseSpec(); + if (timestampSpec != null) { + parseSpec = parseSpec.withTimestampSpec(timestampSpec); + } + if (dimensionsSpec != null) { + parseSpec = parseSpec.withDimensionsSpec(dimensionsSpec); + } + inputRowParser = inputRowParser.withParseSpec(parseSpec); + } + return inputRowParser; + } + public DataSchema withGranularitySpec(GranularitySpec granularitySpec) { - return new DataSchema(dataSource, parser, aggregators, granularitySpec, transformSpec, jsonMapper); + return new DataSchema( + dataSource, + timestampSpec, + dimensionsSpec, + aggregators, + granularitySpec, + transformSpec, + parserMap, + objectMapper + ); } public DataSchema withTransformSpec(TransformSpec transformSpec) { - return new DataSchema(dataSource, parser, aggregators, granularitySpec, transformSpec, jsonMapper); + return new DataSchema( + dataSource, + timestampSpec, + dimensionsSpec, + aggregators, + granularitySpec, + transformSpec, + parserMap, + objectMapper + ); } @Override @@ -227,10 +310,12 @@ public String toString() { return "DataSchema{" + "dataSource='" + dataSource + '\'' + - ", parser=" + parser + ", aggregators=" + Arrays.toString(aggregators) + ", granularitySpec=" + granularitySpec + ", transformSpec=" + transformSpec + + (parserMap == null ? "" : ", parserMap=" + parserMap) + + ", timestampSpec=" + timestampSpec + + ", dimensionsSpec=" + dimensionsSpec + '}'; } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/FireDepartment.java b/server/src/main/java/org/apache/druid/segment/realtime/FireDepartment.java index 2edf910c003f..2c25a1de1749 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/FireDepartment.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/FireDepartment.java @@ -89,7 +89,8 @@ public RealtimeTuningConfig getTuningConfig() public Firehose connect() throws IOException { - return ioConfig.getFirehoseFactory().connect(dataSchema.getParser(), null); + return ioConfig.getFirehoseFactory() + .connect(Preconditions.checkNotNull(dataSchema.getParser(), "inputRowParser"), null); } public FireDepartmentMetrics getMetrics() diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java index 860f6cc11061..202804195121 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java @@ -23,10 +23,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; -import com.google.common.base.Strings; import com.google.common.net.HttpHeaders; import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.impl.HttpSource; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; import org.apache.druid.java.util.common.StringUtils; @@ -39,7 +39,6 @@ import java.io.InputStream; import java.net.URI; import java.net.URLConnection; -import java.util.Base64; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -109,7 +108,11 @@ protected InputStream openObjectStream(URI object) throws IOException @Override protected InputStream openObjectStream(URI object, long start) throws IOException { - URLConnection urlConnection = openURLConnection(object); + URLConnection urlConnection = HttpSource.openURLConnection( + object, + httpAuthenticationUsername, + httpAuthenticationPasswordProvider + ); final String acceptRanges = urlConnection.getHeaderField(HttpHeaders.ACCEPT_RANGES); final boolean withRanges = "bytes".equalsIgnoreCase(acceptRanges); if (withRanges && start > 0) { @@ -195,15 +198,4 @@ public FiniteFirehoseFactory withSplit(InputSplit { ResultIterator> resultIterator = handle.createQuery( - object + sql ).map( (index, r, ctx) -> { Map resultRow = foldCase ? new CaseFoldedMap() : new HashMap<>(); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java index c62533cfce87..ba405f886724 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java @@ -354,9 +354,9 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) { final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder() .withMinTimestamp(minTimestamp) - .withTimestampSpec(schema.getParser()) + .withTimestampSpec(schema.getNonNullTimestampSpec()) .withQueryGranularity(schema.getGranularitySpec().getQueryGranularity()) - .withDimensionsSpec(schema.getParser()) + .withDimensionsSpec(schema.getNonNullDimensionsSpec()) .withMetrics(schema.getAggregators()) .withRollup(schema.getGranularitySpec().isRollup()) .build(); diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 437efd313ce0..435080ffddbc 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -92,7 +92,7 @@ public void testDefaultExclusions() Assert.assertEquals( ImmutableSet.of("time", "col1", "col2", "metric1", "metric2"), - schema.getParser().getParseSpec().getDimensionsSpec().getDimensionExclusions() + schema.getNonNullDimensionsSpec().getDimensionExclusions() ); } @@ -471,13 +471,15 @@ public void testSerdeWithUpdatedDataSchemaRemovedField() throws IOException TestModifiedDataSchema originalSchema = new TestModifiedDataSchema( "test", - parser, + null, + null, new AggregatorFactory[]{ new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric2", "col2"), }, new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), null, + parser, jsonMapper, "some arbitrary string" ); diff --git a/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java b/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java index ca030fe875c5..b10509036243 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java @@ -23,10 +23,13 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.transform.TransformSpec; +import javax.annotation.Nullable; import java.util.Map; public class TestModifiedDataSchema extends DataSchema @@ -36,15 +39,26 @@ public class TestModifiedDataSchema extends DataSchema @JsonCreator public TestModifiedDataSchema( @JsonProperty("dataSource") String dataSource, - @JsonProperty("parser") Map parser, + @JsonProperty("timestampSpec") TimestampSpec timestampSpec, + @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec, @JsonProperty("metricsSpec") AggregatorFactory[] aggregators, @JsonProperty("granularitySpec") GranularitySpec granularitySpec, @JsonProperty("transformSpec") TransformSpec transformSpec, - @JacksonInject ObjectMapper jsonMapper, + @JsonProperty("parser") @Nullable Map parserMap, + @JacksonInject ObjectMapper objectMapper, @JsonProperty("extra") String extra ) { - super(dataSource, parser, aggregators, granularitySpec, transformSpec, jsonMapper); + super( + dataSource, + timestampSpec, + dimensionsSpec, + aggregators, + granularitySpec, + transformSpec, + parserMap, + objectMapper + ); this.extra = extra; } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java index c1a4066c20fa..36446e3b6246 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java @@ -25,7 +25,8 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.Row; -import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -54,11 +55,11 @@ public void testSwap() throws Exception { final DataSchema schema = new DataSchema( "test", - null, + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), - null, - new DefaultObjectMapper() + null ); final Interval interval = Intervals.of("2013-01-01/2013-01-02"); @@ -209,11 +210,11 @@ public void testDedup() throws Exception { final DataSchema schema = new DataSchema( "test", - null, + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), - null, - new DefaultObjectMapper() + null ); final Interval interval = Intervals.of("2013-01-01/2013-01-02"); From f52f96722ee0ae4ca8eb6443e2c8f41613a09413 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 4 Nov 2019 23:38:57 -0800 Subject: [PATCH 02/29] revert orc dependency --- extensions-core/orc-extensions/pom.xml | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/extensions-core/orc-extensions/pom.xml b/extensions-core/orc-extensions/pom.xml index ec8c8a752ea8..c739bedba4ce 100644 --- a/extensions-core/orc-extensions/pom.xml +++ b/extensions-core/orc-extensions/pom.xml @@ -52,7 +52,7 @@ org.apache.hadoop hadoop-client ${hadoop.compile.version} - compile + provided org.apache.orc @@ -178,17 +178,12 @@ org.apache.hadoop hadoop-mapreduce-client-core - compile - - - org.apache.hadoop - hadoop-hdfs-client - compile + provided org.apache.hadoop hadoop-common - compile + provided com.google.inject From 93ab23f6729510723a1bd72da32ead9721b1781f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 4 Nov 2019 23:53:09 -0800 Subject: [PATCH 03/29] fix dimension exclusions and failing unit tests --- core/src/test/java/org/apache/druid/indexer/ChecksTest.java | 2 +- .../java/org/apache/druid/segment/indexing/DataSchema.java | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/test/java/org/apache/druid/indexer/ChecksTest.java b/core/src/test/java/org/apache/druid/indexer/ChecksTest.java index 63dcd0d84a6f..8a7454354052 100644 --- a/core/src/test/java/org/apache/druid/indexer/ChecksTest.java +++ b/core/src/test/java/org/apache/druid/indexer/ChecksTest.java @@ -140,7 +140,7 @@ public void testCheckOneNotNullOrEmptyWithAllNulls() ); exception.expect(IllegalArgumentException.class); exception.expectMessage( - "At most one of [Property{name='p1', value=null}, Property{name='p2', value=null}, " + "At least one of [Property{name='p1', value=null}, Property{name='p2', value=null}, " + "Property{name='p3', value=null}, Property{name='p4', value=null}] must be present" ); Checks.checkOneNotNullOrEmpty(properties); diff --git a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java index 43968b77b1ed..8a8b0a91d34e 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java @@ -266,6 +266,9 @@ public InputRowParser getParser() //noinspection unchecked inputRowParser = transformSpec.decorate(objectMapper.convertValue(this.parserMap, InputRowParser.class)); ParseSpec parseSpec = inputRowParser.getParseSpec(); + parseSpec = parseSpec.withDimensionsSpec( + computeDimensionsSpec(parseSpec.getTimestampSpec(), parseSpec.getDimensionsSpec(), aggregators) + ); if (timestampSpec != null) { parseSpec = parseSpec.withTimestampSpec(timestampSpec); } From e0b80cb4642b515a236f8699833a3e1493a878bd Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 5 Nov 2019 00:58:23 -0800 Subject: [PATCH 04/29] fix tests --- .../parallel/ParallelIndexIngestionSpec.java | 5 +++- .../druid/indexing/common/TestUtils.java | 9 +++++- .../common/task/CompactionTaskTest.java | 4 +-- .../indexing/common/task/TaskSerdeTest.java | 30 +++++++++++-------- 4 files changed, 31 insertions(+), 17 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java index 5f5fdb2d1d0f..8d9a85d43878 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.indexer.Checks; import org.apache.druid.indexer.Property; import org.apache.druid.java.util.common.IAE; @@ -50,7 +51,9 @@ public ParallelIndexIngestionSpec( ) ); if (dataSchema.getParserMap() != null && ioConfig.getInputSource() != null) { - throw new IAE("Cannot use parser and inputSource together. Try use inputFormat instead of parser."); + if (!(ioConfig.getInputSource() instanceof FirehoseFactoryToInputSourceAdaptor)) { + throw new IAE("Cannot use parser and inputSource together. Try use inputFormat instead of parser."); + } } this.dataSchema = dataSchema; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java index 75a198f2f73b..4c81da3bceb5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java @@ -21,11 +21,14 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.base.Stopwatch; import com.google.common.collect.ImmutableMap; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient; +import org.apache.druid.data.input.impl.NoopInputFormat; +import org.apache.druid.data.input.impl.NoopInputSource; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; @@ -100,7 +103,11 @@ public int columnCacheSizeBytes() @Override public void setupModule(SetupContext context) { - context.registerSubtypes(LocalLoadSpec.class); + context.registerSubtypes( + new NamedType(LocalLoadSpec.class, "local"), + new NamedType(NoopInputSource.class, "noop"), + new NamedType(NoopInputFormat.class, "noop") + ); } } ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 89fb241007aa..8c1a0c87f091 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1057,10 +1057,10 @@ private void assertIngestionSchema( final DataSchema dataSchema = ingestionSchema.getDataSchema(); Assert.assertEquals(DATA_SOURCE, dataSchema.getDataSource()); - Assert.assertEquals(new TimestampSpec(null, null, null), dataSchema.getTimestampSpec()); + Assert.assertEquals(new TimestampSpec(null, null, null), dataSchema.getNonNullTimestampSpec()); Assert.assertEquals( new HashSet<>(expectedDimensionsSpec.getDimensions()), - new HashSet<>(dataSchema.getDimensionsSpec().getDimensions()) + new HashSet<>(dataSchema.getNonNullDimensionsSpec().getDimensions()) ); // metrics diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java index e49e61090887..2ba37ff0bd09 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java @@ -24,6 +24,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.client.indexing.ClientKillQuery; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.data.input.impl.NoopInputFormat; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexer.HadoopIOConfig; import org.apache.druid.indexer.HadoopIngestionSpec; @@ -80,7 +84,7 @@ public TaskSerdeTest() public void testIndexTaskIOConfigDefaults() throws Exception { final IndexTask.IndexIOConfig ioConfig = jsonMapper.readValue( - "{\"type\":\"index\"}", + "{\"type\":\"index\",\"inputSource\":{\"type\":\"noop\"},\"inputFormat\":{\"type\":\"noop\"}}", IndexTask.IndexIOConfig.class ); @@ -184,17 +188,17 @@ public void testIndexTaskSerde() throws Exception new IndexIngestionSpec( new DataSchema( "foo", - null, + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, new UniformGranularitySpec( Granularities.DAY, null, ImmutableList.of(Intervals.of("2010-01-01/P2D")) ), - null, - jsonMapper + null ), - new IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), + new IndexIOConfig(null, new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true), new IndexTuningConfig( null, null, @@ -239,8 +243,8 @@ public void testIndexTaskSerde() throws Exception IndexTask.IndexIOConfig taskIoConfig = task.getIngestionSchema().getIOConfig(); IndexTask.IndexIOConfig task2IoConfig = task2.getIngestionSchema().getIOConfig(); - Assert.assertTrue(taskIoConfig.getFirehoseFactory() instanceof LocalFirehoseFactory); - Assert.assertTrue(task2IoConfig.getFirehoseFactory() instanceof LocalFirehoseFactory); + Assert.assertTrue(taskIoConfig.getInputSource() instanceof LocalInputSource); + Assert.assertTrue(task2IoConfig.getInputSource() instanceof LocalInputSource); Assert.assertEquals(taskIoConfig.isAppendToExisting(), task2IoConfig.isAppendToExisting()); IndexTask.IndexTuningConfig taskTuningConfig = task.getIngestionSchema().getTuningConfig(); @@ -268,17 +272,17 @@ public void testIndexTaskwithResourceSerde() throws Exception new IndexIngestionSpec( new DataSchema( "foo", - null, + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, new UniformGranularitySpec( Granularities.DAY, null, ImmutableList.of(Intervals.of("2010-01-01/P2D")) ), - null, - jsonMapper + null ), - new IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), + new IndexIOConfig(null, new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true), new IndexTuningConfig( null, null, @@ -327,8 +331,8 @@ public void testIndexTaskwithResourceSerde() throws Exception Assert.assertEquals(task.getTaskResource().getAvailabilityGroup(), task2.getTaskResource().getAvailabilityGroup()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); - Assert.assertTrue(task.getIngestionSchema().getIOConfig().getFirehoseFactory() instanceof LocalFirehoseFactory); - Assert.assertTrue(task2.getIngestionSchema().getIOConfig().getFirehoseFactory() instanceof LocalFirehoseFactory); + Assert.assertTrue(task.getIngestionSchema().getIOConfig().getInputSource() instanceof LocalInputSource); + Assert.assertTrue(task2.getIngestionSchema().getIOConfig().getInputSource() instanceof LocalInputSource); } @Test From b4f041e9dda2f92322b1ff428b416c7ea5cb96b5 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 5 Nov 2019 09:48:54 -0800 Subject: [PATCH 05/29] fix test --- .../indexing/overlord/TaskLifecycleTest.java | 45 ++++++++++++------- 1 file changed, 30 insertions(+), 15 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index e8f49803078b..64a8dc0d7d3d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -281,30 +281,46 @@ private static InputRow ir(String dt, String dim1, String dim2, float met) ); } - private static class MockExceptionalFirehoseFactory implements FirehoseFactory + private static class MockExceptionInputSource implements InputSource { @Override - public Firehose connect(InputRowParser parser, File temporaryDirectory) + public InputSourceReader reader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) { - return new Firehose() + return new InputSourceReader() { @Override - public boolean hasMore() + public CloseableIterator read() throws IOException { - return true; - } + return new CloseableIterator() + { + @Override + public void close() throws IOException + { + } - @Nullable - @Override - public InputRow nextRow() - { - throw new RuntimeException("HA HA HA"); + @Override + public boolean hasNext() + { + return true; + } + + @Override + public InputRow next() + { + throw new RuntimeException("HA HA HA"); + } + }; } @Override - public void close() + public CloseableIterator sample() throws IOException { - + throw new UnsupportedOperationException(); } }; } @@ -424,7 +440,6 @@ private TaskStorage setUpTaskStorage() case METADATA_TASK_STORAGE: { TestDerbyConnector testDerbyConnector = derbyConnectorRule.getConnector(); mapper.registerSubtypes( - new NamedType(MockExceptionalFirehoseFactory.class, "mockExcepFirehoseFactory"), new NamedType(MockFirehoseFactory.class, "mockFirehoseFactory"), new NamedType(MockInputSource.class, "mockInputSource"), new NamedType(NoopInputFormat.class, "noopInputFormat") @@ -791,7 +806,7 @@ public void testIndexTaskFailure() throws Exception null, mapper ), - new IndexIOConfig(new MockExceptionalFirehoseFactory(), false), + new IndexIOConfig(null, new MockExceptionInputSource(), new NoopInputFormat(), false), new IndexTuningConfig( null, 10000, From d349db5cd37dd87db6ad8129f0a129539af2f50e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 5 Nov 2019 11:55:52 -0800 Subject: [PATCH 06/29] fix test --- extensions-core/kafka-indexing-service/pom.xml | 7 +++++++ extensions-core/kinesis-indexing-service/pom.xml | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index c469ce96f63c..6c78e448fc66 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -151,6 +151,13 @@ ${apache.kafka.version} test + + org.apache.druid + druid-core + ${project.parent.version} + test-jar + test + org.apache.druid druid-processing diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index ee4fa353443d..e5adea55db69 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -150,6 +150,13 @@ junit test + + org.apache.druid + druid-core + ${project.parent.version} + test-jar + test + org.apache.druid druid-server From f308f1300fb4abb669c88c8d0743e05aebbb9e60 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 5 Nov 2019 15:21:41 -0800 Subject: [PATCH 07/29] fix firehose and inputSource for parallel indexing task --- .../druid/data/input/FirehoseFactory.java | 2 ++ .../FirehoseFactoryToInputSourceAdaptor.java | 15 ++++++---- ...rehoseFactoryToInputSourceAdaptorTest.java | 27 ++++++++++++++++- .../data/input/impl/NoopFirehoseFactory.java | 30 +++++++++++++++++-- .../indexing/common/task/CompactionTask.java | 2 -- .../druid/indexing/common/task/IndexTask.java | 6 +++- ...egmentGenerateParallelIndexTaskRunner.java | 15 ++++++++-- .../SinglePhaseParallelIndexTaskRunner.java | 15 ++++++++-- 8 files changed, 97 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/FirehoseFactory.java b/core/src/main/java/org/apache/druid/data/input/FirehoseFactory.java index 287f3d253c2b..986d1d94a5e4 100644 --- a/core/src/main/java/org/apache/druid/data/input/FirehoseFactory.java +++ b/core/src/main/java/org/apache/druid/data/input/FirehoseFactory.java @@ -19,6 +19,7 @@ package org.apache.druid.data.input; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; @@ -85,6 +86,7 @@ default Firehose connectForSampler(T parser, @Nullable File temporaryDirectory) return connect(parser, temporaryDirectory); } + @JsonIgnore default boolean isSplittable() { return false; diff --git a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java index e3a066beb726..b2df278cf966 100644 --- a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java +++ b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java @@ -34,15 +34,20 @@ public class FirehoseFactoryToInputSourceAdaptor implements SplittableInputSource { - private final FirehoseFactory firehoseFactory; + private final FiniteFirehoseFactory firehoseFactory; private final InputRowParser inputRowParser; - public FirehoseFactoryToInputSourceAdaptor(FirehoseFactory firehoseFactory, InputRowParser inputRowParser) + public FirehoseFactoryToInputSourceAdaptor(FiniteFirehoseFactory firehoseFactory, InputRowParser inputRowParser) { this.firehoseFactory = firehoseFactory; this.inputRowParser = Preconditions.checkNotNull(inputRowParser, "inputRowParser"); } + public FiniteFirehoseFactory getFirehoseFactory() + { + return firehoseFactory; + } + @Override public boolean isSplittable() { @@ -54,7 +59,7 @@ public Stream createSplits(InputFormat inputFormat, @Nullable SplitH throws IOException { if (firehoseFactory.isSplittable()) { - return ((FiniteFirehoseFactory) firehoseFactory).getSplits(splitHintSpec); + return firehoseFactory.getSplits(splitHintSpec); } else { throw new UnsupportedOperationException(); } @@ -64,7 +69,7 @@ public Stream createSplits(InputFormat inputFormat, @Nullable SplitH public int getNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException { if (firehoseFactory.isSplittable()) { - return ((FiniteFirehoseFactory) firehoseFactory).getNumSplits(splitHintSpec); + return firehoseFactory.getNumSplits(splitHintSpec); } else { throw new UnsupportedOperationException(); } @@ -75,7 +80,7 @@ public SplittableInputSource withSplit(InputSplit split) { if (firehoseFactory.isSplittable()) { return new FirehoseFactoryToInputSourceAdaptor( - ((FiniteFirehoseFactory) firehoseFactory).withSplit(split), + firehoseFactory.withSplit(split), inputRowParser ); } else { diff --git a/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java b/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java index f9e9ed1700c2..9dc14f66466a 100644 --- a/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java +++ b/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java @@ -39,6 +39,7 @@ import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.stream.Stream; public class FirehoseFactoryToInputSourceAdaptorTest { @@ -113,7 +114,7 @@ public InputFormat toInputFormat() } } - private static class TestFirehoseFactory implements FirehoseFactory + private static class TestFirehoseFactory implements FiniteFirehoseFactory { private final List lines; @@ -148,5 +149,29 @@ public void close() } }; } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec) throws IOException + { + return null; + } + + @Override + public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) throws IOException + { + return 0; + } + + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + return null; + } } } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java index 8b7653ffb91f..8b6fc94f7e67 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java @@ -19,13 +19,39 @@ package org.apache.druid.data.input.impl; -import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.FiniteFirehoseFactory; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; -public class NoopFirehoseFactory implements FirehoseFactory +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.stream.Stream; + +public class NoopFirehoseFactory implements FiniteFirehoseFactory { @Override public String toString() { return "NoopFirehoseFactory{}"; } + + @Override + public Stream getSplits( + @Nullable SplitHintSpec splitHintSpec + ) throws IOException + { + return null; + } + + @Override + public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) throws IOException + { + return 0; + } + + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + return null; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 826bff025b18..115ff8103b08 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -542,8 +542,6 @@ private static ParallelIndexIOConfig createIoConfig( segmentLoaderFactory, retryPolicyFactory ), - null, - null, false ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index d9fc9de6c130..1f2f354737ab 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -35,6 +35,7 @@ import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.data.input.InputRow; @@ -1145,7 +1146,10 @@ public InputFormat getInputFormat() public InputSource getNonNullInputSource(@Nullable InputRowParser inputRowParser) { if (inputSource == null) { - return new FirehoseFactoryToInputSourceAdaptor(firehoseFactory, inputRowParser); + return new FirehoseFactoryToInputSourceAdaptor( + (FiniteFirehoseFactory) firehoseFactory, + inputRowParser + ); } else { return inputSource; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java index 81925b9ed02d..3445eb192c26 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java @@ -21,6 +21,8 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexing.common.TaskToolbox; @@ -101,11 +103,20 @@ SplittableInputSource getBaseInputSource() SubTaskSpec newTaskSpec(InputSplit split) { + final FirehoseFactory firehoseFactory; + final SplittableInputSource inputSource; + if (baseInputSource instanceof FirehoseFactoryToInputSourceAdaptor) { + firehoseFactory = ((FirehoseFactoryToInputSourceAdaptor) baseInputSource).getFirehoseFactory().withSplit(split); + inputSource = null; + } else { + firehoseFactory = null; + inputSource = baseInputSource.withSplit(split); + } final ParallelIndexIngestionSpec subTaskIngestionSpec = new ParallelIndexIngestionSpec( ingestionSchema.getDataSchema(), new ParallelIndexIOConfig( - null, - baseInputSource.withSplit(split), + firehoseFactory, + inputSource, ingestionSchema.getIOConfig().getInputFormat(), ingestionSchema.getIOConfig().isAppendToExisting() ), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index 9ddb10722a3a..da8aed91068e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -21,6 +21,8 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexing.common.TaskToolbox; @@ -99,6 +101,15 @@ int getTotalNumSubTasks() throws IOException @VisibleForTesting SubTaskSpec newTaskSpec(InputSplit split) { + final FirehoseFactory firehoseFactory; + final SplittableInputSource inputSource; + if (baseInputSource instanceof FirehoseFactoryToInputSourceAdaptor) { + firehoseFactory = ((FirehoseFactoryToInputSourceAdaptor) baseInputSource).getFirehoseFactory().withSplit(split); + inputSource = null; + } else { + firehoseFactory = null; + inputSource = baseInputSource.withSplit(split); + } return new SinglePhaseSubTaskSpec( getTaskId() + "_" + getAndIncrementNextSpecId(), getGroupId(), @@ -106,8 +117,8 @@ SubTaskSpec newTaskSpec(InputSplit split) new ParallelIndexIngestionSpec( ingestionSchema.getDataSchema(), new ParallelIndexIOConfig( - null, - baseInputSource.withSplit(split), + firehoseFactory, + inputSource, ingestionSchema.getIOConfig().getInputFormat(), ingestionSchema.getIOConfig().isAppendToExisting() ), From b7c8b875f1ff9562960c0890137614d0a88c746e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 6 Nov 2019 11:22:23 -0800 Subject: [PATCH 08/29] fix tc --- .../src/main/java/org/apache/druid/data/input/InputSource.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/InputSource.java b/core/src/main/java/org/apache/druid/data/input/InputSource.java index 117a8aad9144..3e2bd606f4d1 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSource.java @@ -31,7 +31,6 @@ import javax.annotation.Nullable; import java.io.File; -import java.io.IOException; /** * InputSource abstracts the storage system where input data is stored. @@ -69,5 +68,5 @@ InputSourceReader reader( DimensionsSpec dimensionsSpec, InputFormat inputFormat, @Nullable File temporaryDirectory - ) throws IOException; + ); } From e942a2139a3f8e44187a1686274db136248e5dff Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 6 Nov 2019 14:39:34 -0800 Subject: [PATCH 09/29] fix tc: remove unused method --- .../druid/segment/transform/TransformingInputRowParser.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputRowParser.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputRowParser.java index 0482bb614a19..ea6da0b36a63 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputRowParser.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputRowParser.java @@ -39,11 +39,6 @@ public TransformingInputRowParser(final InputRowParser parser, final Transfor this.transformer = transformSpec.toTransformer(); } - public InputRowParser getParser() - { - return parser; - } - @Override public List parseBatch(final T row) { From 08d7872c33c227f2cbcdf3b693c4883f4d298ceb Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 7 Nov 2019 15:09:22 -0800 Subject: [PATCH 10/29] Formattable --- .../FirehoseFactoryToInputSourceAdaptor.java | 8 +++- .../apache/druid/data/input/Formattable.java | 39 ++++++++++++++++++ .../apache/druid/data/input/InputSource.java | 18 +-------- .../druid/data/input/SplitHintSpec.java | 5 ++- .../druid/data/input/Unformattable.java | 40 +++++++++++++++++++ .../data/input/impl/HttpInputSource.java | 9 ++++- .../data/input/impl/LocalInputSource.java | 9 ++++- .../data/input/impl/LocalInputSourceTest.java | 5 +-- .../data/input/impl/NoopInputSource.java | 23 +++++------ .../ParallelIndexSupervisorTaskKillTest.java | 13 +----- ...rallelIndexSupervisorTaskResourceTest.java | 13 +----- 11 files changed, 122 insertions(+), 60 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/data/input/Formattable.java create mode 100644 core/src/main/java/org/apache/druid/data/input/Unformattable.java diff --git a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java index b2df278cf966..0b68886c1aac 100644 --- a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java +++ b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java @@ -32,7 +32,7 @@ import java.io.IOException; import java.util.stream.Stream; -public class FirehoseFactoryToInputSourceAdaptor implements SplittableInputSource +public class FirehoseFactoryToInputSourceAdaptor implements SplittableInputSource, Formattable { private final FiniteFirehoseFactory firehoseFactory; private final InputRowParser inputRowParser; @@ -88,6 +88,12 @@ public SplittableInputSource withSplit(InputSplit split) } } + @Override + public boolean isFormattable() + { + return true; + } + @Override public InputSourceReader reader( TimestampSpec timestampSpec, diff --git a/core/src/main/java/org/apache/druid/data/input/Formattable.java b/core/src/main/java/org/apache/druid/data/input/Formattable.java new file mode 100644 index 000000000000..0ac69700b6b8 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/Formattable.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputFormat; +import org.apache.druid.data.input.impl.TimestampSpec; + +import javax.annotation.Nullable; +import java.io.File; + +public interface Formattable +{ + boolean isFormattable(); + + InputSourceReader reader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ); +} diff --git a/core/src/main/java/org/apache/druid/data/input/InputSource.java b/core/src/main/java/org/apache/druid/data/input/InputSource.java index 3e2bd606f4d1..b54dacd442e6 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSource.java @@ -22,16 +22,10 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonSubTypes.Type; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.HttpInputSource; -import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.data.input.impl.LocalInputSource; -import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.annotations.ExtensionPoint; -import javax.annotation.Nullable; -import java.io.File; - /** * InputSource abstracts the storage system where input data is stored. * It creates an {@link InputSourceReader} to read data from the given input source. @@ -58,15 +52,7 @@ public interface InputSource /** * Returns true if this inputSource can be processed in parallel using ParallelIndexSupervisorTask. */ - default boolean isSplittable() - { - return false; - } + boolean isSplittable(); - InputSourceReader reader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - InputFormat inputFormat, - @Nullable File temporaryDirectory - ); + boolean isFormattable(); } diff --git a/core/src/main/java/org/apache/druid/data/input/SplitHintSpec.java b/core/src/main/java/org/apache/druid/data/input/SplitHintSpec.java index 5e22160c4526..2d63f0ac08b8 100644 --- a/core/src/main/java/org/apache/druid/data/input/SplitHintSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/SplitHintSpec.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonSubTypes.Type; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.data.input.impl.SplittableInputSource; /** * In native parallel indexing, the supervisor task partitions input data into splits and assigns each of them @@ -31,8 +32,8 @@ * * @see FiniteFirehoseFactory#getSplits(SplitHintSpec) * @see FiniteFirehoseFactory#getNumSplits(SplitHintSpec) - * @see org.apache.druid.data.input.impl.SplittableInputSource#createSplits - * @see org.apache.druid.data.input.impl.SplittableInputSource#getNumSplits + * @see SplittableInputSource#createSplits + * @see SplittableInputSource#getNumSplits */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { diff --git a/core/src/main/java/org/apache/druid/data/input/Unformattable.java b/core/src/main/java/org/apache/druid/data/input/Unformattable.java new file mode 100644 index 000000000000..34f16b7dc5a4 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/Unformattable.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; + +import javax.annotation.Nullable; +import java.io.File; + +public interface Unformattable +{ + default boolean isFormattable() + { + return false; + } + + InputSourceReader reader( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + @Nullable File temporaryDirectory + ); +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java index ac0b41f5afbf..7f0c088fe50b 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.data.input.Formattable; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SplitHintSpec; @@ -35,7 +36,7 @@ import java.util.Objects; import java.util.stream.Stream; -public class HttpInputSource implements SplittableInputSource +public class HttpInputSource implements SplittableInputSource, Formattable { private final List uris; @Nullable @@ -139,4 +140,10 @@ public int hashCode() { return Objects.hash(uris, httpAuthenticationUsername, httpAuthenticationPasswordProvider); } + + @Override + public boolean isFormattable() + { + return true; + } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java index 32126214480d..fa5223fa151c 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java @@ -25,6 +25,7 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.commons.io.filefilter.WildcardFileFilter; +import org.apache.druid.data.input.Formattable; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SplitHintSpec; @@ -39,7 +40,7 @@ import java.util.stream.Stream; import java.util.stream.StreamSupport; -public class LocalInputSource implements SplittableInputSource +public class LocalInputSource implements SplittableInputSource, Formattable { private final File baseDir; private final String filter; @@ -101,6 +102,12 @@ public SplittableInputSource withSplit(InputSplit split) return new LocalInputSource(file.getParentFile(), file.getName()); } + @Override + public boolean isFormattable() + { + return true; + } + @Override public InputSourceReader reader( TimestampSpec timestampSpec, diff --git a/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java b/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java index 3852d10f08e4..c4f0ab54ca05 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java @@ -33,10 +33,7 @@ public class LocalInputSourceTest public void testSerde() throws IOException { final ObjectMapper mapper = new ObjectMapper(); - final LocalInputSource source = new LocalInputSource( - new File("myFile").getAbsoluteFile(), - "myFilter" - ); + final LocalInputSource source = new LocalInputSource(new File("myFile").getAbsoluteFile(), "myFilter"); final byte[] json = mapper.writeValueAsBytes(source); final LocalInputSource fromJson = (LocalInputSource) mapper.readValue(json, InputSource.class); Assert.assertEquals(source, fromJson); diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java index 1f6bd82dc05c..9c97b48a2899 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java @@ -20,27 +20,24 @@ package org.apache.druid.data.input.impl; import org.apache.druid.data.input.InputSource; -import org.apache.druid.data.input.InputSourceReader; - -import javax.annotation.Nullable; -import java.io.File; public class NoopInputSource implements InputSource { @Override - public InputSourceReader reader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - InputFormat inputFormat, - @Nullable File temporaryDirectory - ) + public String toString() { - return null; + return "NoopInputSource{}"; } @Override - public String toString() + public boolean isSplittable() { - return "NoopInputSource{}"; + return false; + } + + @Override + public boolean isFormattable() + { + return false; } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index d845b73aa5d9..ca83397dcae4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -21,14 +21,11 @@ import com.google.common.collect.Iterables; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.SplittableInputSource; -import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; @@ -55,7 +52,6 @@ import org.junit.rules.ExpectedException; import javax.annotation.Nullable; -import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -279,14 +275,9 @@ public SplittableInputSource withSplit(InputSplit split) } @Override - public InputSourceReader reader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - InputFormat inputFormat, - @Nullable File temporaryDirectory - ) + public boolean isFormattable() { - throw new UnsupportedOperationException(); + return false; } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index dc2c871e9f8c..63d371f82a22 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -22,15 +22,12 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.SplittableInputSource; -import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; @@ -66,7 +63,6 @@ import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; -import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -501,14 +497,9 @@ public SplittableInputSource withSplit(InputSplit split) } @Override - public InputSourceReader reader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - InputFormat inputFormat, - @Nullable File temporaryDirectory - ) + public boolean isFormattable() { - throw new UnsupportedOperationException(); + return false; } } From c70af75130a1366b3d27cca54f891402dd06fa82 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 8 Nov 2019 16:39:34 -0800 Subject: [PATCH 11/29] add needsFormat(); renamed to ObjectSource; pass metricsName for reader --- .../druid/data/input/AbstractInputSource.java | 66 +++++++++++++++++++ .../FirehoseFactoryToInputSourceAdaptor.java | 15 ++--- .../apache/druid/data/input/Formattable.java | 39 ----------- ...Unformattable.java => InputRowSchema.java} | 34 +++++++--- .../apache/druid/data/input/InputSource.java | 22 ++++++- .../druid/data/input/InputSourceReader.java | 5 +- .../{SplitReader.java => ObjectReader.java} | 10 +-- .../{SplitSource.java => ObjectSource.java} | 41 +++++++----- .../apache/druid/data/input/TextReader.java | 29 +++----- .../druid/data/input/impl/ByteSource.java | 18 +++-- .../druid/data/input/impl/CsvInputFormat.java | 7 +- .../druid/data/input/impl/CsvReader.java | 14 ++-- .../druid/data/input/impl/FileSource.java | 23 ++++--- .../data/input/impl/HttpInputSource.java | 21 +++--- .../druid/data/input/impl/HttpSource.java | 20 +++--- .../druid/data/input/impl/InputFormat.java | 7 +- .../data/input/impl/JsonInputFormat.java | 7 +- .../druid/data/input/impl/JsonReader.java | 10 +-- .../data/input/impl/LocalInputSource.java | 19 +++--- ...Reader.java => ObjectIteratingReader.java} | 30 ++++----- ...rehoseFactoryToInputSourceAdaptorTest.java | 12 ++-- .../data/input/impl/CsvInputFormatTest.java | 15 +++-- .../druid/data/input/impl/CsvReaderTest.java | 15 +++-- .../druid/data/input/impl/JsonReaderTest.java | 21 ++++-- .../data/input/impl/NoopInputFormat.java | 5 +- .../data/input/impl/NoopInputSource.java | 17 ++++- ...st.java => ObjectIteratingReaderTest.java} | 18 +++-- .../druid/indexing/common/task/IndexTask.java | 14 +++- .../common/task/InputSourceProcessor.java | 15 ++++- .../batch/parallel/SinglePhaseSubTask.java | 13 +++- .../IngestSegmentFirehoseFactory.java | 1 - .../ParallelIndexSupervisorTaskKillTest.java | 5 +- ...rallelIndexSupervisorTaskResourceTest.java | 5 +- .../indexing/overlord/TaskLifecycleTest.java | 52 +++++++++------ 34 files changed, 390 insertions(+), 255 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java delete mode 100644 core/src/main/java/org/apache/druid/data/input/Formattable.java rename core/src/main/java/org/apache/druid/data/input/{Unformattable.java => InputRowSchema.java} (60%) rename core/src/main/java/org/apache/druid/data/input/{SplitReader.java => ObjectReader.java} (72%) rename core/src/main/java/org/apache/druid/data/input/{SplitSource.java => ObjectSource.java} (74%) rename core/src/main/java/org/apache/druid/data/input/impl/{SplitIteratingReader.java => ObjectIteratingReader.java} (78%) rename core/src/test/java/org/apache/druid/data/input/impl/{SplitIteratingReaderTest.java => ObjectIteratingReaderTest.java} (86%) diff --git a/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java b/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java new file mode 100644 index 000000000000..1ac69cf86bbc --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.impl.InputFormat; + +import javax.annotation.Nullable; +import java.io.File; + +/** + * Abstract class for {@link InputSource}. This class provides a default implementation of {@link #reader} with + * a sanity check. Child classes should implement one of {@link #formattableReader} or {@link #unformattableReader} + * depending on {@link #needsFormat()}. + */ +public abstract class AbstractInputSource implements InputSource +{ + @Override + public InputSourceReader reader( + InputRowSchema inputRowSchema, + @Nullable InputFormat inputFormat, + @Nullable File temporaryDirectory + ) + { + if (needsFormat()) { + return formattableReader( + inputRowSchema, + Preconditions.checkNotNull(inputFormat, "inputFormat"), + temporaryDirectory + ); + } else { + return unformattableReader(inputRowSchema, temporaryDirectory); + } + } + + protected InputSourceReader formattableReader( + InputRowSchema inputRowSchema, + @Nullable InputFormat inputFormat, + @Nullable File temporaryDirectory + ) + { + throw new UnsupportedOperationException("Implement this method properly if needsFormat() = true"); + } + + protected InputSourceReader unformattableReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) + { + throw new UnsupportedOperationException("Implement this method properly if needsFormat() = false"); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java index 0b68886c1aac..55806af86b0e 100644 --- a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java +++ b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java @@ -20,19 +20,17 @@ package org.apache.druid.data.input; import com.google.common.base.Preconditions; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FirehoseToInputSourceReaderAdaptor; import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.SplittableInputSource; -import org.apache.druid.data.input.impl.TimestampSpec; import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.stream.Stream; -public class FirehoseFactoryToInputSourceAdaptor implements SplittableInputSource, Formattable +public class FirehoseFactoryToInputSourceAdaptor extends AbstractInputSource implements SplittableInputSource { private final FiniteFirehoseFactory firehoseFactory; private final InputRowParser inputRowParser; @@ -89,18 +87,13 @@ public SplittableInputSource withSplit(InputSplit split) } @Override - public boolean isFormattable() + public boolean needsFormat() { - return true; + return false; } @Override - public InputSourceReader reader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - @Nullable InputFormat inputFormat, // inputFormat will be ignored - @Nullable File temporaryDirectory - ) + protected InputSourceReader unformattableReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) { return new FirehoseToInputSourceReaderAdaptor(firehoseFactory, inputRowParser, temporaryDirectory); } diff --git a/core/src/main/java/org/apache/druid/data/input/Formattable.java b/core/src/main/java/org/apache/druid/data/input/Formattable.java deleted file mode 100644 index 0ac69700b6b8..000000000000 --- a/core/src/main/java/org/apache/druid/data/input/Formattable.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.data.input; - -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputFormat; -import org.apache.druid.data.input.impl.TimestampSpec; - -import javax.annotation.Nullable; -import java.io.File; - -public interface Formattable -{ - boolean isFormattable(); - - InputSourceReader reader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - InputFormat inputFormat, - @Nullable File temporaryDirectory - ); -} diff --git a/core/src/main/java/org/apache/druid/data/input/Unformattable.java b/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java similarity index 60% rename from core/src/main/java/org/apache/druid/data/input/Unformattable.java rename to core/src/main/java/org/apache/druid/data/input/InputRowSchema.java index 34f16b7dc5a4..ca01fca3b5d6 100644 --- a/core/src/main/java/org/apache/druid/data/input/Unformattable.java +++ b/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java @@ -22,19 +22,33 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; -import javax.annotation.Nullable; -import java.io.File; +import java.util.List; -public interface Unformattable +public class InputRowSchema { - default boolean isFormattable() + private final TimestampSpec timestampSpec; + private final DimensionsSpec dimensionsSpec; + private final List metricsNames; + + public InputRowSchema(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec, List metricsNames) { - return false; + this.timestampSpec = timestampSpec; + this.dimensionsSpec = dimensionsSpec; + this.metricsNames = metricsNames; } - InputSourceReader reader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - @Nullable File temporaryDirectory - ); + public TimestampSpec getTimestampSpec() + { + return timestampSpec; + } + + public DimensionsSpec getDimensionsSpec() + { + return dimensionsSpec; + } + + public List getMetricsNames() + { + return metricsNames; + } } diff --git a/core/src/main/java/org/apache/druid/data/input/InputSource.java b/core/src/main/java/org/apache/druid/data/input/InputSource.java index b54dacd442e6..346b7064a920 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSource.java @@ -23,9 +23,13 @@ import com.fasterxml.jackson.annotation.JsonSubTypes.Type; import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.data.input.impl.HttpInputSource; +import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.guice.annotations.ExtensionPoint; +import javax.annotation.Nullable; +import java.io.File; + /** * InputSource abstracts the storage system where input data is stored. * It creates an {@link InputSourceReader} to read data from the given input source. @@ -54,5 +58,21 @@ public interface InputSource */ boolean isSplittable(); - boolean isFormattable(); + /** + * Returns true if this inputSource supports different {@link InputFormat}s. + */ + boolean needsFormat(); + + /** + * Create an {@link InputSourceReader}. + * + * @param inputRowSchema for {@link InputRow} + * @param inputFormat to parse data. It can be null if {@link #needsFormat()} = true + * @param temporaryDirectory to store temp data. It will be cleaned up automatically once the task is finished. + */ + InputSourceReader reader( + InputRowSchema inputRowSchema, + @Nullable InputFormat inputFormat, + @Nullable File temporaryDirectory + ); } diff --git a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java index f29ccf53b743..809b8b1ecb5a 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java @@ -19,6 +19,7 @@ package org.apache.druid.data.input; +import org.apache.druid.data.input.impl.ObjectIteratingReader; import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -28,9 +29,9 @@ * InputSourceReader reads data from {@link InputSource} and returns a {@link CloseableIterator} of * {@link InputRow}. See {@link InputSource} for an example usage. * - * Implementations of this class can use {@link SplitSource} and {@link SplitReader}. + * Implementations of this class can use {@link ObjectSource} and {@link ObjectReader}. * - * See {@link org.apache.druid.data.input.impl.SplitIteratingReader} as an example. + * See {@link ObjectIteratingReader} as an example. */ @ExtensionPoint public interface InputSourceReader diff --git a/core/src/main/java/org/apache/druid/data/input/SplitReader.java b/core/src/main/java/org/apache/druid/data/input/ObjectReader.java similarity index 72% rename from core/src/main/java/org/apache/druid/data/input/SplitReader.java rename to core/src/main/java/org/apache/druid/data/input/ObjectReader.java index cf0f3128225b..6512ecf9f996 100644 --- a/core/src/main/java/org/apache/druid/data/input/SplitReader.java +++ b/core/src/main/java/org/apache/druid/data/input/ObjectReader.java @@ -26,15 +26,15 @@ import java.io.IOException; /** - * SplitReader knows how to parse data into {@link InputRow}. - * This class is stateful and a new SplitReader should be created per {@link InputSplit}. + * ObjectReader knows how to parse data into {@link InputRow}. + * This class is stateful and a new ObjectReader should be created per {@link ObjectSource}. * * @see TextReader for text format readers */ @ExtensionPoint -public interface SplitReader +public interface ObjectReader { - CloseableIterator read(SplitSource source, File temporaryDirectory) throws IOException; + CloseableIterator read(ObjectSource source, File temporaryDirectory) throws IOException; - CloseableIterator sample(SplitSource source, File temporaryDirectory) throws IOException; + CloseableIterator sample(ObjectSource source, File temporaryDirectory) throws IOException; } diff --git a/core/src/main/java/org/apache/druid/data/input/SplitSource.java b/core/src/main/java/org/apache/druid/data/input/ObjectSource.java similarity index 74% rename from core/src/main/java/org/apache/druid/data/input/SplitSource.java rename to core/src/main/java/org/apache/druid/data/input/ObjectSource.java index 0ae0ccce9744..1f522922b87c 100644 --- a/core/src/main/java/org/apache/druid/data/input/SplitSource.java +++ b/core/src/main/java/org/apache/druid/data/input/ObjectSource.java @@ -31,12 +31,12 @@ import java.io.InputStream; /** - * SplitSource abstracts an {@link InputSplit} and knows how to read bytes from the given split. + * ObjectSource abstracts an object and knows how to read bytes from the given object. */ @ExtensionPoint -public interface SplitSource + public interface ObjectSource { - Logger LOG = new Logger(SplitSource.class); + Logger LOG = new Logger(ObjectSource.class); int DEFAULT_FETCH_BUFFER_SIZE = 4 * 1024; // 4 KB int DEFAULT_MAX_FETCH_RETRY = 2; // 3 tries including the initial try @@ -50,18 +50,18 @@ interface CleanableFile extends Closeable File file(); } - InputSplit getSplit(); + T getObject(); /** - * Opens an {@link InputStream} on the split directly. - * This is the basic way to read the given split. + * Opens an {@link InputStream} on the object directly. + * This is the basic way to read the given object. * * @see #fetch as an alternative way to read data. */ InputStream open() throws IOException; /** - * Fetches the split into the local storage. + * Fetches the object into the local storage. * This method might be preferred instead of {@link #open()}, for example * * - {@link org.apache.druid.data.input.impl.InputFormat} requires expensive random access on remote storage. @@ -69,22 +69,27 @@ interface CleanableFile extends Closeable * * @param temporaryDirectory to store temp data. This directory will be removed automatically once * the task finishes. - * @param fetchBuffer is used to fetch remote split into local storage. + * @param fetchBuffer is used to fetch remote object into local storage. * * @see FileUtils#copyLarge */ default CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) throws IOException { - final File tempFile = File.createTempFile("druid-split", ".tmp", temporaryDirectory); - LOG.debug("Fetching split into file[%s]", tempFile.getAbsolutePath()); - FileUtils.copyLarge( - open(), - tempFile, - fetchBuffer, - getRetryCondition(), - DEFAULT_MAX_FETCH_RETRY, - StringUtils.format("Failed to fetch into [%s]", tempFile.getAbsolutePath()) - ); + final File tempFile = File.createTempFile("druid-object-source", ".tmp", temporaryDirectory); + LOG.debug("Fetching object into file[%s]", tempFile.getAbsolutePath()); + try (InputStream is = open()) { + FileUtils.copyLarge( + is, + tempFile, + fetchBuffer, + getRetryCondition(), + DEFAULT_MAX_FETCH_RETRY, + StringUtils.format("Failed to fetch into [%s]", tempFile.getAbsolutePath()) + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } return new CleanableFile() { diff --git a/core/src/main/java/org/apache/druid/data/input/TextReader.java b/core/src/main/java/org/apache/druid/data/input/TextReader.java index 23a5cdbeb063..ff0c4124be1a 100644 --- a/core/src/main/java/org/apache/druid/data/input/TextReader.java +++ b/core/src/main/java/org/apache/druid/data/input/TextReader.java @@ -20,8 +20,6 @@ package org.apache.druid.data.input; import org.apache.commons.io.LineIterator; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; @@ -31,31 +29,24 @@ import java.io.InputStreamReader; /** - * Abstract {@link SplitReader} for text format readers such as CSV or JSON. + * Abstract {@link ObjectReader} for text format readers such as CSV or JSON. */ -public abstract class TextReader implements SplitReader +public abstract class TextReader implements ObjectReader { - private final TimestampSpec timestampSpec; - private final DimensionsSpec dimensionsSpec; + private final InputRowSchema inputRowSchema; - public TextReader(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec) + public TextReader(InputRowSchema inputRowSchema) { - this.timestampSpec = timestampSpec; - this.dimensionsSpec = dimensionsSpec; + this.inputRowSchema = inputRowSchema; } - public TimestampSpec getTimestampSpec() + public InputRowSchema getInputRowSchema() { - return timestampSpec; - } - - public DimensionsSpec getDimensionsSpec() - { - return dimensionsSpec; + return inputRowSchema; } @Override - public CloseableIterator read(SplitSource source, File temporaryDirectory) throws IOException + public CloseableIterator read(ObjectSource source, File temporaryDirectory) throws IOException { return lineIterator(source).map(line -> { try { @@ -68,7 +59,7 @@ public CloseableIterator read(SplitSource source, File temporaryDirect } @Override - public CloseableIterator sample(SplitSource source, File temporaryDirectory) + public CloseableIterator sample(ObjectSource source, File temporaryDirectory) throws IOException { return lineIterator(source).map(line -> { @@ -84,7 +75,7 @@ public CloseableIterator sample(SplitSource source, File tempor }); } - private CloseableIterator lineIterator(SplitSource source) throws IOException + private CloseableIterator lineIterator(ObjectSource source) throws IOException { final LineIterator delegate = new LineIterator( new InputStreamReader(source.open(), StringUtils.UTF8_STRING) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java b/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java index f828b566318f..787969be2f48 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java @@ -21,21 +21,19 @@ import com.google.common.base.Predicate; import com.google.common.base.Predicates; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.SplitSource; +import org.apache.druid.data.input.ObjectSource; import org.apache.druid.io.ByteBufferInputStream; -import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; -public class ByteSource implements SplitSource +public class ByteSource implements ObjectSource { - private final InputSplit split; + private final ByteBuffer buffer; public ByteSource(ByteBuffer buffer) { - this.split = new InputSplit<>(buffer.duplicate()); + this.buffer = buffer.duplicate(); } public ByteSource(byte[] bytes) @@ -44,15 +42,15 @@ public ByteSource(byte[] bytes) } @Override - public InputSplit getSplit() + public ByteBuffer getObject() { - return split; + return buffer; } @Override - public InputStream open() throws IOException + public InputStream open() { - return new ByteBufferInputStream(split.get()); + return new ByteBufferInputStream(buffer); } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java index f4b2ace7b3c1..4760edfb836d 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java @@ -24,7 +24,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import org.apache.druid.data.input.SplitReader; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.ObjectReader; import org.apache.druid.indexer.Checks; import org.apache.druid.indexer.Property; @@ -115,9 +116,9 @@ public boolean isSplittable() } @Override - public SplitReader createReader(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec) + public ObjectReader createReader(InputRowSchema inputRowSchema) { - return new CsvReader(timestampSpec, dimensionsSpec, listDelimiter, columns, findColumnsFromHeader, skipHeaderRows); + return new CsvReader(inputRowSchema, listDelimiter, columns, findColumnsFromHeader, skipHeaderRows); } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java index 5f868c63cfb6..a45e4b0372cc 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java @@ -26,6 +26,7 @@ import com.google.common.collect.Iterables; import com.opencsv.CSVParser; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.TextReader; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.collect.Utils; @@ -50,15 +51,14 @@ public class CsvReader extends TextReader private List columns; CsvReader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, + InputRowSchema inputRowSchema, String listDelimiter, @Nullable List columns, boolean hasHeaderRow, int skipHeaderRows ) { - super(timestampSpec, dimensionsSpec); + super(inputRowSchema); this.hasHeaderRow = hasHeaderRow; this.skipHeaderRows = skipHeaderRows; final String finalListDelimeter = listDelimiter == null ? Parsers.DEFAULT_LIST_DELIMITER : listDelimiter; @@ -69,7 +69,7 @@ public class CsvReader extends TextReader for (String column : this.columns) { Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column); } - verify(this.columns, dimensionsSpec.getDimensionNames()); + verify(this.columns, inputRowSchema.getDimensionsSpec().getDimensionNames()); } else { Preconditions.checkArgument( hasHeaderRow, @@ -87,7 +87,11 @@ public InputRow readLine(String line) throws IOException, ParseException Preconditions.checkNotNull(columns, "columns"), Iterables.transform(Arrays.asList(parsed), multiValueFunction) ); - return MapInputRowParser.parse(getTimestampSpec(), getDimensionsSpec(), zipped); + return MapInputRowParser.parse( + getInputRowSchema().getTimestampSpec(), + getInputRowSchema().getDimensionsSpec(), + zipped + ); } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java b/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java index d77f0eabd6f9..59068ca63479 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java @@ -21,8 +21,7 @@ import com.google.common.base.Predicate; import com.google.common.base.Predicates; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.SplitSource; +import org.apache.druid.data.input.ObjectSource; import org.apache.druid.utils.CompressionUtils; import java.io.File; @@ -33,16 +32,16 @@ import java.nio.channels.Channels; import java.nio.channels.FileChannel; -public class FileSource implements SplitSource +public class FileSource implements ObjectSource { - private final InputSplit split; + private final File file; private final FileChannel channel; - FileSource(InputSplit split) throws FileNotFoundException + FileSource(File file) throws FileNotFoundException { - this.split = split; - final RandomAccessFile file = new RandomAccessFile(split.get(), "r"); - this.channel = file.getChannel(); + this.file = file; + final RandomAccessFile randomAccessFile = new RandomAccessFile(file, "r"); + this.channel = randomAccessFile.getChannel(); } @Override @@ -53,7 +52,7 @@ public CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) @Override public File file() { - return split.get(); + return file; } @Override @@ -65,15 +64,15 @@ public void close() } @Override - public InputSplit getSplit() + public File getObject() { - return split; + return file; } @Override public InputStream open() throws IOException { - return CompressionUtils.decompress(Channels.newInputStream(channel), split.get().getName()); + return CompressionUtils.decompress(Channels.newInputStream(channel), file.getName()); } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java index 7f0c088fe50b..3109e7cf2e0b 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java @@ -22,7 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.data.input.Formattable; +import org.apache.druid.data.input.AbstractInputSource; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SplitHintSpec; @@ -36,7 +37,7 @@ import java.util.Objects; import java.util.stream.Stream; -public class HttpInputSource implements SplittableInputSource, Formattable +public class HttpInputSource extends AbstractInputSource implements SplittableInputSource { private final List uris; @Nullable @@ -100,19 +101,17 @@ public SplittableInputSource withSplit(InputSplit split) } @Override - public InputSourceReader reader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - InputFormat inputFormat, + protected InputSourceReader formattableReader( + InputRowSchema inputRowSchema, + @Nullable InputFormat inputFormat, @Nullable File temporaryDirectory ) { - return new SplitIteratingReader<>( - timestampSpec, - dimensionsSpec, + return new ObjectIteratingReader<>( + inputRowSchema, inputFormat, createSplits(inputFormat, null).map(split -> new HttpSource( - split, + split.get(), httpAuthenticationUsername, httpAuthenticationPasswordProvider )), @@ -142,7 +141,7 @@ public int hashCode() } @Override - public boolean isFormattable() + public boolean needsFormat() { return true; } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java index 899f4069028c..00230a42eef8 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java @@ -21,8 +21,7 @@ import com.google.common.base.Predicate; import com.google.common.base.Strings; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.SplitSource; +import org.apache.druid.data.input.ObjectSource; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.metadata.PasswordProvider; import org.apache.druid.utils.CompressionUtils; @@ -34,16 +33,16 @@ import java.net.URLConnection; import java.util.Base64; -public class HttpSource implements SplitSource +public class HttpSource implements ObjectSource { - private final InputSplit split; + private final URI split; @Nullable private final String httpAuthenticationUsername; @Nullable private final PasswordProvider httpAuthenticationPasswordProvider; HttpSource( - InputSplit split, + URI split, @Nullable String httpAuthenticationUsername, @Nullable PasswordProvider httpAuthenticationPasswordProvider ) @@ -54,7 +53,7 @@ public class HttpSource implements SplitSource } @Override - public InputSplit getSplit() + public URI getObject() { return split; } @@ -62,11 +61,10 @@ public InputSplit getSplit() @Override public InputStream open() throws IOException { - return CompressionUtils.decompress(openURLConnection( - split.get(), - httpAuthenticationUsername, - httpAuthenticationPasswordProvider - ).getInputStream(), split.get().toString()); + return CompressionUtils.decompress( + openURLConnection(split, httpAuthenticationUsername, httpAuthenticationPasswordProvider).getInputStream(), + split.toString() + ); } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/InputFormat.java index 496f159b51f4..e4170f89bc8d 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputFormat.java @@ -23,12 +23,13 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonSubTypes.Type; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.data.input.SplitReader; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.ObjectReader; import org.apache.druid.guice.annotations.ExtensionPoint; /** * InputFormat abstracts the file format of input data. - * It creates a {@link SplitReader} to read data and parse it into {@link org.apache.druid.data.input.InputRow}. + * It creates a {@link ObjectReader} to read data and parse it into {@link org.apache.druid.data.input.InputRow}. * The created SplitReader is used by {@link org.apache.druid.data.input.InputSourceReader}. * * @see NestedInputFormat for nested input formats such as JSON. @@ -50,5 +51,5 @@ public interface InputFormat @JsonIgnore boolean isSplittable(); - SplitReader createReader(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec); + ObjectReader createReader(InputRowSchema inputRowSchema); } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java index 661aee7f3c52..bf43e16fa502 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java @@ -23,7 +23,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonParser.Feature; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.SplitReader; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.ObjectReader; import org.apache.druid.java.util.common.parsers.JSONPathSpec; import javax.annotation.Nullable; @@ -65,9 +66,9 @@ public boolean isSplittable() } @Override - public SplitReader createReader(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec) + public ObjectReader createReader(InputRowSchema inputRowSchema) { - return new JsonReader(timestampSpec, dimensionsSpec, getFlattenSpec(), objectMapper); + return new JsonReader(inputRowSchema, getFlattenSpec(), objectMapper); } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java index 34020516a0e3..6940ee8bb918 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.TextReader; import org.apache.druid.java.util.common.parsers.JSONFlattenerMaker; import org.apache.druid.java.util.common.parsers.JSONPathSpec; @@ -38,13 +39,12 @@ public class JsonReader extends TextReader private final ObjectMapper mapper; public JsonReader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, + InputRowSchema inputRowSchema, JSONPathSpec flattenSpec, ObjectMapper mapper ) { - super(timestampSpec, dimensionsSpec); + super(inputRowSchema); this.flattener = ObjectFlatteners.create(flattenSpec, new JSONFlattenerMaker()); this.mapper = mapper; } @@ -55,8 +55,8 @@ public InputRow readLine(String line) throws IOException, ParseException final JsonNode document = mapper.readValue(line, JsonNode.class); final Map flattened = flattener.flatten(document); return MapInputRowParser.parse( - getTimestampSpec(), - getDimensionsSpec(), + getInputRowSchema().getTimestampSpec(), + getInputRowSchema().getDimensionsSpec(), flattened ); } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java index fa5223fa151c..119d7dcd1257 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java @@ -25,7 +25,8 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.commons.io.filefilter.WildcardFileFilter; -import org.apache.druid.data.input.Formattable; +import org.apache.druid.data.input.AbstractInputSource; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SplitHintSpec; @@ -40,7 +41,7 @@ import java.util.stream.Stream; import java.util.stream.StreamSupport; -public class LocalInputSource implements SplittableInputSource, Formattable +public class LocalInputSource extends AbstractInputSource implements SplittableInputSource { private final File baseDir; private final String filter; @@ -103,29 +104,27 @@ public SplittableInputSource withSplit(InputSplit split) } @Override - public boolean isFormattable() + public boolean needsFormat() { return true; } @Override - public InputSourceReader reader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, + protected InputSourceReader formattableReader( + InputRowSchema inputRowSchema, InputFormat inputFormat, @Nullable File temporaryDirectory ) { - return new SplitIteratingReader<>( - timestampSpec, - dimensionsSpec, + return new ObjectIteratingReader<>( + inputRowSchema, inputFormat, // reader() is supposed to be called in each task that creates segments. // The task should already have only one split in parallel indexing, // while there's no need to make splits using splitHintSpec in sequential indexing. createSplits(inputFormat, null).map(split -> { try { - return new FileSource(split); + return new FileSource(split.get()); } catch (FileNotFoundException e) { throw new RuntimeException(e); diff --git a/core/src/main/java/org/apache/druid/data/input/impl/SplitIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java similarity index 78% rename from core/src/main/java/org/apache/druid/data/input/impl/SplitIteratingReader.java rename to core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java index d9732df2daac..d0193270e11e 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/SplitIteratingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java @@ -21,9 +21,10 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; -import org.apache.druid.data.input.SplitReader; -import org.apache.druid.data.input.SplitSource; +import org.apache.druid.data.input.ObjectSource; +import org.apache.druid.data.input.ObjectReader; import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.File; @@ -34,26 +35,23 @@ import java.util.stream.Stream; /** - * InputSourceReader iterating multiple {@link SplitSource}s. + * InputSourceReader iterating multiple {@link ObjectSource}s. */ -public class SplitIteratingReader implements InputSourceReader +public class ObjectIteratingReader implements InputSourceReader { - private final TimestampSpec timestampSpec; - private final DimensionsSpec dimensionsSpec; + private final InputRowSchema inputRowSchema; private final InputFormat inputFormat; - private final Iterator> sourceIterator; + private final Iterator> sourceIterator; private final File temporaryDirectory; - public SplitIteratingReader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, + ObjectIteratingReader( + InputRowSchema inputRowSchema, InputFormat inputFormat, - Stream> sourceStream, + Stream> sourceStream, File temporaryDirectory ) { - this.timestampSpec = timestampSpec; - this.dimensionsSpec = dimensionsSpec; + this.inputRowSchema = inputRowSchema; this.inputFormat = inputFormat; this.sourceIterator = sourceStream.iterator(); this.temporaryDirectory = temporaryDirectory; @@ -85,7 +83,7 @@ public CloseableIterator sample() }); } - private CloseableIterator createIterator(Function> rowPopulator) + private CloseableIterator createIterator(Function> rowPopulator) { return new CloseableIterator() { @@ -120,8 +118,8 @@ private void checkRowIterator() } if (sourceIterator.hasNext()) { // SplitSampler is stateful and so a new one should be created per split. - final SplitReader splitReader = inputFormat.createReader(timestampSpec, dimensionsSpec); - rowIterator = rowPopulator.apply(splitReader); + final ObjectReader objectReader = inputFormat.createReader(inputRowSchema); + rowIterator = rowPopulator.apply(objectReader); } } } diff --git a/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java b/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java index 9dc14f66466a..5f88b56848c3 100644 --- a/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java +++ b/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java @@ -37,6 +37,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.stream.Stream; @@ -67,8 +68,11 @@ public void testUnimplementedInputFormat() throws IOException inputRowParser ); final InputSourceReader reader = inputSourceAdaptor.reader( - inputRowParser.getParseSpec().getTimestampSpec(), - inputRowParser.getParseSpec().getDimensionsSpec(), + new InputRowSchema( + inputRowParser.getParseSpec().getTimestampSpec(), + inputRowParser.getParseSpec().getDimensionsSpec(), + Collections.emptyList() + ), null, null ); @@ -157,13 +161,13 @@ public boolean isSplittable() } @Override - public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec) throws IOException + public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec) { return null; } @Override - public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) throws IOException + public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) { return 0; } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java index abae7fa21467..cca122c800ac 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java @@ -20,6 +20,7 @@ package org.apache.druid.data.input.impl; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputRowSchema; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -51,8 +52,11 @@ public void testColumnMissing() expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("column[b] not in columns"); format.createReader( - new TimestampSpec("timestamp", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b"))) + new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b"))), + Collections.emptyList() + ) ); } @@ -63,8 +67,11 @@ public void testComma() expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("column[a,] not in columns"); format.createReader( - new TimestampSpec("timestamp", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("a,", "b"))) + new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("a,", "b"))), + Collections.emptyList() + ) ); } } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java index a28ff16e1e99..50b07740c69a 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java @@ -22,7 +22,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.SplitReader; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.ObjectReader; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -32,14 +33,16 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; public class CsvReaderTest { - private static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec("ts", "auto", null); - private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "name")) + private static final InputRowSchema INPUT_ROW_SCHEMA = new InputRowSchema( + new TimestampSpec("ts", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "name"))), + Collections.emptyList() ); @Test @@ -114,7 +117,7 @@ public void testMultiValues() throws IOException ) ); final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), "|", true, 0); - final SplitReader reader = format.createReader(TIMESTAMP_SPEC, DIMENSIONS_SPEC); + final ObjectReader reader = format.createReader(INPUT_ROW_SCHEMA); int numResults = 0; try (CloseableIterator iterator = reader.read(source, null)) { while (iterator.hasNext()) { @@ -153,7 +156,7 @@ private ByteSource writeData(List lines) throws IOException private void assertResult(ByteSource source, CsvInputFormat format) throws IOException { - final SplitReader reader = format.createReader(TIMESTAMP_SPEC, DIMENSIONS_SPEC); + final ObjectReader reader = format.createReader(INPUT_ROW_SCHEMA); int numResults = 0; try (CloseableIterator iterator = reader.read(source, null)) { while (iterator.hasNext()) { diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java index c074bcfd1029..1b0a94e45127 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java @@ -22,7 +22,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.SplitReader; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.ObjectReader; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -59,9 +60,12 @@ public void testParseRow() throws IOException StringUtils.toUtf8("{\"timestamp\":\"2019-01-01\",\"bar\":null,\"foo\":\"x\",\"baz\":4,\"o\":{\"mg\":1}}") ); - final SplitReader reader = format.createReader( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))) + final ObjectReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), + Collections.emptyList() + ) ); try (CloseableIterator iterator = reader.read(source, null)) { while (iterator.hasNext()) { @@ -99,9 +103,12 @@ public void testParseRowWithConditional() throws IOException StringUtils.toUtf8("{\"timestamp\":\"2019-01-01\",\"something_else\": {\"foo\": \"test\"}}") ); - final SplitReader reader = format.createReader( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))) + final ObjectReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), + Collections.emptyList() + ) ); try (CloseableIterator iterator = reader.read(source, null)) { diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java index 5e69e1e77c57..6927f389b7d3 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java @@ -19,7 +19,8 @@ package org.apache.druid.data.input.impl; -import org.apache.druid.data.input.SplitReader; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.ObjectReader; public class NoopInputFormat implements InputFormat { @@ -30,7 +31,7 @@ public boolean isSplittable() } @Override - public SplitReader createReader(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec) + public ObjectReader createReader(InputRowSchema inputRowSchema) { throw new UnsupportedOperationException(); } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java index 9c97b48a2899..ebed0ba01a08 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java @@ -19,7 +19,12 @@ package org.apache.druid.data.input.impl; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; + +import javax.annotation.Nullable; +import java.io.File; public class NoopInputSource implements InputSource { @@ -36,8 +41,18 @@ public boolean isSplittable() } @Override - public boolean isFormattable() + public boolean needsFormat() { return false; } + + @Override + public InputSourceReader reader( + InputRowSchema inputRowSchema, + @Nullable InputFormat inputFormat, + @Nullable File temporaryDirectory + ) + { + return null; + } } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/SplitIteratingReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java similarity index 86% rename from core/src/test/java/org/apache/druid/data/input/impl/SplitIteratingReaderTest.java rename to core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java index 41f417c3e746..cc77d7801def 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/SplitIteratingReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -37,9 +37,10 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.ArrayList; +import java.util.Collections; import java.util.List; -public class SplitIteratingReaderTest +public class ObjectIteratingReaderTest { @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -56,10 +57,13 @@ public void test() throws IOException writer.write(StringUtils.format("%d,%s,%d", 20190102 + i, "name_" + (i + 1), i + 1)); } } - final SplitIteratingReader firehose = new SplitIteratingReader<>( - new TimestampSpec("time", "yyyyMMdd", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "name", "score")) + final ObjectIteratingReader firehose = new ObjectIteratingReader<>( + new InputRowSchema( + new TimestampSpec("time", "yyyyMMdd", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "name", "score")) + ), + Collections.emptyList() ), new CsvInputFormat( ImmutableList.of("time", "name", "score"), @@ -69,7 +73,7 @@ public void test() throws IOException ), files.stream().flatMap(file -> { try { - return ImmutableList.of(new FileSource(new InputSplit<>(file))).stream(); + return ImmutableList.of(new FileSource(file)).stream(); } catch (IOException e) { throw new RuntimeException(e); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 1f2f354737ab..05846fbdf816 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -39,6 +39,7 @@ import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.Rows; @@ -75,6 +76,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.indexing.DataSchema; @@ -119,6 +121,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -129,6 +132,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; public class IndexTask extends AbstractBatchIndexTask implements ChatHandler { @@ -719,10 +723,16 @@ private Map> collectIntervalsAndShardSp Comparators.intervalsByStartThenEnd() ); final Granularity queryGranularity = granularitySpec.getQueryGranularity(); + final List metricsNames = Arrays.stream(ingestionSchema.getDataSchema().getAggregators()) + .map(AggregatorFactory::getName) + .collect(Collectors.toList()); final InputSourceReader inputSourceReader = ingestionSchema.getDataSchema().getTransformSpec().decorate( inputSource.reader( - ingestionSchema.getDataSchema().getNonNullTimestampSpec(), - ingestionSchema.getDataSchema().getNonNullDimensionsSpec(), + new InputRowSchema( + ingestionSchema.getDataSchema().getNonNullTimestampSpec(), + ingestionSchema.getDataSchema().getNonNullDimensionsSpec(), + metricsNames + ), getInputFormat(ingestionSchema), tmpDir ) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java index bece70379861..006dc33fc437 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java @@ -21,6 +21,7 @@ import com.google.common.base.Optional; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.impl.InputFormat; @@ -33,6 +34,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; @@ -44,8 +46,11 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; +import java.util.Arrays; +import java.util.List; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; public class InputSourceProcessor { @@ -95,10 +100,16 @@ public SegmentsAndMetadata process( ? (DynamicPartitionsSpec) partitionsSpec : null; final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); + final List metricsNames = Arrays.stream(dataSchema.getAggregators()) + .map(AggregatorFactory::getName) + .collect(Collectors.toList()); final InputSourceReader inputSourceReader = dataSchema.getTransformSpec().decorate( inputSource.reader( - dataSchema.getNonNullTimestampSpec(), - dataSchema.getNonNullDimensionsSpec(), + new InputRowSchema( + dataSchema.getNonNullTimestampSpec(), + dataSchema.getNonNullDimensionsSpec(), + metricsNames + ), inputFormat, tmpDir ) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 57e305e0a529..a0eb8259f2a4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -27,6 +27,7 @@ import org.apache.commons.io.FileUtils; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.indexer.TaskStatus; @@ -54,6 +55,7 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.query.DruidMetrics; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; @@ -80,6 +82,7 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -423,10 +426,16 @@ private Set generateAndPushSegments( tuningConfig, getContextValue(Tasks.STORE_COMPACTION_STATE_KEY, Tasks.DEFAULT_STORE_COMPACTION_STATE) ); + final List metricsNames = Arrays.stream(ingestionSchema.getDataSchema().getAggregators()) + .map(AggregatorFactory::getName) + .collect(Collectors.toList()); final InputSourceReader inputSourceReader = dataSchema.getTransformSpec().decorate( inputSource.reader( - ingestionSchema.getDataSchema().getNonNullTimestampSpec(), - ingestionSchema.getDataSchema().getNonNullDimensionsSpec(), + new InputRowSchema( + ingestionSchema.getDataSchema().getNonNullTimestampSpec(), + ingestionSchema.getDataSchema().getNonNullDimensionsSpec(), + metricsNames + ), ParallelIndexSupervisorTask.getInputFormat(ingestionSchema), tmpDir ) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index c8211511fd02..4cc9f4521e6b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -227,7 +227,6 @@ public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) throw new RuntimeException(e); } }); - } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index ca83397dcae4..5d0a0f20cdfe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.Iterables; import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.AbstractInputSource; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.InputFormat; @@ -235,7 +236,7 @@ private TestInput(long runTime, TaskState finalState) } } - private static class TestInputSource implements SplittableInputSource + private static class TestInputSource extends AbstractInputSource implements SplittableInputSource { private final List> splits; @@ -275,7 +276,7 @@ public SplittableInputSource withSplit(InputSplit split) } @Override - public boolean isFormattable() + public boolean needsFormat() { return false; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index 63d371f82a22..cb9bdda56b79 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -22,6 +22,7 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.AbstractInputSource; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.SplitHintSpec; @@ -469,7 +470,7 @@ private TestSupervisorTask newTask( ); } - private static class TestInputSource implements SplittableInputSource + private static class TestInputSource extends AbstractInputSource implements SplittableInputSource { private final List ids; @@ -497,7 +498,7 @@ public SplittableInputSource withSplit(InputSplit split) } @Override - public boolean isFormattable() + public boolean needsFormat() { return false; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 64a8dc0d7d3d..b0047be7123c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -33,15 +33,15 @@ import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.MapCache; +import org.apache.druid.data.input.AbstractInputSource; import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowPlusRaw; -import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.NoopInputFormat; @@ -281,25 +281,20 @@ private static InputRow ir(String dt, String dim1, String dim2, float met) ); } - private static class MockExceptionInputSource implements InputSource + private static class MockExceptionInputSource extends AbstractInputSource { @Override - public InputSourceReader reader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - InputFormat inputFormat, - @Nullable File temporaryDirectory - ) + protected InputSourceReader unformattableReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) { return new InputSourceReader() { @Override - public CloseableIterator read() throws IOException + public CloseableIterator read() { return new CloseableIterator() { @Override - public void close() throws IOException + public void close() { } @@ -318,23 +313,30 @@ public InputRow next() } @Override - public CloseableIterator sample() throws IOException + public CloseableIterator sample() { throw new UnsupportedOperationException(); } }; } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public boolean needsFormat() + { + return false; + } } - private static class MockInputSource implements InputSource + private static class MockInputSource extends AbstractInputSource { @Override - public InputSourceReader reader( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - InputFormat inputFormat, - @Nullable File temporaryDirectory - ) + protected InputSourceReader unformattableReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) { return new InputSourceReader() { @@ -352,6 +354,18 @@ public CloseableIterator sample() } }; } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public boolean needsFormat() + { + return false; + } } private static class MockFirehoseFactory implements FirehoseFactory From 546d957d52cfc3b2ba3c36c633e50679d72720a1 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 8 Nov 2019 17:44:43 -0800 Subject: [PATCH 12/29] address comments --- .../druid/data/input/InputRowSchema.java | 10 +++---- .../apache/druid/data/input/TextReader.java | 15 ++++++++--- .../druid/data/input/impl/CsvReader.java | 27 ++++++++++++------- .../druid/data/input/impl/FileSource.java | 11 ++++---- .../FirehoseToInputSourceReaderAdaptor.java | 9 ++++--- .../druid/data/input/impl/JSONParseSpec.java | 2 -- .../data/input/impl/JsonInputFormat.java | 2 +- .../druid/data/input/impl/JsonReader.java | 8 +++++- .../data/input/impl/LocalInputSource.java | 9 ++----- .../data/input/impl/NestedInputFormat.java | 2 ++ .../input/impl/ObjectIteratingReader.java | 4 +-- .../druid/java/util/common/FileUtils.java | 17 ++++-------- ...rehoseFactoryToInputSourceAdaptorTest.java | 6 ++--- .../data/input/impl/CsvInputFormatTest.java | 15 +++-------- .../druid/data/input/impl/CsvReaderTest.java | 2 +- .../druid/data/input/impl/JsonReaderTest.java | 8 ++++++ .../data/input/impl/NoopFirehoseFactory.java | 13 ++++----- .../data/input/impl/NoopInputSource.java | 2 +- .../input/impl/ObjectIteratingReaderTest.java | 4 ++- .../indexer/HadoopDruidIndexerConfig.java | 1 - .../druid/indexer/IndexGeneratorJob.java | 10 +++---- .../druid/indexing/common/TaskToolbox.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 8 +++--- .../common/task/InputSourceProcessor.java | 4 +-- .../parallel/ParallelIndexIngestionSpec.java | 2 +- .../batch/parallel/SinglePhaseSubTask.java | 4 +-- .../common/task/CompactionTaskTest.java | 4 +-- .../indexing/common/task/IndexTaskTest.java | 1 - .../MultiPhaseParallelIndexingTest.java | 1 - .../SinglePhaseParallelIndexingTest.java | 1 - .../druid/segment/indexing/DataSchema.java | 15 ++++++----- .../druid/segment/realtime/plumber/Sink.java | 4 +-- .../segment/indexing/DataSchemaTest.java | 3 +-- 33 files changed, 117 insertions(+), 109 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java b/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java index ca01fca3b5d6..b4b7fc477fd4 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java +++ b/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java @@ -28,13 +28,13 @@ public class InputRowSchema { private final TimestampSpec timestampSpec; private final DimensionsSpec dimensionsSpec; - private final List metricsNames; + private final List metricNames; - public InputRowSchema(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec, List metricsNames) + public InputRowSchema(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec, List metricNames) { this.timestampSpec = timestampSpec; this.dimensionsSpec = dimensionsSpec; - this.metricsNames = metricsNames; + this.metricNames = metricNames; } public TimestampSpec getTimestampSpec() @@ -47,8 +47,8 @@ public DimensionsSpec getDimensionsSpec() return dimensionsSpec; } - public List getMetricsNames() + public List getMetricNames() { - return metricsNames; + return metricNames; } } diff --git a/core/src/main/java/org/apache/druid/data/input/TextReader.java b/core/src/main/java/org/apache/druid/data/input/TextReader.java index ff0c4124be1a..9571eda010dc 100644 --- a/core/src/main/java/org/apache/druid/data/input/TextReader.java +++ b/core/src/main/java/org/apache/druid/data/input/TextReader.java @@ -80,8 +80,11 @@ private CloseableIterator lineIterator(ObjectSource source) throws IOExc final LineIterator delegate = new LineIterator( new InputStreamReader(source.open(), StringUtils.UTF8_STRING) ); - final int numHeaderLines = getNumHeaderLines(); + final int numHeaderLines = getNumHeaderLinesToSkip(); for (int i = 0; i < numHeaderLines && delegate.hasNext(); i++) { + delegate.nextLine(); // skip lines + } + if (needsToProcessHeaderLine() && delegate.hasNext()) { processHeaderLine(delegate.nextLine()); } @@ -116,10 +119,16 @@ public void close() throws IOException * Returns the number of header lines to skip. * {@link #processHeaderLine} will be called as many times as the returned number. */ - public abstract int getNumHeaderLines(); + public abstract int getNumHeaderLinesToSkip(); + + /** + * Returns true if the file format needs to process a header line. + * This method will be called after skipping lines as many as {@link #getNumHeaderLinesToSkip()}. + */ + public abstract boolean needsToProcessHeaderLine(); /** - * Processes a header line. This will be called as many times as {@link #getNumHeaderLines()}. + * Processes a header line. This will be called as many times as {@link #getNumHeaderLinesToSkip()}. */ public abstract void processHeaderLine(String line) throws IOException; } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java index a45e4b0372cc..644a7e0cdf6d 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java @@ -44,7 +44,7 @@ public class CsvReader extends TextReader { private final CSVParser parser = new CSVParser(); - private final boolean hasHeaderRow; + private final boolean findColumnsFromHeader; private final int skipHeaderRows; private final Function multiValueFunction; @Nullable @@ -54,16 +54,16 @@ public class CsvReader extends TextReader InputRowSchema inputRowSchema, String listDelimiter, @Nullable List columns, - boolean hasHeaderRow, + boolean findColumnsFromHeader, int skipHeaderRows ) { super(inputRowSchema); - this.hasHeaderRow = hasHeaderRow; + this.findColumnsFromHeader = findColumnsFromHeader; this.skipHeaderRows = skipHeaderRows; final String finalListDelimeter = listDelimiter == null ? Parsers.DEFAULT_LIST_DELIMITER : listDelimiter; this.multiValueFunction = ParserUtils.getMultiValueFunction(finalListDelimeter, Splitter.on(finalListDelimeter)); - this.columns = hasHeaderRow ? null : columns; // columns will be overriden by header row + this.columns = findColumnsFromHeader ? null : columns; // columns will be overriden by header row if (this.columns != null) { for (String column : this.columns) { @@ -72,7 +72,7 @@ public class CsvReader extends TextReader verify(this.columns, inputRowSchema.getDimensionsSpec().getDimensionNames()); } else { Preconditions.checkArgument( - hasHeaderRow, + findColumnsFromHeader, "If columns field is not set, the first row of your data must have your header" + " and hasHeaderRow must be set to true." ); @@ -95,18 +95,25 @@ public InputRow readLine(String line) throws IOException, ParseException } @Override - public int getNumHeaderLines() + public int getNumHeaderLinesToSkip() { - return (hasHeaderRow ? 1 : 0) + skipHeaderRows; + return skipHeaderRows; + } + + @Override + public boolean needsToProcessHeaderLine() + { + return findColumnsFromHeader; } @Override public void processHeaderLine(String line) throws IOException { - if (hasHeaderRow && (columns == null || columns.isEmpty())) { - columns = findOrCreateColumnNames(Arrays.asList(parser.parseLine(line))); + if (!findColumnsFromHeader) { + throw new ISE("Don't call this if findColumnsFromHeader = false"); } - if (columns == null || columns.isEmpty()) { + columns = findOrCreateColumnNames(Arrays.asList(parser.parseLine(line))); + if (columns.isEmpty()) { throw new ISE("Empty columns"); } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java b/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java index 59068ca63479..58b3fcc9bb8a 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java @@ -30,18 +30,16 @@ import java.io.InputStream; import java.io.RandomAccessFile; import java.nio.channels.Channels; -import java.nio.channels.FileChannel; public class FileSource implements ObjectSource { private final File file; - private final FileChannel channel; + private final RandomAccessFile randomAccessFile; FileSource(File file) throws FileNotFoundException { this.file = file; - final RandomAccessFile randomAccessFile = new RandomAccessFile(file, "r"); - this.channel = randomAccessFile.getChannel(); + this.randomAccessFile = new RandomAccessFile(file, "r"); } @Override @@ -56,9 +54,10 @@ public File file() } @Override - public void close() + public void close() throws IOException { // do nothing + randomAccessFile.close(); } }; } @@ -72,7 +71,7 @@ public File getObject() @Override public InputStream open() throws IOException { - return CompressionUtils.decompress(Channels.newInputStream(channel), file.getName()); + return CompressionUtils.decompress(Channels.newInputStream(randomAccessFile.getChannel()), file.getName()); } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java index f62efc44aecd..afadec7de7ef 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java @@ -28,6 +28,7 @@ import java.io.File; import java.io.IOException; +import java.io.UncheckedIOException; public class FirehoseToInputSourceReaderAdaptor implements InputSourceReader { @@ -60,7 +61,7 @@ public boolean hasNext() return firehose.hasMore(); } catch (IOException e) { - throw new RuntimeException(e); + throw new UncheckedIOException(e); } } @@ -71,7 +72,7 @@ public InputRow next() return firehose.nextRow(); } catch (IOException e) { - throw new RuntimeException(e); + throw new UncheckedIOException(e); } } @@ -97,7 +98,7 @@ public boolean hasNext() return firehose.hasMore(); } catch (IOException e) { - throw new RuntimeException(e); + throw new UncheckedIOException(e); } } @@ -108,7 +109,7 @@ public InputRowPlusRaw next() return firehose.nextRowWithRaw(); } catch (IOException e) { - throw new RuntimeException(e); + throw new UncheckedIOException(e); } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JSONParseSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/JSONParseSpec.java index d2e96f60b650..d4147bd97c52 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JSONParseSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JSONParseSpec.java @@ -27,7 +27,6 @@ import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.java.util.common.parsers.Parser; -import javax.annotation.Nullable; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -74,7 +73,6 @@ public Parser makeParser() return new JSONPathParser(getFlattenSpec(), objectMapper); } - @Nullable @Override public InputFormat toInputFormat() { diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java index bf43e16fa502..9e7dbd88105d 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java @@ -40,7 +40,7 @@ public class JsonInputFormat extends NestedInputFormat @JsonCreator public JsonInputFormat( - @JsonProperty("flattenSpec") JSONPathSpec flattenSpec, + @JsonProperty("flattenSpec") @Nullable JSONPathSpec flattenSpec, @JsonProperty("featureSpec") @Nullable Map featureSpec ) { diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java index 6940ee8bb918..3a9f886c291f 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java @@ -62,11 +62,17 @@ public InputRow readLine(String line) throws IOException, ParseException } @Override - public int getNumHeaderLines() + public int getNumHeaderLinesToSkip() { return 0; } + @Override + public boolean needsToProcessHeaderLine() + { + return false; + } + @Override public void processHeaderLine(String line) { diff --git a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java index 119d7dcd1257..422112e26415 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import com.google.common.collect.Iterators; import org.apache.commons.io.FileUtils; import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.commons.io.filefilter.WildcardFileFilter; @@ -78,13 +79,7 @@ public Stream> createSplits(InputFormat inputFormat, @Nullable @Override public int getNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) { - final Iterator fileIterator = getFileIterator(); - int num = 0; - while (fileIterator.hasNext()) { - fileIterator.next(); - num++; - } - return num; + return Iterators.size(getFileIterator()); } private Iterator getFileIterator() diff --git a/core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java index d63c3629667f..0ee8a867d892 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java @@ -31,6 +31,7 @@ */ public abstract class NestedInputFormat implements InputFormat { + @Nullable private final JSONPathSpec flattenSpec; protected NestedInputFormat(@Nullable JSONPathSpec flattenSpec) @@ -38,6 +39,7 @@ protected NestedInputFormat(@Nullable JSONPathSpec flattenSpec) this.flattenSpec = flattenSpec == null ? JSONPathSpec.DEFAULT : flattenSpec; } + @Nullable @JsonProperty("flattenSpec") public JSONPathSpec getFlattenSpec() { diff --git a/core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java index d0193270e11e..b2e9347839d9 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java @@ -92,7 +92,7 @@ private CloseableIterator createIterator(Function long copyLarge( String messageOnRetry ) throws IOException { - try { - return RetryUtils.retry( - () -> { - try (InputStream inputStream = objectOpenFunction.open(object); - OutputStream out = new FileOutputStream(outFile)) { - return IOUtils.copyLarge(inputStream, out, fetchBuffer); - } - }, + try (InputStream inputStream = objectOpenFunction.open(object)) { + return copyLarge( + inputStream, + outFile, + fetchBuffer, retryCondition, - outFile::delete, numRetries, messageOnRetry ); } - catch (Exception e) { - throw new IOException(e); - } } public static long copyLarge( diff --git a/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java b/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java index 5f88b56848c3..faeed4951e2e 100644 --- a/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java +++ b/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java @@ -53,7 +53,7 @@ public void testUnimplementedInputFormat() throws IOException } final TestFirehoseFactory firehoseFactory = new TestFirehoseFactory(lines); final StringInputRowParser inputRowParser = new StringInputRowParser( - new TestCsvParseSpec( + new UnimplementedInputFormatCsvParseSpec( new TimestampSpec(null, "yyyyMMdd", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "name", "score"))), ",", @@ -96,9 +96,9 @@ public void testUnimplementedInputFormat() throws IOException } } - private static class TestCsvParseSpec extends CSVParseSpec + private static class UnimplementedInputFormatCsvParseSpec extends CSVParseSpec { - private TestCsvParseSpec( + private UnimplementedInputFormatCsvParseSpec( TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec, String listDelimiter, diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java index cca122c800ac..aa04d685860a 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java @@ -46,22 +46,15 @@ public void testSerde() throws IOException } @Test - public void testColumnMissing() + public void testComma() { - final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), ",", false, 0); expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("column[b] not in columns"); - format.createReader( - new InputRowSchema( - new TimestampSpec("timestamp", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b"))), - Collections.emptyList() - ) - ); + expectedException.expectMessage("Column[a,] has a comma, it cannot"); + new CsvInputFormat(Collections.singletonList("a,"), ",", false, 0); } @Test - public void testComma() + public void testComma2() { final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), ",", false, 0); expectedException.expect(IllegalArgumentException.class); diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java index 50b07740c69a..0041fb44f819 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java @@ -94,8 +94,8 @@ public void testFindColumnAndSkipHeaders() throws IOException { final ByteSource source = writeData( ImmutableList.of( - "ts,name,score", "this,is,a,row,to,skip", + "ts,name,score", "2019-01-01T00:00:10Z,name_1,5", "2019-01-01T00:00:20Z,name_2,10", "2019-01-01T00:00:30Z,name_3,15" diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java index 1b0a94e45127..c16fe260b515 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java @@ -67,7 +67,9 @@ public void testParseRow() throws IOException Collections.emptyList() ) ); + final int numExpectedIterations = 1; try (CloseableIterator iterator = reader.read(source, null)) { + int numActualIterations = 0; while (iterator.hasNext()) { final InputRow row = iterator.next(); Assert.assertEquals(DateTimes.of("2019-01-01"), row.getTimestamp()); @@ -80,7 +82,9 @@ public void testParseRow() throws IOException Assert.assertTrue(row.getDimension("root_baz2").isEmpty()); Assert.assertTrue(row.getDimension("path_omg2").isEmpty()); Assert.assertTrue(row.getDimension("jq_omg2").isEmpty()); + numActualIterations++; } + Assert.assertEquals(numExpectedIterations, numActualIterations); } } @@ -111,13 +115,17 @@ public void testParseRowWithConditional() throws IOException ) ); + final int numExpectedIterations = 1; try (CloseableIterator iterator = reader.read(source, null)) { + int numActualIterations = 0; while (iterator.hasNext()) { final InputRow row = iterator.next(); Assert.assertEquals("test", Iterables.getOnlyElement(row.getDimension("bar"))); Assert.assertEquals(Collections.emptyList(), row.getDimension("foo")); Assert.assertTrue(row.getDimension("baz").isEmpty()); + numActualIterations++; } + Assert.assertEquals(numExpectedIterations, numActualIterations); } } } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java index 8b6fc94f7e67..2669a968d045 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java @@ -24,7 +24,6 @@ import org.apache.druid.data.input.SplitHintSpec; import javax.annotation.Nullable; -import java.io.IOException; import java.util.stream.Stream; public class NoopFirehoseFactory implements FiniteFirehoseFactory @@ -36,22 +35,20 @@ public String toString() } @Override - public Stream getSplits( - @Nullable SplitHintSpec splitHintSpec - ) throws IOException + public Stream getSplits(@Nullable SplitHintSpec splitHintSpec) { - return null; + throw new UnsupportedOperationException(); } @Override - public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) throws IOException + public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) { - return 0; + throw new UnsupportedOperationException(); } @Override public FiniteFirehoseFactory withSplit(InputSplit split) { - return null; + throw new UnsupportedOperationException(); } } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java index ebed0ba01a08..49cb6e9ab00a 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java @@ -53,6 +53,6 @@ public InputSourceReader reader( @Nullable File temporaryDirectory ) { - return null; + throw new UnsupportedOperationException(); } } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java index cc77d7801def..0f22d273b9d7 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java @@ -48,8 +48,9 @@ public class ObjectIteratingReaderTest @Test public void test() throws IOException { + final int numFiles = 5; final List files = new ArrayList<>(); - for (int i = 0; i < 5; i++) { + for (int i = 0; i < numFiles; i++) { final File file = temporaryFolder.newFile("test_" + i); files.add(file); try (Writer writer = Files.newBufferedWriter(file.toPath(), StandardCharsets.UTF_8)) { @@ -97,6 +98,7 @@ public void test() throws IOException Assert.assertEquals(Integer.toString(i + 1), Iterables.getOnlyElement(row.getDimension("score"))); i++; } + Assert.assertEquals(numFiles, i); } } } diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java index 80a44676f94f..c7abd34c1172 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java @@ -581,7 +581,6 @@ public void verify() Preconditions.checkNotNull(schema.getDataSchema().getDataSource(), "dataSource"); Preconditions.checkNotNull(schema.getDataSchema().getParser(), "inputRowParser"); Preconditions.checkNotNull(schema.getDataSchema().getParser().getParseSpec(), "parseSpec"); - Preconditions.checkNotNull(schema.getDataSchema().getNonNullTimestampSpec(), "timestampSpec"); Preconditions.checkNotNull(schema.getDataSchema().getGranularitySpec(), "granularitySpec"); Preconditions.checkNotNull(pathSpec, "inputSpec"); Preconditions.checkNotNull(schema.getTuningConfig().getWorkingPath(), "workingPath"); diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java index f3ff85df5295..8fcbb8dc674f 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java @@ -294,8 +294,8 @@ private static IncrementalIndex makeIncrementalIndex( final HadoopTuningConfig tuningConfig = config.getSchema().getTuningConfig(); final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder() .withMinTimestamp(theBucket.time.getMillis()) - .withTimestampSpec(config.getSchema().getDataSchema().getNonNullTimestampSpec()) - .withDimensionsSpec(config.getSchema().getDataSchema().getNonNullDimensionsSpec()) + .withTimestampSpec(config.getSchema().getDataSchema().getTimestampSpec()) + .withDimensionsSpec(config.getSchema().getDataSchema().getDimensionsSpec()) .withQueryGranularity(config.getSchema().getDataSchema().getGranularitySpec().getQueryGranularity()) .withMetrics(aggs) .withRollup(config.getSchema().getDataSchema().getGranularitySpec().isRollup()) @@ -341,7 +341,7 @@ protected void setup(Context context) aggsForSerializingSegmentInputRow[i] = aggregators[i].getCombiningFactory(); } } - typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema().getDataSchema().getNonNullDimensionsSpec()); + typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema().getDataSchema().getDimensionsSpec()); } @Override @@ -427,7 +427,7 @@ protected void setup(Context context) for (int i = 0; i < aggregators.length; ++i) { combiningAggs[i] = aggregators[i].getCombiningFactory(); } - typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema().getDataSchema().getNonNullDimensionsSpec()); + typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema().getDataSchema().getDimensionsSpec()); } @Override @@ -623,7 +623,7 @@ protected void setup(Context context) metricNames.add(aggregators[i].getName()); combiningAggs[i] = aggregators[i].getCombiningFactory(); } - typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema().getDataSchema().getNonNullDimensionsSpec()); + typeHelperMap = InputRowSerde.getTypeHelperMap(config.getSchema().getDataSchema().getDimensionsSpec()); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index 1bd29dd6d195..d1e04598ef1a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -296,7 +296,7 @@ public IndexMergerV9 getIndexMergerV9() public File getIndexingTmpDir() { - return new File(taskWorkDir, "firehose"); + return new File(taskWorkDir, "indexing-tmp"); } public File getMergeDir() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 05846fbdf816..53953754f805 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -729,8 +729,8 @@ private Map> collectIntervalsAndShardSp final InputSourceReader inputSourceReader = ingestionSchema.getDataSchema().getTransformSpec().decorate( inputSource.reader( new InputRowSchema( - ingestionSchema.getDataSchema().getNonNullTimestampSpec(), - ingestionSchema.getDataSchema().getNonNullDimensionsSpec(), + ingestionSchema.getDataSchema().getTimestampSpec(), + ingestionSchema.getDataSchema().getDimensionsSpec(), metricsNames ), getInputFormat(ingestionSchema), @@ -1061,7 +1061,7 @@ public IndexIngestionSpec( ) ); if (dataSchema.getParserMap() != null && ioConfig.getInputSource() != null) { - throw new IAE("Cannot use parser and inputSource together. Try use inputFormat instead of parser."); + throw new IAE("Cannot use parser and inputSource together. Try using inputFormat instead of parser."); } this.dataSchema = dataSchema; @@ -1113,7 +1113,7 @@ public IndexIOConfig( ImmutableList.of(new Property<>("firehose", firehoseFactory), new Property<>("inputSource", inputSource)) ); if (firehoseFactory != null && inputFormat != null) { - throw new IAE("Cannot use firehose and inputFormat together. Try use inputSource instead of firehose."); + throw new IAE("Cannot use firehose and inputFormat together. Try using inputSource instead of firehose."); } this.firehoseFactory = firehoseFactory; this.inputSource = inputSource; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java index 006dc33fc437..4162831494d2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java @@ -106,8 +106,8 @@ public SegmentsAndMetadata process( final InputSourceReader inputSourceReader = dataSchema.getTransformSpec().decorate( inputSource.reader( new InputRowSchema( - dataSchema.getNonNullTimestampSpec(), - dataSchema.getNonNullDimensionsSpec(), + dataSchema.getTimestampSpec(), + dataSchema.getDimensionsSpec(), metricsNames ), inputFormat, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java index 8d9a85d43878..9c448992eccf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java @@ -52,7 +52,7 @@ public ParallelIndexIngestionSpec( ); if (dataSchema.getParserMap() != null && ioConfig.getInputSource() != null) { if (!(ioConfig.getInputSource() instanceof FirehoseFactoryToInputSourceAdaptor)) { - throw new IAE("Cannot use parser and inputSource together. Try use inputFormat instead of parser."); + throw new IAE("Cannot use parser and inputSource together. Try using inputFormat instead of parser."); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index a0eb8259f2a4..6408299d084f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -432,8 +432,8 @@ private Set generateAndPushSegments( final InputSourceReader inputSourceReader = dataSchema.getTransformSpec().decorate( inputSource.reader( new InputRowSchema( - ingestionSchema.getDataSchema().getNonNullTimestampSpec(), - ingestionSchema.getDataSchema().getNonNullDimensionsSpec(), + ingestionSchema.getDataSchema().getTimestampSpec(), + ingestionSchema.getDataSchema().getDimensionsSpec(), metricsNames ), ParallelIndexSupervisorTask.getInputFormat(ingestionSchema), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 8c1a0c87f091..89fb241007aa 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1057,10 +1057,10 @@ private void assertIngestionSchema( final DataSchema dataSchema = ingestionSchema.getDataSchema(); Assert.assertEquals(DATA_SOURCE, dataSchema.getDataSource()); - Assert.assertEquals(new TimestampSpec(null, null, null), dataSchema.getNonNullTimestampSpec()); + Assert.assertEquals(new TimestampSpec(null, null, null), dataSchema.getTimestampSpec()); Assert.assertEquals( new HashSet<>(expectedDimensionsSpec.getDimensions()), - new HashSet<>(dataSchema.getNonNullDimensionsSpec().getDimensions()) + new HashSet<>(dataSchema.getDimensionsSpec().getDimensions()) ); // metrics diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 8b5c2b46dc27..6c2144b5eeab 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -138,7 +138,6 @@ public static Iterable constructorFeeder() return ImmutableList.of( new Object[]{LockGranularity.TIME_CHUNK, false}, new Object[]{LockGranularity.TIME_CHUNK, true}, - new Object[]{LockGranularity.SEGMENT, false}, new Object[]{LockGranularity.SEGMENT, true} ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java index a7bc506fe9bf..b543247697f5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java @@ -97,7 +97,6 @@ public static Iterable constructorFeeder() return ImmutableList.of( new Object[]{LockGranularity.TIME_CHUNK, false}, new Object[]{LockGranularity.TIME_CHUNK, true}, - new Object[]{LockGranularity.SEGMENT, false}, new Object[]{LockGranularity.SEGMENT, true} ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index c2efc2aed18b..cf59e8fbb02f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -74,7 +74,6 @@ public static Iterable constructorFeeder() return ImmutableList.of( new Object[]{LockGranularity.TIME_CHUNK, false}, new Object[]{LockGranularity.TIME_CHUNK, true}, - new Object[]{LockGranularity.SEGMENT, false}, new Object[]{LockGranularity.SEGMENT, true} ); } diff --git a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java index 8a8b0a91d34e..1c088f13293f 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Sets; @@ -118,6 +119,7 @@ public DataSchema( } } + @VisibleForTesting public DataSchema( String dataSource, TimestampSpec timestampSpec, @@ -197,12 +199,12 @@ public String getDataSource() @Nullable @JsonProperty - public TimestampSpec getTimestampSpec() + private TimestampSpec getGivenTimestampSpec() { return timestampSpec; } - public TimestampSpec getNonNullTimestampSpec() + public TimestampSpec getTimestampSpec() { if (timestampSpec == null) { timestampSpec = Preconditions.checkNotNull(getParser(), "inputRowParser").getParseSpec().getTimestampSpec(); @@ -212,16 +214,16 @@ public TimestampSpec getNonNullTimestampSpec() @Nullable @JsonProperty - public DimensionsSpec getDimensionsSpec() + private DimensionsSpec getGivenDimensionsSpec() { return dimensionsSpec; } - public DimensionsSpec getNonNullDimensionsSpec() + public DimensionsSpec getDimensionsSpec() { if (dimensionsSpec == null) { dimensionsSpec = computeDimensionsSpec( - getNonNullTimestampSpec(), + getTimestampSpec(), Preconditions.checkNotNull(getParser(), "inputRowParser").getParseSpec().getDimensionsSpec(), aggregators ); @@ -316,9 +318,10 @@ public String toString() ", aggregators=" + Arrays.toString(aggregators) + ", granularitySpec=" + granularitySpec + ", transformSpec=" + transformSpec + - (parserMap == null ? "" : ", parserMap=" + parserMap) + + ", parserMap=" + parserMap + ", timestampSpec=" + timestampSpec + ", dimensionsSpec=" + dimensionsSpec + + ", inputRowParser=" + inputRowParser + '}'; } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java index ba405f886724..7002ac22fa66 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java @@ -354,9 +354,9 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) { final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder() .withMinTimestamp(minTimestamp) - .withTimestampSpec(schema.getNonNullTimestampSpec()) + .withTimestampSpec(schema.getTimestampSpec()) .withQueryGranularity(schema.getGranularitySpec().getQueryGranularity()) - .withDimensionsSpec(schema.getNonNullDimensionsSpec()) + .withDimensionsSpec(schema.getDimensionsSpec()) .withMetrics(schema.getAggregators()) .withRollup(schema.getGranularitySpec().isRollup()) .build(); diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 435080ffddbc..29842276bad3 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -92,7 +92,7 @@ public void testDefaultExclusions() Assert.assertEquals( ImmutableSet.of("time", "col1", "col2", "metric1", "metric2"), - schema.getNonNullDimensionsSpec().getDimensionExclusions() + schema.getDimensionsSpec().getDimensionExclusions() ); } @@ -361,7 +361,6 @@ private void testInvalidWhitespaceDatasourceHelper(String dataSource, String inv } } - @Test public void testSerde() throws Exception { From 7bb5d5fb1e86e65b4b8aca69a4150df272f149f3 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 8 Nov 2019 23:41:31 -0800 Subject: [PATCH 13/29] fix closing resource --- .../apache/druid/data/input/impl/FileSource.java | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java b/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java index 58b3fcc9bb8a..9a94689031e7 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java @@ -25,21 +25,17 @@ import org.apache.druid.utils.CompressionUtils; import java.io.File; -import java.io.FileNotFoundException; +import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; -import java.io.RandomAccessFile; -import java.nio.channels.Channels; public class FileSource implements ObjectSource { private final File file; - private final RandomAccessFile randomAccessFile; - FileSource(File file) throws FileNotFoundException + FileSource(File file) { this.file = file; - this.randomAccessFile = new RandomAccessFile(file, "r"); } @Override @@ -54,10 +50,9 @@ public File file() } @Override - public void close() throws IOException + public void close() { // do nothing - randomAccessFile.close(); } }; } @@ -71,7 +66,7 @@ public File getObject() @Override public InputStream open() throws IOException { - return CompressionUtils.decompress(Channels.newInputStream(randomAccessFile.getChannel()), file.getName()); + return CompressionUtils.decompress(new FileInputStream(file), file.getName()); } @Override From 6dba81a04617ec6cee1751863bb2361c93cbfc9c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 8 Nov 2019 23:43:54 -0800 Subject: [PATCH 14/29] fix checkstyle --- .../src/main/java/org/apache/druid/data/input/ObjectSource.java | 2 +- .../org/apache/druid/data/input/impl/ObjectIteratingReader.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/ObjectSource.java b/core/src/main/java/org/apache/druid/data/input/ObjectSource.java index 1f522922b87c..e83a67648ce9 100644 --- a/core/src/main/java/org/apache/druid/data/input/ObjectSource.java +++ b/core/src/main/java/org/apache/druid/data/input/ObjectSource.java @@ -34,7 +34,7 @@ * ObjectSource abstracts an object and knows how to read bytes from the given object. */ @ExtensionPoint - public interface ObjectSource +public interface ObjectSource { Logger LOG = new Logger(ObjectSource.class); diff --git a/core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java index b2e9347839d9..0828636c9f2a 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java @@ -23,8 +23,8 @@ import org.apache.druid.data.input.InputRowPlusRaw; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; -import org.apache.druid.data.input.ObjectSource; import org.apache.druid.data.input.ObjectReader; +import org.apache.druid.data.input.ObjectSource; import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.File; From ea2c8f9db662e6d03eac46266fa601ce177f8b86 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 8 Nov 2019 23:59:18 -0800 Subject: [PATCH 15/29] fix tests --- .../druid/data/input/impl/LocalInputSource.java | 10 +--------- .../druid/java/util/common/FileUtils.java | 17 ++++++++++++----- .../input/impl/ObjectIteratingReaderTest.java | 9 +-------- .../druid/segment/indexing/DataSchema.java | 4 ++-- 4 files changed, 16 insertions(+), 24 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java index 422112e26415..04dde8de170b 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java @@ -34,7 +34,6 @@ import javax.annotation.Nullable; import java.io.File; -import java.io.FileNotFoundException; import java.util.Iterator; import java.util.Objects; import java.util.Spliterator; @@ -117,14 +116,7 @@ protected InputSourceReader formattableReader( // reader() is supposed to be called in each task that creates segments. // The task should already have only one split in parallel indexing, // while there's no need to make splits using splitHintSpec in sequential indexing. - createSplits(inputFormat, null).map(split -> { - try { - return new FileSource(split.get()); - } - catch (FileNotFoundException e) { - throw new RuntimeException(e); - } - }), + createSplits(inputFormat, null).map(split -> new FileSource(split.get())), temporaryDirectory ); } diff --git a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java index fc78fc7e29a8..aedf67166b0e 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java @@ -287,16 +287,23 @@ public static long copyLarge( String messageOnRetry ) throws IOException { - try (InputStream inputStream = objectOpenFunction.open(object)) { - return copyLarge( - inputStream, - outFile, - fetchBuffer, + try { + return RetryUtils.retry( + () -> { + try (InputStream inputStream = objectOpenFunction.open(object); + OutputStream out = new FileOutputStream(outFile)) { + return IOUtils.copyLarge(inputStream, out, fetchBuffer); + } + }, retryCondition, + outFile::delete, numRetries, messageOnRetry ); } + catch (Exception e) { + throw new IOException(e); + } } public static long copyLarge( diff --git a/core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java index 0f22d273b9d7..ba9838627f7b 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java @@ -72,14 +72,7 @@ public void test() throws IOException false, 0 ), - files.stream().flatMap(file -> { - try { - return ImmutableList.of(new FileSource(file)).stream(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - }), + files.stream().flatMap(file -> ImmutableList.of(new FileSource(file)).stream()), temporaryFolder.newFolder() ); diff --git a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java index 1c088f13293f..f0c7bffa9885 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java @@ -198,7 +198,7 @@ public String getDataSource() } @Nullable - @JsonProperty + @JsonProperty("timestampSpec") private TimestampSpec getGivenTimestampSpec() { return timestampSpec; @@ -213,7 +213,7 @@ public TimestampSpec getTimestampSpec() } @Nullable - @JsonProperty + @JsonProperty("dimensionsSpec") private DimensionsSpec getGivenDimensionsSpec() { return dimensionsSpec; From 1ea7758489cc8c9d708bd691fd48e62085fd9455 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 10 Nov 2019 12:01:38 -0800 Subject: [PATCH 16/29] remove verify from csv --- .../java/org/apache/druid/data/input/impl/CsvReader.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java index 644a7e0cdf6d..de92812820b1 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java @@ -69,7 +69,6 @@ public class CsvReader extends TextReader for (String column : this.columns) { Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column); } - verify(this.columns, inputRowSchema.getDimensionsSpec().getDimensionNames()); } else { Preconditions.checkArgument( findColumnsFromHeader, @@ -118,13 +117,6 @@ public void processHeaderLine(String line) throws IOException } } - public static void verify(List columns, List dimensionNames) - { - for (String columnName : dimensionNames) { - Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName); - } - } - public static List findOrCreateColumnNames(List parsedLine) { final List columns = new ArrayList<>(parsedLine.size()); From 218b3926859a9fe9ea7763249bd9d2c9764b4003 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 10 Nov 2019 22:11:04 -0800 Subject: [PATCH 17/29] Revert "remove verify from csv" This reverts commit 1ea7758489cc8c9d708bd691fd48e62085fd9455. --- .../java/org/apache/druid/data/input/impl/CsvReader.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java index de92812820b1..644a7e0cdf6d 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java @@ -69,6 +69,7 @@ public class CsvReader extends TextReader for (String column : this.columns) { Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column); } + verify(this.columns, inputRowSchema.getDimensionsSpec().getDimensionNames()); } else { Preconditions.checkArgument( findColumnsFromHeader, @@ -117,6 +118,13 @@ public void processHeaderLine(String line) throws IOException } } + public static void verify(List columns, List dimensionNames) + { + for (String columnName : dimensionNames) { + Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName); + } + } + public static List findOrCreateColumnNames(List parsedLine) { final List columns = new ArrayList<>(parsedLine.size()); From 7098056ccf46e0380f5d95f52b2e39161fcf710a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 11 Nov 2019 15:10:31 -0800 Subject: [PATCH 18/29] address comments --- .../druid/data/input/AbstractInputSource.java | 1 - .../data/input/FiniteFirehoseFactory.java | 2 ++ .../FirehoseFactoryToInputSourceAdaptor.java | 1 - .../{ObjectSource.java => InputEntity.java} | 17 ++++++-------- ...jectReader.java => InputEntityReader.java} | 10 ++++----- .../data/input/{impl => }/InputFormat.java | 16 ++++++++------ .../apache/druid/data/input/InputSource.java | 11 ++++++---- .../druid/data/input/InputSourceReader.java | 7 +++--- .../apache/druid/data/input/TextReader.java | 10 ++++----- .../druid/data/input/impl/ByteSource.java | 6 ++--- .../druid/data/input/impl/CSVParseSpec.java | 1 + .../druid/data/input/impl/CsvInputFormat.java | 5 +++-- .../druid/data/input/impl/FileSource.java | 6 ++--- .../data/input/impl/HttpInputSource.java | 3 ++- .../druid/data/input/impl/HttpSource.java | 6 ++--- ...r.java => InputEntityIteratingReader.java} | 22 ++++++++++--------- .../druid/data/input/impl/JSONParseSpec.java | 1 + .../data/input/impl/JsonInputFormat.java | 4 ++-- .../data/input/impl/LocalInputSource.java | 3 ++- .../data/input/impl/NestedInputFormat.java | 1 + .../druid/data/input/impl/ParseSpec.java | 1 + .../input/impl/SplittableInputSource.java | 1 + ...rehoseFactoryToInputSourceAdaptorTest.java | 1 - .../data/input/impl/CsvInputFormatTest.java | 1 + .../druid/data/input/impl/CsvReaderTest.java | 6 ++--- ...va => InputEntityIteratingReaderTest.java} | 4 ++-- .../data/input/impl/JsonInputFormatTest.java | 1 + .../druid/data/input/impl/JsonReaderTest.java | 6 ++--- .../data/input/impl/NoopInputFormat.java | 5 +++-- .../data/input/impl/NoopInputSource.java | 1 + .../druid/indexing/common/task/IndexTask.java | 2 +- .../common/task/InputSourceProcessor.java | 2 +- .../batch/parallel/ParallelIndexIOConfig.java | 2 +- .../parallel/ParallelIndexPhaseRunner.java | 2 +- .../parallel/ParallelIndexSupervisorTask.java | 2 +- .../indexing/common/task/IndexTaskTest.java | 2 +- ...stractParallelIndexSupervisorTaskTest.java | 2 +- .../ParallelIndexSupervisorTaskKillTest.java | 2 +- ...rallelIndexSupervisorTaskResourceTest.java | 2 +- .../druid/segment/indexing/BatchIOConfig.java | 2 +- 40 files changed, 98 insertions(+), 82 deletions(-) rename core/src/main/java/org/apache/druid/data/input/{ObjectSource.java => InputEntity.java} (87%) rename core/src/main/java/org/apache/druid/data/input/{ObjectReader.java => InputEntityReader.java} (72%) rename core/src/main/java/org/apache/druid/data/input/{impl => }/InputFormat.java (77%) rename core/src/main/java/org/apache/druid/data/input/impl/{ObjectIteratingReader.java => InputEntityIteratingReader.java} (81%) rename core/src/test/java/org/apache/druid/data/input/impl/{ObjectIteratingReaderTest.java => InputEntityIteratingReaderTest.java} (96%) diff --git a/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java b/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java index 1ac69cf86bbc..e52f9da36f9e 100644 --- a/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java @@ -20,7 +20,6 @@ package org.apache.druid.data.input; import com.google.common.base.Preconditions; -import org.apache.druid.data.input.impl.InputFormat; import javax.annotation.Nullable; import java.io.File; diff --git a/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java b/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java index 5bfc5f85d565..d63ac3b69631 100644 --- a/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java +++ b/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java @@ -30,6 +30,8 @@ * {@link FiniteFirehoseFactory} designed for batch processing. Its implementations assume that the amount of inputs is * limited. * + * This class is deprecated in favor of {@link InputSource}. + * * @param parser type * @param input split type */ diff --git a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java index 55806af86b0e..8ffc88581475 100644 --- a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java +++ b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java @@ -21,7 +21,6 @@ import com.google.common.base.Preconditions; import org.apache.druid.data.input.impl.FirehoseToInputSourceReaderAdaptor; -import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.SplittableInputSource; diff --git a/core/src/main/java/org/apache/druid/data/input/ObjectSource.java b/core/src/main/java/org/apache/druid/data/input/InputEntity.java similarity index 87% rename from core/src/main/java/org/apache/druid/data/input/ObjectSource.java rename to core/src/main/java/org/apache/druid/data/input/InputEntity.java index e83a67648ce9..62cc7fc2d2ac 100644 --- a/core/src/main/java/org/apache/druid/data/input/ObjectSource.java +++ b/core/src/main/java/org/apache/druid/data/input/InputEntity.java @@ -31,12 +31,12 @@ import java.io.InputStream; /** - * ObjectSource abstracts an object and knows how to read bytes from the given object. + * InputEntity abstracts an object and knows how to read bytes from the given object. */ @ExtensionPoint -public interface ObjectSource +public interface InputEntity { - Logger LOG = new Logger(ObjectSource.class); + Logger LOG = new Logger(InputEntity.class); int DEFAULT_FETCH_BUFFER_SIZE = 4 * 1024; // 4 KB int DEFAULT_MAX_FETCH_RETRY = 2; // 3 tries including the initial try @@ -64,7 +64,7 @@ interface CleanableFile extends Closeable * Fetches the object into the local storage. * This method might be preferred instead of {@link #open()}, for example * - * - {@link org.apache.druid.data.input.impl.InputFormat} requires expensive random access on remote storage. + * - {@link InputFormat} requires expensive random access on remote storage. * - Holding a connection until you consume the entire InputStream is expensive. * * @param temporaryDirectory to store temp data. This directory will be removed automatically once @@ -82,14 +82,11 @@ default CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) throws is, tempFile, fetchBuffer, - getRetryCondition(), + getFetchRetryCondition(), DEFAULT_MAX_FETCH_RETRY, StringUtils.format("Failed to fetch into [%s]", tempFile.getAbsolutePath()) ); } - catch (IOException e) { - throw new RuntimeException(e); - } return new CleanableFile() { @@ -110,7 +107,7 @@ public void close() } /** - * {@link #fetch} will retry during the fetch if it sees an exception mathing to the returned predicate. + * {@link #fetch} will retry during the fetch if it sees an exception matching to the returned predicate. */ - Predicate getRetryCondition(); + Predicate getFetchRetryCondition(); } diff --git a/core/src/main/java/org/apache/druid/data/input/ObjectReader.java b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java similarity index 72% rename from core/src/main/java/org/apache/druid/data/input/ObjectReader.java rename to core/src/main/java/org/apache/druid/data/input/InputEntityReader.java index 6512ecf9f996..d275d1e8f3b7 100644 --- a/core/src/main/java/org/apache/druid/data/input/ObjectReader.java +++ b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java @@ -26,15 +26,15 @@ import java.io.IOException; /** - * ObjectReader knows how to parse data into {@link InputRow}. - * This class is stateful and a new ObjectReader should be created per {@link ObjectSource}. + * InputEntityReader knows how to parse data into {@link InputRow}. + * This class is stateful and a new InputEntityReader should be created per {@link InputEntity}. * * @see TextReader for text format readers */ @ExtensionPoint -public interface ObjectReader +public interface InputEntityReader { - CloseableIterator read(ObjectSource source, File temporaryDirectory) throws IOException; + CloseableIterator read(InputEntity source, File temporaryDirectory) throws IOException; - CloseableIterator sample(ObjectSource source, File temporaryDirectory) throws IOException; + CloseableIterator sample(InputEntity source, File temporaryDirectory) throws IOException; } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputFormat.java b/core/src/main/java/org/apache/druid/data/input/InputFormat.java similarity index 77% rename from core/src/main/java/org/apache/druid/data/input/impl/InputFormat.java rename to core/src/main/java/org/apache/druid/data/input/InputFormat.java index e4170f89bc8d..a638e0b5eb5d 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/InputFormat.java @@ -17,20 +17,22 @@ * under the License. */ -package org.apache.druid.data.input.impl; +package org.apache.druid.data.input; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonSubTypes.Type; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.data.input.InputRowSchema; -import org.apache.druid.data.input.ObjectReader; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.NestedInputFormat; +import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.guice.annotations.ExtensionPoint; /** * InputFormat abstracts the file format of input data. - * It creates a {@link ObjectReader} to read data and parse it into {@link org.apache.druid.data.input.InputRow}. - * The created SplitReader is used by {@link org.apache.druid.data.input.InputSourceReader}. + * It creates a {@link InputEntityReader} to read data and parse it into {@link InputRow}. + * The created SplitReader is used by {@link InputSourceReader}. * * @see NestedInputFormat for nested input formats such as JSON. */ @@ -43,7 +45,7 @@ public interface InputFormat { /** - * Trait to indicate that a file can be split into multiple {@link org.apache.druid.data.input.InputSplit}s. + * Trait to indicate that a file can be split into multiple {@link InputSplit}s. * * This method is not being used anywhere for now, but should be considered * in {@link SplittableInputSource#createSplits}. @@ -51,5 +53,5 @@ public interface InputFormat @JsonIgnore boolean isSplittable(); - ObjectReader createReader(InputRowSchema inputRowSchema); + InputEntityReader createReader(InputRowSchema inputRowSchema); } diff --git a/core/src/main/java/org/apache/druid/data/input/InputSource.java b/core/src/main/java/org/apache/druid/data/input/InputSource.java index 346b7064a920..4ce64e3f0b88 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSource.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes.Type; import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.data.input.impl.HttpInputSource; -import org.apache.druid.data.input.impl.InputFormat; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.guice.annotations.ExtensionPoint; @@ -31,8 +30,8 @@ import java.io.File; /** - * InputSource abstracts the storage system where input data is stored. - * It creates an {@link InputSourceReader} to read data from the given input source. + * InputSource abstracts the storage system where input data is stored. It creates an {@link InputSourceReader} + * to read data from the given input source. * The most common use case would be: * *
{@code
@@ -59,7 +58,11 @@ public interface InputSource
   boolean isSplittable();
 
   /**
-   * Returns true if this inputSource supports different {@link InputFormat}s.
+   * Returns true if this inputSource supports different {@link InputFormat}s. Some inputSources such as
+   * {@link LocalInputSource} can store files of any format. These storage types require an {@link InputFormat}
+   * to be passed so that {@link InputSourceReader} can parse data properly. However, some storage types have
+   * a fixed format. For example, druid inputSource always reads segments. These inputSources should return false for
+   * this method.
    */
   boolean needsFormat();
 
diff --git a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java
index 809b8b1ecb5a..2c7b32cb00da 100644
--- a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java
+++ b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java
@@ -19,7 +19,7 @@
 
 package org.apache.druid.data.input;
 
-import org.apache.druid.data.input.impl.ObjectIteratingReader;
+import org.apache.druid.data.input.impl.InputEntityIteratingReader;
 import org.apache.druid.guice.annotations.ExtensionPoint;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
 
@@ -29,9 +29,10 @@
  * InputSourceReader reads data from {@link InputSource} and returns a {@link CloseableIterator} of
  * {@link InputRow}. See {@link InputSource} for an example usage.
  *
- * Implementations of this class can use {@link ObjectSource} and {@link ObjectReader}.
+ * Implementations of this class can use {@link InputEntity} and {@link InputEntityReader}. {@link InputFormat}
+ * can be useful to understand how to create an InputEntityReader.
  *
- * See {@link ObjectIteratingReader} as an example.
+ * See {@link InputEntityIteratingReader} as an example.
  */
 @ExtensionPoint
 public interface InputSourceReader
diff --git a/core/src/main/java/org/apache/druid/data/input/TextReader.java b/core/src/main/java/org/apache/druid/data/input/TextReader.java
index 9571eda010dc..b1b288737a44 100644
--- a/core/src/main/java/org/apache/druid/data/input/TextReader.java
+++ b/core/src/main/java/org/apache/druid/data/input/TextReader.java
@@ -29,9 +29,9 @@
 import java.io.InputStreamReader;
 
 /**
- * Abstract {@link ObjectReader} for text format readers such as CSV or JSON.
+ * Abstract {@link InputEntityReader} for text format readers such as CSV or JSON.
  */
-public abstract class TextReader implements ObjectReader
+public abstract class TextReader implements InputEntityReader
 {
   private final InputRowSchema inputRowSchema;
 
@@ -46,7 +46,7 @@ public InputRowSchema getInputRowSchema()
   }
 
   @Override
-  public CloseableIterator read(ObjectSource source, File temporaryDirectory) throws IOException
+  public CloseableIterator read(InputEntity source, File temporaryDirectory) throws IOException
   {
     return lineIterator(source).map(line -> {
       try {
@@ -59,7 +59,7 @@ public CloseableIterator read(ObjectSource source, File temporaryDirec
   }
 
   @Override
-  public CloseableIterator sample(ObjectSource source, File temporaryDirectory)
+  public CloseableIterator sample(InputEntity source, File temporaryDirectory)
       throws IOException
   {
     return lineIterator(source).map(line -> {
@@ -75,7 +75,7 @@ public CloseableIterator sample(ObjectSource source, File te
     });
   }
 
-  private CloseableIterator lineIterator(ObjectSource source) throws IOException
+  private CloseableIterator lineIterator(InputEntity source) throws IOException
   {
     final LineIterator delegate = new LineIterator(
         new InputStreamReader(source.open(), StringUtils.UTF8_STRING)
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java b/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java
index 787969be2f48..6c8cc0edbaab 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java
@@ -21,13 +21,13 @@
 
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
-import org.apache.druid.data.input.ObjectSource;
+import org.apache.druid.data.input.InputEntity;
 import org.apache.druid.io.ByteBufferInputStream;
 
 import java.io.InputStream;
 import java.nio.ByteBuffer;
 
-public class ByteSource implements ObjectSource
+public class ByteSource implements InputEntity
 {
   private final ByteBuffer buffer;
 
@@ -54,7 +54,7 @@ public InputStream open()
   }
 
   @Override
-  public Predicate getRetryCondition()
+  public Predicate getFetchRetryCondition()
   {
     return Predicates.alwaysFalse();
   }
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java
index 24f11e3e08d0..af5aeadf2cad 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java
@@ -22,6 +22,7 @@
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.java.util.common.parsers.CSVParser;
 import org.apache.druid.java.util.common.parsers.Parser;
 
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java
index 4760edfb836d..a9ce6c13cc6f 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java
@@ -24,8 +24,9 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRowSchema;
-import org.apache.druid.data.input.ObjectReader;
+import org.apache.druid.data.input.InputEntityReader;
 import org.apache.druid.indexer.Checks;
 import org.apache.druid.indexer.Property;
 
@@ -116,7 +117,7 @@ public boolean isSplittable()
   }
 
   @Override
-  public ObjectReader createReader(InputRowSchema inputRowSchema)
+  public InputEntityReader createReader(InputRowSchema inputRowSchema)
   {
     return new CsvReader(inputRowSchema, listDelimiter, columns, findColumnsFromHeader, skipHeaderRows);
   }
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java b/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java
index 9a94689031e7..0d1299781d24 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java
@@ -21,7 +21,7 @@
 
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
-import org.apache.druid.data.input.ObjectSource;
+import org.apache.druid.data.input.InputEntity;
 import org.apache.druid.utils.CompressionUtils;
 
 import java.io.File;
@@ -29,7 +29,7 @@
 import java.io.IOException;
 import java.io.InputStream;
 
-public class FileSource implements ObjectSource
+public class FileSource implements InputEntity
 {
   private final File file;
 
@@ -70,7 +70,7 @@ public InputStream open() throws IOException
   }
 
   @Override
-  public Predicate getRetryCondition()
+  public Predicate getFetchRetryCondition()
   {
     return Predicates.alwaysFalse();
   }
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java
index 3109e7cf2e0b..7ed909ef9bfe 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java
@@ -23,6 +23,7 @@
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 import org.apache.druid.data.input.AbstractInputSource;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.InputSourceReader;
 import org.apache.druid.data.input.InputSplit;
@@ -107,7 +108,7 @@ protected InputSourceReader formattableReader(
       @Nullable File temporaryDirectory
   )
   {
-    return new ObjectIteratingReader<>(
+    return new InputEntityIteratingReader<>(
         inputRowSchema,
         inputFormat,
         createSplits(inputFormat, null).map(split -> new HttpSource(
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java
index 00230a42eef8..c332ef466d3e 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java
@@ -21,7 +21,7 @@
 
 import com.google.common.base.Predicate;
 import com.google.common.base.Strings;
-import org.apache.druid.data.input.ObjectSource;
+import org.apache.druid.data.input.InputEntity;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.metadata.PasswordProvider;
 import org.apache.druid.utils.CompressionUtils;
@@ -33,7 +33,7 @@
 import java.net.URLConnection;
 import java.util.Base64;
 
-public class HttpSource implements ObjectSource
+public class HttpSource implements InputEntity
 {
   private final URI split;
   @Nullable
@@ -68,7 +68,7 @@ public InputStream open() throws IOException
   }
 
   @Override
-  public Predicate getRetryCondition()
+  public Predicate getFetchRetryCondition()
   {
     return t -> t instanceof IOException;
   }
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java
similarity index 81%
rename from core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java
rename to core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java
index 0828636c9f2a..c1acd8f3cf18 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/ObjectIteratingReader.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java
@@ -19,12 +19,13 @@
 
 package org.apache.druid.data.input.impl;
 
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRow;
 import org.apache.druid.data.input.InputRowPlusRaw;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.InputSourceReader;
-import org.apache.druid.data.input.ObjectReader;
-import org.apache.druid.data.input.ObjectSource;
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputEntity;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
 
 import java.io.File;
@@ -35,19 +36,20 @@
 import java.util.stream.Stream;
 
 /**
- * InputSourceReader iterating multiple {@link ObjectSource}s.
+ * InputSourceReader iterating multiple {@link InputEntity}s. This class could be used for
+ * most of {@link org.apache.druid.data.input.InputSource}s.
  */
-public class ObjectIteratingReader implements InputSourceReader
+public class InputEntityIteratingReader implements InputSourceReader
 {
   private final InputRowSchema inputRowSchema;
   private final InputFormat inputFormat;
-  private final Iterator> sourceIterator;
+  private final Iterator> sourceIterator;
   private final File temporaryDirectory;
 
-  ObjectIteratingReader(
+  InputEntityIteratingReader(
       InputRowSchema inputRowSchema,
       InputFormat inputFormat,
-      Stream> sourceStream,
+      Stream> sourceStream,
       File temporaryDirectory
   )
   {
@@ -83,7 +85,7 @@ public CloseableIterator sample()
     });
   }
 
-  private  CloseableIterator createIterator(Function> rowPopulator)
+  private  CloseableIterator createIterator(Function> rowPopulator)
   {
     return new CloseableIterator()
     {
@@ -118,8 +120,8 @@ private void updateRowIteratorIfNeeded()
           }
           if (sourceIterator.hasNext()) {
             // SplitSampler is stateful and so a new one should be created per split.
-            final ObjectReader objectReader = inputFormat.createReader(inputRowSchema);
-            rowIterator = rowPopulator.apply(objectReader);
+            final InputEntityReader inputEntityReader = inputFormat.createReader(inputRowSchema);
+            rowIterator = rowPopulator.apply(inputEntityReader);
           }
         }
       }
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JSONParseSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/JSONParseSpec.java
index d4147bd97c52..1dc73ce9e71d 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/JSONParseSpec.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/JSONParseSpec.java
@@ -23,6 +23,7 @@
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonParser.Feature;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.java.util.common.parsers.JSONPathParser;
 import org.apache.druid.java.util.common.parsers.JSONPathSpec;
 import org.apache.druid.java.util.common.parsers.Parser;
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java
index 9e7dbd88105d..717ee2f6fda5 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java
@@ -24,7 +24,7 @@
 import com.fasterxml.jackson.core.JsonParser.Feature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.druid.data.input.InputRowSchema;
-import org.apache.druid.data.input.ObjectReader;
+import org.apache.druid.data.input.InputEntityReader;
 import org.apache.druid.java.util.common.parsers.JSONPathSpec;
 
 import javax.annotation.Nullable;
@@ -66,7 +66,7 @@ public boolean isSplittable()
   }
 
   @Override
-  public ObjectReader createReader(InputRowSchema inputRowSchema)
+  public InputEntityReader createReader(InputRowSchema inputRowSchema)
   {
     return new JsonReader(inputRowSchema, getFlattenSpec(), objectMapper);
   }
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java
index 04dde8de170b..022e5d7ed842 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java
@@ -27,6 +27,7 @@
 import org.apache.commons.io.filefilter.TrueFileFilter;
 import org.apache.commons.io.filefilter.WildcardFileFilter;
 import org.apache.druid.data.input.AbstractInputSource;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.InputSourceReader;
 import org.apache.druid.data.input.InputSplit;
@@ -110,7 +111,7 @@ protected InputSourceReader formattableReader(
       @Nullable File temporaryDirectory
   )
   {
-    return new ObjectIteratingReader<>(
+    return new InputEntityIteratingReader<>(
         inputRowSchema,
         inputFormat,
         // reader() is supposed to be called in each task that creates segments.
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java
index 0ee8a867d892..90faea6ce704 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java
@@ -20,6 +20,7 @@
 package org.apache.druid.data.input.impl;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.java.util.common.parsers.JSONPathSpec;
 
 import javax.annotation.Nullable;
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/ParseSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/ParseSpec.java
index c8783c214f3e..7db92d2038a3 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/ParseSpec.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/ParseSpec.java
@@ -23,6 +23,7 @@
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import com.google.common.base.Preconditions;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.guice.annotations.ExtensionPoint;
 import org.apache.druid.guice.annotations.PublicApi;
 import org.apache.druid.java.util.common.parsers.Parser;
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java
index 06bf3b68ffb0..ed7de80d5469 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java
@@ -20,6 +20,7 @@
 package org.apache.druid.data.input.impl;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputSource;
 import org.apache.druid.data.input.InputSplit;
 import org.apache.druid.data.input.SplitHintSpec;
diff --git a/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java b/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java
index faeed4951e2e..c163de1ecb3d 100644
--- a/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java
+++ b/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java
@@ -22,7 +22,6 @@
 import com.google.common.collect.Iterables;
 import org.apache.druid.data.input.impl.CSVParseSpec;
 import org.apache.druid.data.input.impl.DimensionsSpec;
-import org.apache.druid.data.input.impl.InputFormat;
 import org.apache.druid.data.input.impl.StringInputRowParser;
 import org.apache.druid.data.input.impl.TimestampSpec;
 import org.apache.druid.java.util.common.DateTimes;
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java
index aa04d685860a..893687f3b7e0 100644
--- a/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java
+++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java
@@ -20,6 +20,7 @@
 package org.apache.druid.data.input.impl;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRowSchema;
 import org.junit.Assert;
 import org.junit.Rule;
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java
index 0041fb44f819..adf99c77cc61 100644
--- a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java
+++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java
@@ -23,7 +23,7 @@
 import com.google.common.collect.Iterables;
 import org.apache.druid.data.input.InputRow;
 import org.apache.druid.data.input.InputRowSchema;
-import org.apache.druid.data.input.ObjectReader;
+import org.apache.druid.data.input.InputEntityReader;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
@@ -117,7 +117,7 @@ public void testMultiValues() throws IOException
         )
     );
     final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), "|", true, 0);
-    final ObjectReader reader = format.createReader(INPUT_ROW_SCHEMA);
+    final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA);
     int numResults = 0;
     try (CloseableIterator iterator = reader.read(source, null)) {
       while (iterator.hasNext()) {
@@ -156,7 +156,7 @@ private ByteSource writeData(List lines) throws IOException
 
   private void assertResult(ByteSource source, CsvInputFormat format) throws IOException
   {
-    final ObjectReader reader = format.createReader(INPUT_ROW_SCHEMA);
+    final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA);
     int numResults = 0;
     try (CloseableIterator iterator = reader.read(source, null)) {
       while (iterator.hasNext()) {
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java
similarity index 96%
rename from core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java
rename to core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java
index ba9838627f7b..af5ba136c7f3 100644
--- a/core/src/test/java/org/apache/druid/data/input/impl/ObjectIteratingReaderTest.java
+++ b/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java
@@ -40,7 +40,7 @@
 import java.util.Collections;
 import java.util.List;
 
-public class ObjectIteratingReaderTest
+public class InputEntityIteratingReaderTest
 {
   @Rule
   public final TemporaryFolder temporaryFolder = new TemporaryFolder();
@@ -58,7 +58,7 @@ public void test() throws IOException
         writer.write(StringUtils.format("%d,%s,%d", 20190102 + i, "name_" + (i + 1), i + 1));
       }
     }
-    final ObjectIteratingReader firehose = new ObjectIteratingReader<>(
+    final InputEntityIteratingReader firehose = new InputEntityIteratingReader<>(
         new InputRowSchema(
             new TimestampSpec("time", "yyyyMMdd", null),
             new DimensionsSpec(
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JsonInputFormatTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JsonInputFormatTest.java
index 239ebc4b4ba7..5530b946864d 100644
--- a/core/src/test/java/org/apache/druid/data/input/impl/JsonInputFormatTest.java
+++ b/core/src/test/java/org/apache/druid/data/input/impl/JsonInputFormatTest.java
@@ -23,6 +23,7 @@
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec;
 import org.apache.druid.java.util.common.parsers.JSONPathFieldType;
 import org.apache.druid.java.util.common.parsers.JSONPathSpec;
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java
index c16fe260b515..b8eef05dfa70 100644
--- a/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java
+++ b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java
@@ -23,7 +23,7 @@
 import com.google.common.collect.Iterables;
 import org.apache.druid.data.input.InputRow;
 import org.apache.druid.data.input.InputRowSchema;
-import org.apache.druid.data.input.ObjectReader;
+import org.apache.druid.data.input.InputEntityReader;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
@@ -60,7 +60,7 @@ public void testParseRow() throws IOException
         StringUtils.toUtf8("{\"timestamp\":\"2019-01-01\",\"bar\":null,\"foo\":\"x\",\"baz\":4,\"o\":{\"mg\":1}}")
     );
 
-    final ObjectReader reader = format.createReader(
+    final InputEntityReader reader = format.createReader(
         new InputRowSchema(
             new TimestampSpec("timestamp", "iso", null),
             new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))),
@@ -107,7 +107,7 @@ public void testParseRowWithConditional() throws IOException
         StringUtils.toUtf8("{\"timestamp\":\"2019-01-01\",\"something_else\": {\"foo\": \"test\"}}")
     );
 
-    final ObjectReader reader = format.createReader(
+    final InputEntityReader reader = format.createReader(
         new InputRowSchema(
             new TimestampSpec("timestamp", "iso", null),
             new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))),
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java
index 6927f389b7d3..3bd5550f2c8c 100644
--- a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java
+++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java
@@ -19,8 +19,9 @@
 
 package org.apache.druid.data.input.impl;
 
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRowSchema;
-import org.apache.druid.data.input.ObjectReader;
+import org.apache.druid.data.input.InputEntityReader;
 
 public class NoopInputFormat implements InputFormat
 {
@@ -31,7 +32,7 @@ public boolean isSplittable()
   }
 
   @Override
-  public ObjectReader createReader(InputRowSchema inputRowSchema)
+  public InputEntityReader createReader(InputRowSchema inputRowSchema)
   {
     throw new UnsupportedOperationException();
   }
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java
index 49cb6e9ab00a..d1d18a827a07 100644
--- a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java
+++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.data.input.impl;
 
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.InputSource;
 import org.apache.druid.data.input.InputSourceReader;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
index 53953754f805..30b8fa404aa7 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
@@ -43,7 +43,7 @@
 import org.apache.druid.data.input.InputSource;
 import org.apache.druid.data.input.InputSourceReader;
 import org.apache.druid.data.input.Rows;
-import org.apache.druid.data.input.impl.InputFormat;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.impl.InputRowParser;
 import org.apache.druid.data.input.impl.ParseSpec;
 import org.apache.druid.hll.HyperLogLogCollector;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java
index 4162831494d2..73026ba65417 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java
@@ -24,7 +24,7 @@
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.InputSource;
 import org.apache.druid.data.input.InputSourceReader;
-import org.apache.druid.data.input.impl.InputFormat;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
 import org.apache.druid.indexer.partitions.PartitionsSpec;
 import org.apache.druid.indexing.common.stats.RowIngestionMeters;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java
index ef540bbfec0c..3eba9c6c9a25 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java
@@ -24,7 +24,7 @@
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import org.apache.druid.data.input.FirehoseFactory;
 import org.apache.druid.data.input.InputSource;
-import org.apache.druid.data.input.impl.InputFormat;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig;
 
 import javax.annotation.Nullable;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java
index a454ef8d9eeb..fda3e4d52bf9 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java
@@ -28,7 +28,7 @@
 import org.apache.druid.data.input.InputSource;
 import org.apache.druid.data.input.InputSplit;
 import org.apache.druid.data.input.SplitHintSpec;
-import org.apache.druid.data.input.impl.InputFormat;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.impl.InputRowParser;
 import org.apache.druid.data.input.impl.SplittableInputSource;
 import org.apache.druid.indexer.TaskState;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
index c7df03c1c25b..3009e651b247 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
@@ -30,7 +30,7 @@
 import org.apache.druid.client.indexing.IndexingServiceClient;
 import org.apache.druid.data.input.FiniteFirehoseFactory;
 import org.apache.druid.data.input.InputSource;
-import org.apache.druid.data.input.impl.InputFormat;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.impl.InputRowParser;
 import org.apache.druid.indexer.TaskState;
 import org.apache.druid.indexer.TaskStatus;
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java
index 6c2144b5eeab..03d3ff3b0da8 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java
@@ -29,7 +29,7 @@
 import org.apache.druid.data.input.impl.CSVParseSpec;
 import org.apache.druid.data.input.impl.DimensionsSpec;
 import org.apache.druid.data.input.impl.FloatDimensionSchema;
-import org.apache.druid.data.input.impl.InputFormat;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.impl.JSONParseSpec;
 import org.apache.druid.data.input.impl.LocalInputSource;
 import org.apache.druid.data.input.impl.LongDimensionSchema;
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
index 5e40b6b1a844..0c2d9ca2711e 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
@@ -30,7 +30,7 @@
 import org.apache.druid.client.indexing.TaskStatusResponse;
 import org.apache.druid.data.input.impl.CSVParseSpec;
 import org.apache.druid.data.input.impl.DimensionsSpec;
-import org.apache.druid.data.input.impl.InputFormat;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.impl.ParseSpec;
 import org.apache.druid.data.input.impl.TimestampSpec;
 import org.apache.druid.indexer.RunnerTaskState;
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java
index 5d0a0f20cdfe..39b6ca9caf80 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java
@@ -24,7 +24,7 @@
 import org.apache.druid.data.input.AbstractInputSource;
 import org.apache.druid.data.input.InputSplit;
 import org.apache.druid.data.input.SplitHintSpec;
-import org.apache.druid.data.input.impl.InputFormat;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.impl.NoopInputFormat;
 import org.apache.druid.data.input.impl.SplittableInputSource;
 import org.apache.druid.indexer.TaskState;
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java
index cb9bdda56b79..34f108c0eab7 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java
@@ -26,7 +26,7 @@
 import org.apache.druid.data.input.InputSplit;
 import org.apache.druid.data.input.MapBasedInputRow;
 import org.apache.druid.data.input.SplitHintSpec;
-import org.apache.druid.data.input.impl.InputFormat;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.impl.NoopInputFormat;
 import org.apache.druid.data.input.impl.SplittableInputSource;
 import org.apache.druid.indexer.RunnerTaskState;
diff --git a/server/src/main/java/org/apache/druid/segment/indexing/BatchIOConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/BatchIOConfig.java
index f53f22acdece..c6d523d11f29 100644
--- a/server/src/main/java/org/apache/druid/segment/indexing/BatchIOConfig.java
+++ b/server/src/main/java/org/apache/druid/segment/indexing/BatchIOConfig.java
@@ -20,7 +20,7 @@
 package org.apache.druid.segment.indexing;
 
 import org.apache.druid.data.input.InputSource;
-import org.apache.druid.data.input.impl.InputFormat;
+import org.apache.druid.data.input.InputFormat;
 
 /**
  * IOConfig for all batch tasks except compactionTask.

From 73812774c3d2af4f40fc84ae1918026eb5fccc0d Mon Sep 17 00:00:00 2001
From: Jihoon Son 
Date: Mon, 11 Nov 2019 16:04:05 -0800
Subject: [PATCH 19/29] fix import order and javadoc

---
 .../src/main/java/org/apache/druid/data/input/TextReader.java | 2 +-
 .../java/org/apache/druid/data/input/impl/CsvInputFormat.java | 2 +-
 .../druid/data/input/impl/InputEntityIteratingReader.java     | 4 ++--
 .../org/apache/druid/data/input/impl/JsonInputFormat.java     | 2 +-
 .../java/org/apache/druid/data/input/impl/CsvReaderTest.java  | 2 +-
 .../java/org/apache/druid/data/input/impl/JsonReaderTest.java | 2 +-
 .../org/apache/druid/data/input/impl/NoopInputFormat.java     | 2 +-
 7 files changed, 8 insertions(+), 8 deletions(-)

diff --git a/core/src/main/java/org/apache/druid/data/input/TextReader.java b/core/src/main/java/org/apache/druid/data/input/TextReader.java
index b1b288737a44..2096b9409dd9 100644
--- a/core/src/main/java/org/apache/druid/data/input/TextReader.java
+++ b/core/src/main/java/org/apache/druid/data/input/TextReader.java
@@ -128,7 +128,7 @@ public void close() throws IOException
   public abstract boolean needsToProcessHeaderLine();
 
   /**
-   * Processes a header line. This will be called as many times as {@link #getNumHeaderLinesToSkip()}.
+   * Processes a header line. This will be called if {@link #needsToProcessHeaderLine()} = true.
    */
   public abstract void processHeaderLine(String line) throws IOException;
 }
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java
index a9ce6c13cc6f..315580eea4dd 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java
@@ -24,9 +24,9 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import org.apache.druid.data.input.InputEntityReader;
 import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRowSchema;
-import org.apache.druid.data.input.InputEntityReader;
 import org.apache.druid.indexer.Checks;
 import org.apache.druid.indexer.Property;
 
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java
index c1acd8f3cf18..9f6c3235483e 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java
@@ -19,13 +19,13 @@
 
 package org.apache.druid.data.input.impl;
 
+import org.apache.druid.data.input.InputEntity;
+import org.apache.druid.data.input.InputEntityReader;
 import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRow;
 import org.apache.druid.data.input.InputRowPlusRaw;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.InputSourceReader;
-import org.apache.druid.data.input.InputEntityReader;
-import org.apache.druid.data.input.InputEntity;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
 
 import java.io.File;
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java
index 717ee2f6fda5..e335d9ba671a 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java
@@ -23,8 +23,8 @@
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonParser.Feature;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.java.util.common.parsers.JSONPathSpec;
 
 import javax.annotation.Nullable;
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java
index adf99c77cc61..e01d639af19a 100644
--- a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java
+++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java
@@ -21,9 +21,9 @@
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
+import org.apache.druid.data.input.InputEntityReader;
 import org.apache.druid.data.input.InputRow;
 import org.apache.druid.data.input.InputRowSchema;
-import org.apache.druid.data.input.InputEntityReader;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java
index b8eef05dfa70..a364e21d86b1 100644
--- a/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java
+++ b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java
@@ -21,9 +21,9 @@
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
+import org.apache.druid.data.input.InputEntityReader;
 import org.apache.druid.data.input.InputRow;
 import org.apache.druid.data.input.InputRowSchema;
-import org.apache.druid.data.input.InputEntityReader;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java
index 3bd5550f2c8c..8520671645b9 100644
--- a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java
+++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java
@@ -19,9 +19,9 @@
 
 package org.apache.druid.data.input.impl;
 
+import org.apache.druid.data.input.InputEntityReader;
 import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRowSchema;
-import org.apache.druid.data.input.InputEntityReader;
 
 public class NoopInputFormat implements InputFormat
 {

From 2a3b11460c7b05a724a73f0ba8d242ce608afe15 Mon Sep 17 00:00:00 2001
From: Jihoon Son 
Date: Mon, 11 Nov 2019 18:11:37 -0800
Subject: [PATCH 20/29] flatMap

---
 .../impl/InputEntityIteratingReader.java      | 56 +++----------------
 .../common/parsers/CloseableIterator.java     | 43 ++++++++++++++
 .../druid/segment/indexing/BatchIOConfig.java |  2 +-
 3 files changed, 52 insertions(+), 49 deletions(-)

diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java
index 9f6c3235483e..cb59c0a10fd9 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java
@@ -26,12 +26,12 @@
 import org.apache.druid.data.input.InputRowPlusRaw;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.InputSourceReader;
+import org.apache.druid.java.util.common.CloseableIterators;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
 
 import java.io.File;
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.NoSuchElementException;
 import java.util.function.Function;
 import java.util.stream.Stream;
 
@@ -87,52 +87,12 @@ public CloseableIterator sample()
 
   private  CloseableIterator createIterator(Function> rowPopulator)
   {
-    return new CloseableIterator()
-    {
-      CloseableIterator rowIterator = null;
-
-      @Override
-      public boolean hasNext()
-      {
-        updateRowIteratorIfNeeded();
-        return rowIterator != null && rowIterator.hasNext();
-      }
-
-      @Override
-      public R next()
-      {
-        if (!hasNext()) {
-          throw new NoSuchElementException();
-        }
-        return rowIterator.next();
-      }
-
-      private void updateRowIteratorIfNeeded()
-      {
-        if (rowIterator == null || !rowIterator.hasNext()) {
-          try {
-            if (rowIterator != null) {
-              rowIterator.close();
-            }
-          }
-          catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-          if (sourceIterator.hasNext()) {
-            // SplitSampler is stateful and so a new one should be created per split.
-            final InputEntityReader inputEntityReader = inputFormat.createReader(inputRowSchema);
-            rowIterator = rowPopulator.apply(inputEntityReader);
-          }
-        }
-      }
-
-      @Override
-      public void close() throws IOException
-      {
-        if (rowIterator != null) {
-          rowIterator.close();
-        }
-      }
-    };
+    return CloseableIterators
+        .withEmptyBaggage(sourceIterator)
+        .flatMap(entity -> {
+          // InputEntityReader is stateful and so a new one should be created per split.
+          final InputEntityReader inputEntityReader = inputFormat.createReader(inputRowSchema);
+          return rowPopulator.apply(inputEntityReader);
+        });
   }
 }
diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java
index 56b5816f4bb4..4cf5addb6428 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java
@@ -21,6 +21,7 @@
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 import java.util.function.Function;
@@ -57,4 +58,46 @@ public void close() throws IOException
       }
     };
   }
+
+  default  CloseableIterator flatMap(Function> function)
+  {
+    final CloseableIterator delegate = this;
+
+    return new CloseableIterator()
+    {
+      CloseableIterator iterator = null;
+
+      @Override
+      public boolean hasNext()
+      {
+        return (iterator != null && iterator.hasNext()) || delegate.hasNext();
+      }
+
+      @Override
+      public R next()
+      {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+        if (iterator == null || !iterator.hasNext()) {
+          if (iterator != null) {
+            try {
+              iterator.close();
+            }
+            catch (IOException e) {
+              throw new UncheckedIOException(e);
+            }
+          }
+          iterator = function.apply(delegate.next());
+        }
+        return iterator.next();
+      }
+
+      @Override
+      public void close() throws IOException
+      {
+        delegate.close();
+      }
+    };
+  }
 }
diff --git a/server/src/main/java/org/apache/druid/segment/indexing/BatchIOConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/BatchIOConfig.java
index c6d523d11f29..f964055bed23 100644
--- a/server/src/main/java/org/apache/druid/segment/indexing/BatchIOConfig.java
+++ b/server/src/main/java/org/apache/druid/segment/indexing/BatchIOConfig.java
@@ -19,8 +19,8 @@
 
 package org.apache.druid.segment.indexing;
 
-import org.apache.druid.data.input.InputSource;
 import org.apache.druid.data.input.InputFormat;
+import org.apache.druid.data.input.InputSource;
 
 /**
  * IOConfig for all batch tasks except compactionTask.

From 355777ca8c5c87ffce3a171c16bd51223bf24e2a Mon Sep 17 00:00:00 2001
From: Jihoon Son 
Date: Wed, 13 Nov 2019 13:48:28 -0800
Subject: [PATCH 21/29] sampleLine

---
 .../druid/data/input/AbstractInputSource.java |  2 +-
 .../org/apache/druid/data/input/Firehose.java |  6 +-
 .../druid/data/input/InputEntityReader.java   |  2 -
 .../druid/data/input/InputEntitySampler.java  | 36 ++++++++++
 .../apache/druid/data/input/InputFormat.java  |  2 +
 ...PlusRaw.java => InputRowListPlusJson.java} | 72 ++++++++++++++-----
 .../druid/data/input/InputSourceReader.java   |  2 +-
 .../apache/druid/data/input/TextReader.java   | 39 +++++++---
 .../druid/data/input/impl/CSVParseSpec.java   | 11 ---
 .../druid/data/input/impl/CsvInputFormat.java | 21 +++++-
 .../druid/data/input/impl/CsvReader.java      | 39 +++++++---
 .../input/impl/FileIteratingFirehose.java     |  8 +--
 .../FirehoseToInputSourceReaderAdaptor.java   |  8 +--
 .../data/input/impl/HttpInputSource.java      |  2 +-
 .../impl/InputEntityIteratingReader.java      | 25 ++++---
 .../data/input/impl/JsonInputFormat.java      |  7 ++
 .../druid/data/input/impl/JsonReader.java     | 42 ++++++++---
 .../data/input/impl/MapInputRowParser.java    |  5 +-
 .../data/input/impl/NoopInputFormat.java      |  7 ++
 .../druid/indexing/common/task/IndexTask.java |  2 +-
 .../common/task/InputSourceProcessor.java     |  2 +-
 .../batch/parallel/ParallelIndexIOConfig.java |  2 +-
 .../parallel/ParallelIndexPhaseRunner.java    |  2 +-
 .../parallel/ParallelIndexSupervisorTask.java |  2 +-
 .../overlord/sampler/FirehoseSampler.java     |  4 +-
 .../overlord/sampler/SamplerCache.java        |  8 +--
 .../overlord/sampler/SamplerConfig.java       |  3 +-
 .../SeekableStreamSamplerSpec.java            | 14 ++--
 .../druid/indexing/common/TestFirehose.java   |  8 +--
 .../indexing/common/task/IndexTaskTest.java   |  2 +-
 ...stractParallelIndexSupervisorTaskTest.java |  2 +-
 .../ParallelIndexSupervisorTaskKillTest.java  |  2 +-
 ...rallelIndexSupervisorTaskResourceTest.java |  2 +-
 .../indexing/overlord/TaskLifecycleTest.java  |  6 +-
 .../overlord/sampler/SamplerCacheTest.java    |  6 +-
 .../druid/segment/transform/Transformer.java  |  8 +--
 .../segment/transform/TransformingReader.java |  4 +-
 .../realtime/firehose/InlineFirehose.java     |  8 +--
 .../firehose/TimedShutoffFirehoseFactory.java |  4 +-
 .../realtime/firehose/InlineFirehoseTest.java |  8 +--
 40 files changed, 299 insertions(+), 136 deletions(-)
 create mode 100644 core/src/main/java/org/apache/druid/data/input/InputEntitySampler.java
 rename core/src/main/java/org/apache/druid/data/input/{InputRowPlusRaw.java => InputRowListPlusJson.java} (51%)

diff --git a/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java b/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java
index e52f9da36f9e..20936eab9a24 100644
--- a/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java
+++ b/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java
@@ -51,7 +51,7 @@ public InputSourceReader reader(
 
   protected InputSourceReader formattableReader(
       InputRowSchema inputRowSchema,
-      @Nullable InputFormat inputFormat,
+      InputFormat inputFormat,
       @Nullable File temporaryDirectory
   )
   {
diff --git a/core/src/main/java/org/apache/druid/data/input/Firehose.java b/core/src/main/java/org/apache/druid/data/input/Firehose.java
index c732d9dc322a..6957c4b2ad3b 100644
--- a/core/src/main/java/org/apache/druid/data/input/Firehose.java
+++ b/core/src/main/java/org/apache/druid/data/input/Firehose.java
@@ -74,13 +74,13 @@ public interface Firehose extends Closeable
    *
    * @return an InputRowPlusRaw which may contain any of: an InputRow, the raw data, or a ParseException
    */
-  default InputRowPlusRaw nextRowWithRaw() throws IOException
+  default InputRowListPlusJson nextRowWithRaw() throws IOException
   {
     try {
-      return InputRowPlusRaw.of(nextRow(), null);
+      return InputRowListPlusJson.of(nextRow(), null);
     }
     catch (ParseException e) {
-      return InputRowPlusRaw.of(null, e);
+      return InputRowListPlusJson.of((byte[]) null, e);
     }
   }
 
diff --git a/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java
index d275d1e8f3b7..410dc8607a76 100644
--- a/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java
+++ b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java
@@ -35,6 +35,4 @@
 public interface InputEntityReader
 {
   CloseableIterator read(InputEntity source, File temporaryDirectory) throws IOException;
-
-  CloseableIterator sample(InputEntity source, File temporaryDirectory) throws IOException;
 }
diff --git a/core/src/main/java/org/apache/druid/data/input/InputEntitySampler.java b/core/src/main/java/org/apache/druid/data/input/InputEntitySampler.java
new file mode 100644
index 000000000000..a2f6038ee536
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/data/input/InputEntitySampler.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.data.input;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+
+import java.io.File;
+import java.io.IOException;
+
+// TODO: maybe merge with reader back?
+public interface InputEntitySampler
+{
+  // TODO: should be clear to convert into map first and then json?
+  ObjectWriter SAMPLER_JSON_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter();
+
+  CloseableIterator sample(InputEntity source, File temporaryDirectory) throws IOException;
+}
diff --git a/core/src/main/java/org/apache/druid/data/input/InputFormat.java b/core/src/main/java/org/apache/druid/data/input/InputFormat.java
index a638e0b5eb5d..958079efa660 100644
--- a/core/src/main/java/org/apache/druid/data/input/InputFormat.java
+++ b/core/src/main/java/org/apache/druid/data/input/InputFormat.java
@@ -54,4 +54,6 @@ public interface InputFormat
   boolean isSplittable();
 
   InputEntityReader createReader(InputRowSchema inputRowSchema);
+
+  InputEntitySampler createSampler(InputRowSchema inputRowSchema);
 }
diff --git a/core/src/main/java/org/apache/druid/data/input/InputRowPlusRaw.java b/core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java
similarity index 51%
rename from core/src/main/java/org/apache/druid/data/input/InputRowPlusRaw.java
rename to core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java
index e5696c9a1069..3ea3df01fddf 100644
--- a/core/src/main/java/org/apache/druid/data/input/InputRowPlusRaw.java
+++ b/core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java
@@ -19,34 +19,78 @@
 
 package org.apache.druid.data.input;
 
+import com.google.common.collect.Iterables;
 import org.apache.druid.java.util.common.parsers.ParseException;
 
 import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.List;
 
-public class InputRowPlusRaw
+public class InputRowListPlusJson
 {
   @Nullable
-  private final InputRow inputRow;
+  private final List inputRows;
 
+  // TODO: remove
   @Nullable
   private final byte[] raw;
 
+  @Nullable
+  private final String jsonRaw;
+
   @Nullable
   private final ParseException parseException;
 
-  private InputRowPlusRaw(@Nullable InputRow inputRow, @Nullable byte[] raw, @Nullable ParseException parseException)
+  //TODO: remove
+  public static InputRowListPlusJson of(@Nullable InputRow inputRow, @Nullable byte[] raw)
+  {
+    return new InputRowListPlusJson(inputRow == null ? null : Collections.singletonList(inputRow), raw, null, null);
+  }
+
+  // TODO: rename
+  public static InputRowListPlusJson ofJson(@Nullable InputRow inputRow, @Nullable String jsonRaw)
+  {
+    return of(inputRow == null ? null : Collections.singletonList(inputRow), jsonRaw);
+  }
+
+  public static InputRowListPlusJson of(@Nullable List inputRows, @Nullable String jsonRaw)
   {
-    this.inputRow = inputRow;
+    return new InputRowListPlusJson(inputRows, null, jsonRaw, null);
+  }
+
+  // TODO: remove
+  public static InputRowListPlusJson of(@Nullable byte[] raw, @Nullable ParseException parseException)
+  {
+    return new InputRowListPlusJson(null, raw, null, parseException);
+  }
+
+  public static InputRowListPlusJson of(@Nullable String jsonRaw, @Nullable ParseException parseException)
+  {
+    return new InputRowListPlusJson(null, null, jsonRaw, parseException);
+  }
+
+  // TODO: remove byte[]
+  private InputRowListPlusJson(@Nullable List inputRows, @Nullable byte[] raw, @Nullable String jsonRaw, @Nullable ParseException parseException)
+  {
+    this.inputRows = inputRows;
     this.raw = raw;
+    this.jsonRaw = jsonRaw;
     this.parseException = parseException;
   }
 
   @Nullable
   public InputRow getInputRow()
   {
-    return inputRow;
+    return Iterables.getOnlyElement(inputRows);
+  }
+
+  @Nullable
+  public List getInputRows()
+  {
+    return inputRows;
   }
 
+  // TODO: remove
   /**
    * The raw, unparsed event (as opposed to an {@link InputRow} which is the output of a parser). The interface default
    * for {@link Firehose#nextRowWithRaw()} sets this to null, so this will only be non-null if nextRowWithRaw() is
@@ -62,23 +106,19 @@ public byte[] getRaw()
   }
 
   @Nullable
-  public ParseException getParseException()
+  public String getJsonRaw()
   {
-    return parseException;
+    return jsonRaw;
   }
 
-  public boolean isEmpty()
-  {
-    return inputRow == null && raw == null && parseException == null;
-  }
-
-  public static InputRowPlusRaw of(@Nullable InputRow inputRow, @Nullable byte[] raw)
+  @Nullable
+  public ParseException getParseException()
   {
-    return new InputRowPlusRaw(inputRow, raw, null);
+    return parseException;
   }
 
-  public static InputRowPlusRaw of(@Nullable byte[] raw, @Nullable ParseException parseException)
+  public boolean isEmpty()
   {
-    return new InputRowPlusRaw(null, raw, parseException);
+    return (inputRows == null || inputRows.isEmpty()) && raw == null && jsonRaw == null && parseException == null;
   }
 }
diff --git a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java
index 2c7b32cb00da..8086a75a9fa1 100644
--- a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java
+++ b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java
@@ -39,5 +39,5 @@ public interface InputSourceReader
 {
   CloseableIterator read() throws IOException;
 
-  CloseableIterator sample() throws IOException;
+  CloseableIterator sample() throws IOException;
 }
diff --git a/core/src/main/java/org/apache/druid/data/input/TextReader.java b/core/src/main/java/org/apache/druid/data/input/TextReader.java
index 2096b9409dd9..1b659a80f781 100644
--- a/core/src/main/java/org/apache/druid/data/input/TextReader.java
+++ b/core/src/main/java/org/apache/druid/data/input/TextReader.java
@@ -20,6 +20,7 @@
 package org.apache.druid.data.input;
 
 import org.apache.commons.io.LineIterator;
+import org.apache.druid.java.util.common.CloseableIterators;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
 import org.apache.druid.java.util.common.parsers.ParseException;
@@ -27,11 +28,12 @@
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.util.List;
 
 /**
  * Abstract {@link InputEntityReader} for text format readers such as CSV or JSON.
  */
-public abstract class TextReader implements InputEntityReader
+public abstract class TextReader implements InputEntityReader, InputEntitySampler
 {
   private final InputRowSchema inputRowSchema;
 
@@ -48,9 +50,15 @@ public InputRowSchema getInputRowSchema()
   @Override
   public CloseableIterator read(InputEntity source, File temporaryDirectory) throws IOException
   {
-    return lineIterator(source).map(line -> {
+    return lineIterator(source).flatMap(line -> {
       try {
-        return readLine(line);
+        // since readLine() returns a list, the below line always iterates over the list,
+        // which means it calls Iterator.hasNext() and Iterator.next() at least once per line.
+        // This could be unnecessary if the line wouldn't be exploded into multiple rows.
+        // If this line turned out to be a performance bottleneck, perhaps readLine() interface might not be a good
+        // idea. Subclasses could implement read() with some duplicate codes to avoid unnecessary iteration on
+        // a singleton list.
+        return CloseableIterators.withEmptyBaggage(readLine(line).iterator());
       }
       catch (IOException e) {
         throw new ParseException(e, "Unable to parse row [%s]", line);
@@ -59,15 +67,12 @@ public CloseableIterator read(InputEntity source, File temporaryDirect
   }
 
   @Override
-  public CloseableIterator sample(InputEntity source, File temporaryDirectory)
+  public CloseableIterator sample(InputEntity source, File temporaryDirectory)
       throws IOException
   {
     return lineIterator(source).map(line -> {
       try {
-        return InputRowPlusRaw.of(readLine(line), StringUtils.toUtf8(line));
-      }
-      catch (ParseException e) {
-        return InputRowPlusRaw.of(StringUtils.toUtf8(line), e);
+        return sampleLine(line);
       }
       catch (IOException e) {
         throw new RuntimeException(e);
@@ -111,9 +116,23 @@ public void close() throws IOException
   }
 
   /**
-   * Parses the given line into {@link InputRow}.
+   * Parses the given line into a list of {@link InputRow}s. Note that some file formats can explode a single line of
+   * input into multiple inputRows.
+   *
+   * This method will be called after {@link #getNumHeaderLinesToSkip()} and {@link #processHeaderLine}.
+   */
+  public abstract List readLine(String line) throws IOException, ParseException;
+
+  /**
+   * TODO
+   *
+   * Should handle {@link ParseException} properly.
+   *
+   * @param line
+   * @return
+   * @throws IOException
    */
-  public abstract InputRow readLine(String line) throws IOException, ParseException;
+  public abstract InputRowListPlusJson sampleLine(String line) throws IOException;
 
   /**
    * Returns the number of header lines to skip.
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java
index af5aeadf2cad..5a2c63673a66 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java
@@ -68,17 +68,6 @@ public CSVParseSpec(
     }
   }
 
-  @Deprecated
-  public CSVParseSpec(
-      TimestampSpec timestampSpec,
-      DimensionsSpec dimensionsSpec,
-      String listDelimiter,
-      List columns
-  )
-  {
-    this(timestampSpec, dimensionsSpec, listDelimiter, columns, false, 0);
-  }
-
   @JsonProperty
   public String getListDelimiter()
   {
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java
index 315580eea4dd..de4c296209dd 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java
@@ -25,6 +25,7 @@
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputEntitySampler;
 import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.indexer.Checks;
@@ -119,7 +120,25 @@ public boolean isSplittable()
   @Override
   public InputEntityReader createReader(InputRowSchema inputRowSchema)
   {
-    return new CsvReader(inputRowSchema, listDelimiter, columns, findColumnsFromHeader, skipHeaderRows);
+    return new CsvReader(
+        inputRowSchema,
+        listDelimiter,
+        columns,
+        findColumnsFromHeader,
+        skipHeaderRows
+    );
+  }
+
+  @Override
+  public InputEntitySampler createSampler(InputRowSchema inputRowSchema)
+  {
+    return new CsvReader(
+        inputRowSchema,
+        listDelimiter,
+        columns,
+        findColumnsFromHeader,
+        skipHeaderRows
+    );
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java
index 644a7e0cdf6d..a5f02cfc4362 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java
@@ -26,6 +26,7 @@
 import com.google.common.collect.Iterables;
 import com.opencsv.CSVParser;
 import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowListPlusJson;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.TextReader;
 import org.apache.druid.java.util.common.ISE;
@@ -38,6 +39,7 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -52,7 +54,7 @@ public class CsvReader extends TextReader
 
   CsvReader(
       InputRowSchema inputRowSchema,
-      String listDelimiter,
+      @Nullable String listDelimiter,
       @Nullable List columns,
       boolean findColumnsFromHeader,
       int skipHeaderRows
@@ -80,18 +82,39 @@ public class CsvReader extends TextReader
   }
 
   @Override
-  public InputRow readLine(String line) throws IOException, ParseException
+  public List readLine(String line) throws IOException, ParseException
+  {
+    final Map zipped = parseLine(line);
+    return Collections.singletonList(
+        MapInputRowParser.parse(
+            getInputRowSchema().getTimestampSpec(),
+            getInputRowSchema().getDimensionsSpec(),
+            zipped
+        )
+    );
+  }
+
+  @Override
+  public InputRowListPlusJson sampleLine(String line) throws IOException
+  {
+    final Map zipped = parseLine(line);
+    return InputRowListPlusJson.ofJson(
+        MapInputRowParser.parse(
+            getInputRowSchema().getTimestampSpec(),
+            getInputRowSchema().getDimensionsSpec(),
+            zipped
+        ),
+        SAMPLER_JSON_WRITER.writeValueAsString(zipped)
+    );
+  }
+
+  private Map parseLine(String line) throws IOException
   {
     final String[] parsed = parser.parseLine(line);
-    final Map zipped = Utils.zipMapPartial(
+    return Utils.zipMapPartial(
         Preconditions.checkNotNull(columns, "columns"),
         Iterables.transform(Arrays.asList(parsed), multiValueFunction)
     );
-    return MapInputRowParser.parse(
-        getInputRowSchema().getTimestampSpec(),
-        getInputRowSchema().getDimensionsSpec(),
-        zipped
-    );
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java b/core/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java
index 9c167b7bbc62..591e8f5ab4ac 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java
@@ -22,7 +22,7 @@
 import org.apache.commons.io.LineIterator;
 import org.apache.druid.data.input.Firehose;
 import org.apache.druid.data.input.InputRow;
-import org.apache.druid.data.input.InputRowPlusRaw;
+import org.apache.druid.data.input.InputRowListPlusJson;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.parsers.ParseException;
 
@@ -82,7 +82,7 @@ public InputRow nextRow() throws IOException
   }
 
   @Override
-  public InputRowPlusRaw nextRowWithRaw() throws IOException
+  public InputRowListPlusJson nextRowWithRaw() throws IOException
   {
     if (!hasMore()) {
       throw new NoSuchElementException();
@@ -90,10 +90,10 @@ public InputRowPlusRaw nextRowWithRaw() throws IOException
 
     String raw = lineIterator.next();
     try {
-      return InputRowPlusRaw.of(parser.parse(raw), StringUtils.toUtf8(raw));
+      return InputRowListPlusJson.of(parser.parse(raw), StringUtils.toUtf8(raw));
     }
     catch (ParseException e) {
-      return InputRowPlusRaw.of(StringUtils.toUtf8(raw), e);
+      return InputRowListPlusJson.of(StringUtils.toUtf8(raw), e);
     }
   }
 
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java
index afadec7de7ef..2fb3cde38d6f 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java
@@ -22,7 +22,7 @@
 import org.apache.druid.data.input.Firehose;
 import org.apache.druid.data.input.FirehoseFactory;
 import org.apache.druid.data.input.InputRow;
-import org.apache.druid.data.input.InputRowPlusRaw;
+import org.apache.druid.data.input.InputRowListPlusJson;
 import org.apache.druid.data.input.InputSourceReader;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
 
@@ -85,9 +85,9 @@ public void close() throws IOException
   }
 
   @Override
-  public CloseableIterator sample() throws IOException
+  public CloseableIterator sample() throws IOException
   {
-    return new CloseableIterator()
+    return new CloseableIterator()
     {
       final Firehose firehose = firehoseFactory.connectForSampler(inputRowParser, temporaryDirectory);
 
@@ -103,7 +103,7 @@ public boolean hasNext()
       }
 
       @Override
-      public InputRowPlusRaw next()
+      public InputRowListPlusJson next()
       {
         try {
           return firehose.nextRowWithRaw();
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java
index 7ed909ef9bfe..666626725fee 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java
@@ -104,7 +104,7 @@ public SplittableInputSource withSplit(InputSplit split)
   @Override
   protected InputSourceReader formattableReader(
       InputRowSchema inputRowSchema,
-      @Nullable InputFormat inputFormat,
+      InputFormat inputFormat,
       @Nullable File temporaryDirectory
   )
   {
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java
index cb59c0a10fd9..e20ce259b333 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java
@@ -21,9 +21,10 @@
 
 import org.apache.druid.data.input.InputEntity;
 import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputEntitySampler;
 import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRow;
-import org.apache.druid.data.input.InputRowPlusRaw;
+import org.apache.druid.data.input.InputRowListPlusJson;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.InputSourceReader;
 import org.apache.druid.java.util.common.CloseableIterators;
@@ -62,7 +63,9 @@ public class InputEntityIteratingReader implements InputSourceReader
   @Override
   public CloseableIterator read()
   {
-    return createIterator(reader -> {
+    return createIterator(entity -> {
+      // InputEntityReader is stateful and so a new one should be created per entity.
+      final InputEntityReader reader = inputFormat.createReader(inputRowSchema);
       try {
         return reader.read(sourceIterator.next(), temporaryDirectory);
       }
@@ -73,11 +76,13 @@ public CloseableIterator read()
   }
 
   @Override
-  public CloseableIterator sample()
+  public CloseableIterator sample()
   {
-    return createIterator(reader -> {
+    return createIterator(entity -> {
+      // InputEntitySampler is stateful and so a new one should be created per entity.
+      final InputEntitySampler sampler = inputFormat.createSampler(inputRowSchema);
       try {
-        return reader.sample(sourceIterator.next(), temporaryDirectory);
+        return sampler.sample(sourceIterator.next(), temporaryDirectory);
       }
       catch (IOException e) {
         throw new RuntimeException(e);
@@ -85,14 +90,8 @@ public CloseableIterator sample()
     });
   }
 
-  private  CloseableIterator createIterator(Function> rowPopulator)
+  private  CloseableIterator createIterator(Function, CloseableIterator> rowPopulator)
   {
-    return CloseableIterators
-        .withEmptyBaggage(sourceIterator)
-        .flatMap(entity -> {
-          // InputEntityReader is stateful and so a new one should be created per split.
-          final InputEntityReader inputEntityReader = inputFormat.createReader(inputRowSchema);
-          return rowPopulator.apply(inputEntityReader);
-        });
+    return CloseableIterators.withEmptyBaggage(sourceIterator).flatMap(rowPopulator);
   }
 }
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java
index e335d9ba671a..348f6c3574ed 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java
@@ -24,6 +24,7 @@
 import com.fasterxml.jackson.core.JsonParser.Feature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputEntitySampler;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.java.util.common.parsers.JSONPathSpec;
 
@@ -71,6 +72,12 @@ public InputEntityReader createReader(InputRowSchema inputRowSchema)
     return new JsonReader(inputRowSchema, getFlattenSpec(), objectMapper);
   }
 
+  @Override
+  public InputEntitySampler createSampler(InputRowSchema inputRowSchema)
+  {
+    return new JsonReader(inputRowSchema, getFlattenSpec(), objectMapper);
+  }
+
   @Override
   public boolean equals(Object o)
   {
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java
index 3a9f886c291f..c0daabc75585 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java
@@ -22,6 +22,7 @@
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowListPlusJson;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.TextReader;
 import org.apache.druid.java.util.common.parsers.JSONFlattenerMaker;
@@ -31,6 +32,8 @@
 import org.apache.druid.java.util.common.parsers.ParseException;
 
 import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 
 public class JsonReader extends TextReader
@@ -38,11 +41,7 @@ public class JsonReader extends TextReader
   private final ObjectFlattener flattener;
   private final ObjectMapper mapper;
 
-  public JsonReader(
-      InputRowSchema inputRowSchema,
-      JSONPathSpec flattenSpec,
-      ObjectMapper mapper
-  )
+  JsonReader(InputRowSchema inputRowSchema, JSONPathSpec flattenSpec, ObjectMapper mapper)
   {
     super(inputRowSchema);
     this.flattener = ObjectFlatteners.create(flattenSpec, new JSONFlattenerMaker());
@@ -50,17 +49,40 @@ public JsonReader(
   }
 
   @Override
-  public InputRow readLine(String line) throws IOException, ParseException
+  public List readLine(String line) throws IOException, ParseException
   {
     final JsonNode document = mapper.readValue(line, JsonNode.class);
     final Map flattened = flattener.flatten(document);
-    return MapInputRowParser.parse(
-        getInputRowSchema().getTimestampSpec(),
-        getInputRowSchema().getDimensionsSpec(),
-        flattened
+    return Collections.singletonList(
+        MapInputRowParser.parse(
+            getInputRowSchema().getTimestampSpec(),
+            getInputRowSchema().getDimensionsSpec(),
+            flattened
+        )
     );
   }
 
+  @Override
+  public InputRowListPlusJson sampleLine(String line) throws IOException
+  {
+    final JsonNode document = mapper.readValue(line, JsonNode.class);
+    final String rawJson = SAMPLER_JSON_WRITER.writeValueAsString(document);
+    final Map flattened = flattener.flatten(document);
+    try {
+      return InputRowListPlusJson.ofJson(
+          MapInputRowParser.parse(
+              getInputRowSchema().getTimestampSpec(),
+              getInputRowSchema().getDimensionsSpec(),
+              flattened
+          ),
+          rawJson
+      );
+    }
+    catch (ParseException e) {
+      return InputRowListPlusJson.of(rawJson, e);
+    }
+  }
+
   @Override
   public int getNumHeaderLinesToSkip()
   {
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java b/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java
index 9f605c383033..debf62a82dd9 100644
--- a/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java
+++ b/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java
@@ -65,17 +65,18 @@ public static InputRow parse(
       TimestampSpec timestampSpec,
       DimensionsSpec dimensionsSpec,
       Map theMap
-  )
+  ) throws ParseException
   {
     return parse(timestampSpec, dimensionsSpec.getDimensionNames(), dimensionsSpec.getDimensionExclusions(), theMap);
   }
 
+  // TODO: maybe prune columns with metrics + dimensions
   public static InputRow parse(
       TimestampSpec timestampSpec,
       List dimensions,
       Set dimensionExclusions,
       Map theMap
-  )
+  ) throws ParseException
   {
     final List dimensionsToUse;
     if (!dimensions.isEmpty()) {
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java
index 8520671645b9..6db7abdeaae4 100644
--- a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java
+++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java
@@ -20,6 +20,7 @@
 package org.apache.druid.data.input.impl;
 
 import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputEntitySampler;
 import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRowSchema;
 
@@ -37,6 +38,12 @@ public InputEntityReader createReader(InputRowSchema inputRowSchema)
     throw new UnsupportedOperationException();
   }
 
+  @Override
+  public InputEntitySampler createSampler(InputRowSchema inputRowSchema)
+  {
+    throw new UnsupportedOperationException();
+  }
+
   @Override
   public String toString()
   {
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
index 30b8fa404aa7..b3fecb19c979 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
@@ -38,12 +38,12 @@
 import org.apache.druid.data.input.FiniteFirehoseFactory;
 import org.apache.druid.data.input.FirehoseFactory;
 import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRow;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.InputSource;
 import org.apache.druid.data.input.InputSourceReader;
 import org.apache.druid.data.input.Rows;
-import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.impl.InputRowParser;
 import org.apache.druid.data.input.impl.ParseSpec;
 import org.apache.druid.hll.HyperLogLogCollector;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java
index 73026ba65417..65c4291438ae 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java
@@ -20,11 +20,11 @@
 package org.apache.druid.indexing.common.task;
 
 import com.google.common.base.Optional;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRow;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.InputSource;
 import org.apache.druid.data.input.InputSourceReader;
-import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
 import org.apache.druid.indexer.partitions.PartitionsSpec;
 import org.apache.druid.indexing.common.stats.RowIngestionMeters;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java
index 3eba9c6c9a25..b272dd876165 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java
@@ -23,8 +23,8 @@
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import org.apache.druid.data.input.FirehoseFactory;
-import org.apache.druid.data.input.InputSource;
 import org.apache.druid.data.input.InputFormat;
+import org.apache.druid.data.input.InputSource;
 import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig;
 
 import javax.annotation.Nullable;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java
index fda3e4d52bf9..7a92697b8eff 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java
@@ -25,10 +25,10 @@
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import org.apache.druid.client.indexing.IndexingServiceClient;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputSource;
 import org.apache.druid.data.input.InputSplit;
 import org.apache.druid.data.input.SplitHintSpec;
-import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.impl.InputRowParser;
 import org.apache.druid.data.input.impl.SplittableInputSource;
 import org.apache.druid.indexer.TaskState;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
index 3009e651b247..f821c2f09b3b 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
@@ -29,8 +29,8 @@
 import com.google.common.base.Throwables;
 import org.apache.druid.client.indexing.IndexingServiceClient;
 import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.InputSource;
 import org.apache.druid.data.input.InputFormat;
+import org.apache.druid.data.input.InputSource;
 import org.apache.druid.data.input.impl.InputRowParser;
 import org.apache.druid.indexer.TaskState;
 import org.apache.druid.indexer.TaskStatus;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/FirehoseSampler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/FirehoseSampler.java
index e0ae3ebe65e1..133b92b2b15e 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/FirehoseSampler.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/FirehoseSampler.java
@@ -30,7 +30,7 @@
 import org.apache.druid.data.input.Firehose;
 import org.apache.druid.data.input.FirehoseFactory;
 import org.apache.druid.data.input.InputRow;
-import org.apache.druid.data.input.InputRowPlusRaw;
+import org.apache.druid.data.input.InputRowListPlusJson;
 import org.apache.druid.data.input.Row;
 import org.apache.druid.data.input.impl.AbstractTextFilesFirehoseFactory;
 import org.apache.druid.data.input.impl.DimensionsSpec;
@@ -207,7 +207,7 @@ public SamplerResponse sample(FirehoseFactory firehoseFactory, DataSchema dataSc
       while (counter < responseRows.length && firehose.hasMore()) {
         String raw = null;
         try {
-          final InputRowPlusRaw row = firehose.nextRowWithRaw();
+          final InputRowListPlusJson row = firehose.nextRowWithRaw();
 
           if (row == null || row.isEmpty()) {
             continue;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerCache.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerCache.java
index dc7e04d71420..f3e4becefb5c 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerCache.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerCache.java
@@ -24,7 +24,7 @@
 import org.apache.druid.data.input.Firehose;
 import org.apache.druid.data.input.FirehoseFactory;
 import org.apache.druid.data.input.InputRow;
-import org.apache.druid.data.input.InputRowPlusRaw;
+import org.apache.druid.data.input.InputRowListPlusJson;
 import org.apache.druid.data.input.impl.InputRowParser;
 import org.apache.druid.data.input.impl.StringInputRowParser;
 import org.apache.druid.java.util.common.StringUtils;
@@ -154,7 +154,7 @@ public InputRow nextRow()
     }
 
     @Override
-    public InputRowPlusRaw nextRowWithRaw()
+    public InputRowListPlusJson nextRowWithRaw()
     {
       if (!hasMore()) {
         throw new NoSuchElementException();
@@ -164,10 +164,10 @@ public InputRowPlusRaw nextRowWithRaw()
 
       try {
         List rows = parser.parseBatch(ByteBuffer.wrap(raw));
-        return InputRowPlusRaw.of(rows.isEmpty() ? null : rows.get(0), raw);
+        return InputRowListPlusJson.of(rows.isEmpty() ? null : rows.get(0), raw);
       }
       catch (ParseException e) {
-        return InputRowPlusRaw.of(raw, e);
+        return InputRowListPlusJson.of(raw, e);
       }
     }
 
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java
index a7d6ca957a06..cde5fb8f55ec 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java
@@ -22,6 +22,7 @@
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
+import org.apache.druid.data.input.InputRowListPlusJson;
 
 public class SamplerConfig
 {
@@ -73,7 +74,7 @@ public int getNumRows()
    * better user experience for sources such as streams, where repeated calls to the sampler (which would happen as the
    * user tweaks data schema configurations) would otherwise return a different set of sampled data every time. For the
    * caching system to work, 1) the sampler must have access to the raw data (e.g. for {@link FirehoseSampler},
-   * {@link org.apache.druid.data.input.InputRowPlusRaw#getRaw()} must be non-null) and 2) the parser must be an
+   * {@link InputRowListPlusJson#getRaw()} must be non-null) and 2) the parser must be an
    * implementation of {@link org.apache.druid.data.input.ByteBufferInputRowParser} since the data is cached as a byte
    * array. If these conditions are not satisfied, the cache returns a miss and the sampler would read from source.
    * 

diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java index 119e516c1a37..b219d4baaf50 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java @@ -23,7 +23,7 @@ import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.indexing.overlord.sampler.FirehoseSampler; @@ -129,7 +129,7 @@ public boolean hasMore() @Override public InputRow nextRow() { - InputRowPlusRaw row = nextRowWithRaw(); + InputRowListPlusJson row = nextRowWithRaw(); if (row.getParseException() != null) { throw row.getParseException(); } @@ -138,21 +138,21 @@ public InputRow nextRow() } @Override - public InputRowPlusRaw nextRowWithRaw() + public InputRowListPlusJson nextRowWithRaw() { if (recordDataIterator == null || !recordDataIterator.hasNext()) { if (recordIterator == null || !recordIterator.hasNext()) { recordIterator = recordSupplier.poll(POLL_TIMEOUT_MS).iterator(); if (!recordIterator.hasNext()) { - return InputRowPlusRaw.of((InputRow) null, null); + return InputRowListPlusJson.of((InputRow) null, null); } } recordDataIterator = recordIterator.next().getData().iterator(); if (!recordDataIterator.hasNext()) { - return InputRowPlusRaw.of((InputRow) null, null); + return InputRowListPlusJson.of((InputRow) null, null); } } @@ -160,10 +160,10 @@ public InputRowPlusRaw nextRowWithRaw() try { List rows = parser.parseBatch(ByteBuffer.wrap(raw)); - return InputRowPlusRaw.of(rows.isEmpty() ? null : rows.get(0), raw); + return InputRowListPlusJson.of(rows.isEmpty() ? null : rows.get(0), raw); } catch (ParseException e) { - return InputRowPlusRaw.of(raw, e); + return InputRowListPlusJson.of(raw, e); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java index f55c854d0e2c..fea30f23a29a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java @@ -23,7 +23,7 @@ import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.AbstractTextFilesFirehoseFactory; import org.apache.druid.data.input.impl.InputRowParser; @@ -168,7 +168,7 @@ public InputRow nextRow() } @Override - public InputRowPlusRaw nextRowWithRaw() + public InputRowListPlusJson nextRowWithRaw() { Object next = queue.removeFirst().orElse(null); @@ -181,10 +181,10 @@ public InputRowPlusRaw nextRowWithRaw() if (row != null && row.getRaw(FAIL_DIM) != null) { throw new ParseException(FAIL_DIM); } - return InputRowPlusRaw.of(row, next != null ? StringUtils.toUtf8(next.toString()) : null); + return InputRowListPlusJson.of(row, next != null ? StringUtils.toUtf8(next.toString()) : null); } catch (ParseException e) { - return InputRowPlusRaw.of(next != null ? StringUtils.toUtf8(next.toString()) : null, e); + return InputRowListPlusJson.of(next != null ? StringUtils.toUtf8(next.toString()) : null, e); } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 03d3ff3b0da8..9d2a480d7908 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -26,10 +26,10 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import com.google.common.io.Files; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; -import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.LongDimensionSchema; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 0c2d9ca2711e..bf8da6ece522 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -28,9 +28,9 @@ import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.RunnerTaskState; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index 39b6ca9caf80..9c87b5d8970b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -22,9 +22,9 @@ import com.google.common.collect.Iterables; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.AbstractInputSource; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexer.TaskState; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index 34f108c0eab7..413e99a39a50 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -23,10 +23,10 @@ import com.google.common.base.Preconditions; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.AbstractInputSource; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexer.RunnerTaskState; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index b0047be7123c..740560e141d6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -37,7 +37,7 @@ import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.MapBasedInputRow; @@ -313,7 +313,7 @@ public InputRow next() } @Override - public CloseableIterator sample() + public CloseableIterator sample() { throw new UnsupportedOperationException(); } @@ -348,7 +348,7 @@ public CloseableIterator read() } @Override - public CloseableIterator sample() + public CloseableIterator sample() { throw new UnsupportedOperationException(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerCacheTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerCacheTest.java index ff53cf420b95..c1d82c98c6d3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerCacheTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerCacheTest.java @@ -26,7 +26,7 @@ import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.StringInputRowParser; @@ -91,7 +91,7 @@ public void testOneEntryNextRowWithRaw() throws IOException Assert.assertTrue(firehose1.hasMore()); - InputRowPlusRaw row = firehose1.nextRowWithRaw(); + InputRowListPlusJson row = firehose1.nextRowWithRaw(); Assert.assertArrayEquals(VALUE_1_1, row.getRaw()); Assert.assertEquals("The quick", row.getInputRow().getDimension("col").get(0)); row = firehose1.nextRowWithRaw(); @@ -144,7 +144,7 @@ public void testTwoEntriesNextRowWithRaw() throws IOException Assert.assertTrue(firehose1.hasMore()); - InputRowPlusRaw row = firehose1.nextRowWithRaw(); + InputRowListPlusJson row = firehose1.nextRowWithRaw(); Assert.assertArrayEquals(VALUE_1_1, row.getRaw()); Assert.assertEquals("The quick", row.getInputRow().getDimension("col").get(0)); row = firehose1.nextRowWithRaw(); diff --git a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java index c18e0ddcb54f..109195f38242 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java @@ -20,7 +20,7 @@ package org.apache.druid.segment.transform; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.Row; import org.apache.druid.data.input.Rows; import org.apache.druid.java.util.common.DateTimes; @@ -94,18 +94,18 @@ public InputRow transform(@Nullable final InputRow row) } @Nullable - public InputRowPlusRaw transform(@Nullable final InputRowPlusRaw row) + public InputRowListPlusJson transform(@Nullable final InputRowListPlusJson row) { if (row == null) { return null; } - final InputRowPlusRaw transformedRow; + final InputRowListPlusJson transformedRow; if (transforms.isEmpty()) { transformedRow = row; } else { - transformedRow = InputRowPlusRaw.of(new TransformedInputRow(row.getInputRow(), transforms), row.getRaw()); + transformedRow = InputRowListPlusJson.of(new TransformedInputRow(row.getInputRow(), transforms), row.getRaw()); } if (valueMatcher != null) { diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformingReader.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformingReader.java index 87f441534ca9..25944f6b6c27 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/TransformingReader.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformingReader.java @@ -20,7 +20,7 @@ package org.apache.druid.segment.transform; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -44,7 +44,7 @@ public CloseableIterator read() throws IOException } @Override - public CloseableIterator sample() throws IOException + public CloseableIterator sample() throws IOException { return delegate.sample().map(transformer::transform); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java index 0c93d25d652e..d8d85179ec03 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java @@ -23,7 +23,7 @@ import org.apache.commons.io.LineIterator; import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; @@ -73,14 +73,14 @@ private String nextRaw() } @Override - public InputRowPlusRaw nextRowWithRaw() + public InputRowListPlusJson nextRowWithRaw() { String raw = nextRaw(); try { - return InputRowPlusRaw.of(parser.parse(raw), StringUtils.toUtf8(raw)); + return InputRowListPlusJson.of(parser.parse(raw), StringUtils.toUtf8(raw)); } catch (ParseException e) { - return InputRowPlusRaw.of(StringUtils.toUtf8(raw), e); + return InputRowListPlusJson.of(StringUtils.toUtf8(raw), e); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java index 5a132c35d37f..321d216480c5 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java @@ -25,7 +25,7 @@ import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -120,7 +120,7 @@ public InputRow nextRow() throws IOException } @Override - public InputRowPlusRaw nextRowWithRaw() throws IOException + public InputRowListPlusJson nextRowWithRaw() throws IOException { return firehose.nextRowWithRaw(); } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseTest.java index f24cc8528ec7..d761566aa060 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseTest.java @@ -20,7 +20,7 @@ package org.apache.druid.segment.realtime.firehose; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowPlusRaw; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.StringInputRowParser; @@ -116,7 +116,7 @@ public void testNextRowWithRawParseable() { final String data = PARSEABLE; InlineFirehose target = create(data); - InputRowPlusRaw rowPlusRaw = target.nextRowWithRaw(); + InputRowListPlusJson rowPlusRaw = target.nextRowWithRaw(); InputRow row = rowPlusRaw.getInputRow(); assertRowValue(VALUE_0, row); @@ -132,7 +132,7 @@ public void testNextRowWithRawNotParseable() { final String data = NOT_PARSEABLE; InlineFirehose target = create(data); - InputRowPlusRaw rowPlusRaw = target.nextRowWithRaw(); + InputRowListPlusJson rowPlusRaw = target.nextRowWithRaw(); InputRow row = rowPlusRaw.getInputRow(); Assert.assertNull(row); @@ -185,7 +185,7 @@ public void testMultiline() assertRowValue(VALUE_0, row0); // Second line - InputRowPlusRaw rowPlusRaw = target.nextRowWithRaw(); + InputRowListPlusJson rowPlusRaw = target.nextRowWithRaw(); assertRowValue(VALUE_1, rowPlusRaw.getInputRow()); assertRawValue(LINE_1, rowPlusRaw.getRaw()); Assert.assertNull(rowPlusRaw.getParseException()); From e466ea9ee196a134d4b9c2e54ff5feffe7402a29 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 13 Nov 2019 16:48:32 -0800 Subject: [PATCH 22/29] Add IntermediateRowParsingReader --- .../druid/data/input/InputEntityReader.java | 11 ++ .../druid/data/input/InputEntitySampler.java | 36 ------- .../apache/druid/data/input/InputFormat.java | 2 - .../data/input/InputRowListPlusJson.java | 25 ++--- .../input/IntermediateRowParsingReader.java | 101 ++++++++++++++++++ .../apache/druid/data/input/TextReader.java | 50 +-------- .../druid/data/input/impl/CsvInputFormat.java | 13 --- .../druid/data/input/impl/CsvReader.java | 16 +-- .../impl/InputEntityIteratingReader.java | 5 +- .../data/input/impl/JsonInputFormat.java | 7 -- .../druid/data/input/impl/JsonReader.java | 23 +--- .../data/input/impl/MapInputRowParser.java | 1 - .../data/input/impl/NoopInputFormat.java | 7 -- 13 files changed, 133 insertions(+), 164 deletions(-) delete mode 100644 core/src/main/java/org/apache/druid/data/input/InputEntitySampler.java create mode 100644 core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java diff --git a/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java index 410dc8607a76..6b10728c3914 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java +++ b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java @@ -19,6 +19,8 @@ package org.apache.druid.data.input; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -34,5 +36,14 @@ @ExtensionPoint public interface InputEntityReader { + /** + * Default JSON writer for sampler. This writer can be used to create an {@link InputRowListPlusJson}. + * Note that this writer uses the default serializer of Jackson. You may want to create a custom writer + * to serialize your custom types. + */ + ObjectWriter DEFAULT_JSON_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter(); + CloseableIterator read(InputEntity source, File temporaryDirectory) throws IOException; + + CloseableIterator sample(InputEntity source, File temporaryDirectory) throws IOException; } diff --git a/core/src/main/java/org/apache/druid/data/input/InputEntitySampler.java b/core/src/main/java/org/apache/druid/data/input/InputEntitySampler.java deleted file mode 100644 index a2f6038ee536..000000000000 --- a/core/src/main/java/org/apache/druid/data/input/InputEntitySampler.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.data.input; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.ObjectWriter; -import org.apache.druid.java.util.common.parsers.CloseableIterator; - -import java.io.File; -import java.io.IOException; - -// TODO: maybe merge with reader back? -public interface InputEntitySampler -{ - // TODO: should be clear to convert into map first and then json? - ObjectWriter SAMPLER_JSON_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter(); - - CloseableIterator sample(InputEntity source, File temporaryDirectory) throws IOException; -} diff --git a/core/src/main/java/org/apache/druid/data/input/InputFormat.java b/core/src/main/java/org/apache/druid/data/input/InputFormat.java index 958079efa660..a638e0b5eb5d 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/InputFormat.java @@ -54,6 +54,4 @@ public interface InputFormat boolean isSplittable(); InputEntityReader createReader(InputRowSchema inputRowSchema); - - InputEntitySampler createSampler(InputRowSchema inputRowSchema); } diff --git a/core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java b/core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java index 3ea3df01fddf..27c42b3b06fa 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java +++ b/core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java @@ -31,34 +31,25 @@ public class InputRowListPlusJson @Nullable private final List inputRows; - // TODO: remove @Nullable private final byte[] raw; @Nullable - private final String jsonRaw; + private final String rawJson; @Nullable private final ParseException parseException; - //TODO: remove public static InputRowListPlusJson of(@Nullable InputRow inputRow, @Nullable byte[] raw) { return new InputRowListPlusJson(inputRow == null ? null : Collections.singletonList(inputRow), raw, null, null); } - // TODO: rename - public static InputRowListPlusJson ofJson(@Nullable InputRow inputRow, @Nullable String jsonRaw) - { - return of(inputRow == null ? null : Collections.singletonList(inputRow), jsonRaw); - } - public static InputRowListPlusJson of(@Nullable List inputRows, @Nullable String jsonRaw) { return new InputRowListPlusJson(inputRows, null, jsonRaw, null); } - // TODO: remove public static InputRowListPlusJson of(@Nullable byte[] raw, @Nullable ParseException parseException) { return new InputRowListPlusJson(null, raw, null, parseException); @@ -69,19 +60,18 @@ public static InputRowListPlusJson of(@Nullable String jsonRaw, @Nullable ParseE return new InputRowListPlusJson(null, null, jsonRaw, parseException); } - // TODO: remove byte[] - private InputRowListPlusJson(@Nullable List inputRows, @Nullable byte[] raw, @Nullable String jsonRaw, @Nullable ParseException parseException) + private InputRowListPlusJson(@Nullable List inputRows, @Nullable byte[] raw, @Nullable String rawJson, @Nullable ParseException parseException) { this.inputRows = inputRows; this.raw = raw; - this.jsonRaw = jsonRaw; + this.rawJson = rawJson; this.parseException = parseException; } @Nullable public InputRow getInputRow() { - return Iterables.getOnlyElement(inputRows); + return inputRows == null ? null : Iterables.getOnlyElement(inputRows); } @Nullable @@ -90,7 +80,6 @@ public List getInputRows() return inputRows; } - // TODO: remove /** * The raw, unparsed event (as opposed to an {@link InputRow} which is the output of a parser). The interface default * for {@link Firehose#nextRowWithRaw()} sets this to null, so this will only be non-null if nextRowWithRaw() is @@ -106,9 +95,9 @@ public byte[] getRaw() } @Nullable - public String getJsonRaw() + public String getRawJson() { - return jsonRaw; + return rawJson; } @Nullable @@ -119,6 +108,6 @@ public ParseException getParseException() public boolean isEmpty() { - return (inputRows == null || inputRows.isEmpty()) && raw == null && jsonRaw == null && parseException == null; + return (inputRows == null || inputRows.isEmpty()) && raw == null && rawJson == null && parseException == null; } } diff --git a/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java b/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java new file mode 100644 index 000000000000..0693831b9ef6 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.ParseException; + +import java.io.File; +import java.io.IOException; +import java.util.List; + +/** + * {@link InputEntityReader} that parses bytes into some intermediate rows first, and then parse them + * into {@link InputRow}s. For example, {@link org.apache.druid.data.input.impl.CsvReader} parses bytes + * into string lines, and then parses those lines into InputRows. + * + * @param type of intermediate row. For example, it can be {@link String} for text formats. + */ +public abstract class IntermediateRowParsingReader implements InputEntityReader +{ + @Override + public CloseableIterator read(InputEntity source, File temporaryDirectory) throws IOException + { + return intermediateRowIterator(source, temporaryDirectory).flatMap(row -> { + try { + // since parseInputRows() returns a list, the below line always iterates over the list, + // which means it calls Iterator.hasNext() and Iterator.next() at least once per row. + // This could be unnecessary if the row wouldn't be exploded into multiple inputRows. + // If this line turned out to be a performance bottleneck, perhaps parseInputRows() interface might not be a + // good idea. Subclasses could implement read() with some duplicate codes to avoid unnecessary iteration on + // a singleton list. + return CloseableIterators.withEmptyBaggage(parseInputRows(row).iterator()); + } + catch (IOException e) { + throw new ParseException(e, "Unable to parse row [%s]", row); + } + }); + } + + @Override + public CloseableIterator sample(InputEntity source, File temporaryDirectory) + throws IOException + { + return intermediateRowIterator(source, temporaryDirectory).map(row -> { + final String json; + try { + json = toJson(row); + } + catch (IOException e) { + throw new RuntimeException(e); + } + try { + return InputRowListPlusJson.of(parseInputRows(row), json); + } + catch (ParseException e) { + return InputRowListPlusJson.of(json, e); + } + catch (IOException e) { + return InputRowListPlusJson.of(json, new ParseException(e, "Unable to parse row [%s]", row)); + } + }); + } + + /** + * Creates an iterator of intermediate rows. The returned rows will be consumed by {@link #parseInputRows} and + * {@link #toJson}. + */ + protected abstract CloseableIterator intermediateRowIterator(InputEntity source, File temporaryDirectory) + throws IOException; + + /** + * Parses the given intermediate row into a list of {@link InputRow}s. + */ + protected abstract List parseInputRows(T intermediateRow) throws IOException, ParseException; + + /** + * Converts the given intermediate row into a JSON string. The returned JSON will be used by FirehoseSampler. + * Implementations can use any method to convert the given row into JSON string. If the implementation knows how + * to convert the given row into {@link java.util.Map}, {@link InputEntityReader#DEFAULT_JSON_WRITER} can be + * used to convert the Map into JSON string. + */ + protected abstract String toJson(T intermediateRow) throws IOException; +} diff --git a/core/src/main/java/org/apache/druid/data/input/TextReader.java b/core/src/main/java/org/apache/druid/data/input/TextReader.java index 1b659a80f781..c421bd3511fa 100644 --- a/core/src/main/java/org/apache/druid/data/input/TextReader.java +++ b/core/src/main/java/org/apache/druid/data/input/TextReader.java @@ -20,7 +20,6 @@ package org.apache.druid.data.input; import org.apache.commons.io.LineIterator; -import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; @@ -33,7 +32,7 @@ /** * Abstract {@link InputEntityReader} for text format readers such as CSV or JSON. */ -public abstract class TextReader implements InputEntityReader, InputEntitySampler +public abstract class TextReader extends IntermediateRowParsingReader { private final InputRowSchema inputRowSchema; @@ -48,39 +47,8 @@ public InputRowSchema getInputRowSchema() } @Override - public CloseableIterator read(InputEntity source, File temporaryDirectory) throws IOException - { - return lineIterator(source).flatMap(line -> { - try { - // since readLine() returns a list, the below line always iterates over the list, - // which means it calls Iterator.hasNext() and Iterator.next() at least once per line. - // This could be unnecessary if the line wouldn't be exploded into multiple rows. - // If this line turned out to be a performance bottleneck, perhaps readLine() interface might not be a good - // idea. Subclasses could implement read() with some duplicate codes to avoid unnecessary iteration on - // a singleton list. - return CloseableIterators.withEmptyBaggage(readLine(line).iterator()); - } - catch (IOException e) { - throw new ParseException(e, "Unable to parse row [%s]", line); - } - }); - } - - @Override - public CloseableIterator sample(InputEntity source, File temporaryDirectory) + public CloseableIterator intermediateRowIterator(InputEntity source, File temporaryDirectory) throws IOException - { - return lineIterator(source).map(line -> { - try { - return sampleLine(line); - } - catch (IOException e) { - throw new RuntimeException(e); - } - }); - } - - private CloseableIterator lineIterator(InputEntity source) throws IOException { final LineIterator delegate = new LineIterator( new InputStreamReader(source.open(), StringUtils.UTF8_STRING) @@ -121,18 +89,8 @@ public void close() throws IOException * * This method will be called after {@link #getNumHeaderLinesToSkip()} and {@link #processHeaderLine}. */ - public abstract List readLine(String line) throws IOException, ParseException; - - /** - * TODO - * - * Should handle {@link ParseException} properly. - * - * @param line - * @return - * @throws IOException - */ - public abstract InputRowListPlusJson sampleLine(String line) throws IOException; + @Override + public abstract List parseInputRows(String intermediateRow) throws IOException, ParseException; /** * Returns the number of header lines to skip. diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java index de4c296209dd..79c2c6b657d4 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java @@ -25,7 +25,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputEntityReader; -import org.apache.druid.data.input.InputEntitySampler; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.indexer.Checks; @@ -129,18 +128,6 @@ public InputEntityReader createReader(InputRowSchema inputRowSchema) ); } - @Override - public InputEntitySampler createSampler(InputRowSchema inputRowSchema) - { - return new CsvReader( - inputRowSchema, - listDelimiter, - columns, - findColumnsFromHeader, - skipHeaderRows - ); - } - @Override public boolean equals(Object o) { diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java index a5f02cfc4362..d52e89b926e7 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java @@ -26,7 +26,6 @@ import com.google.common.collect.Iterables; import com.opencsv.CSVParser; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.TextReader; import org.apache.druid.java.util.common.ISE; @@ -82,7 +81,7 @@ public class CsvReader extends TextReader } @Override - public List readLine(String line) throws IOException, ParseException + public List parseInputRows(String line) throws IOException, ParseException { final Map zipped = parseLine(line); return Collections.singletonList( @@ -95,17 +94,10 @@ public List readLine(String line) throws IOException, ParseException } @Override - public InputRowListPlusJson sampleLine(String line) throws IOException + public String toJson(String intermediateRow) throws IOException { - final Map zipped = parseLine(line); - return InputRowListPlusJson.ofJson( - MapInputRowParser.parse( - getInputRowSchema().getTimestampSpec(), - getInputRowSchema().getDimensionsSpec(), - zipped - ), - SAMPLER_JSON_WRITER.writeValueAsString(zipped) - ); + final Map zipped = parseLine(intermediateRow); + return DEFAULT_JSON_WRITER.writeValueAsString(zipped); } private Map parseLine(String line) throws IOException diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java index e20ce259b333..9790b7589c31 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java @@ -21,7 +21,6 @@ import org.apache.druid.data.input.InputEntity; import org.apache.druid.data.input.InputEntityReader; -import org.apache.druid.data.input.InputEntitySampler; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusJson; @@ -80,9 +79,9 @@ public CloseableIterator sample() { return createIterator(entity -> { // InputEntitySampler is stateful and so a new one should be created per entity. - final InputEntitySampler sampler = inputFormat.createSampler(inputRowSchema); + final InputEntityReader reader = inputFormat.createReader(inputRowSchema); try { - return sampler.sample(sourceIterator.next(), temporaryDirectory); + return reader.sample(sourceIterator.next(), temporaryDirectory); } catch (IOException e) { throw new RuntimeException(e); diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java index 348f6c3574ed..e335d9ba671a 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.core.JsonParser.Feature; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.data.input.InputEntityReader; -import org.apache.druid.data.input.InputEntitySampler; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.java.util.common.parsers.JSONPathSpec; @@ -72,12 +71,6 @@ public InputEntityReader createReader(InputRowSchema inputRowSchema) return new JsonReader(inputRowSchema, getFlattenSpec(), objectMapper); } - @Override - public InputEntitySampler createSampler(InputRowSchema inputRowSchema) - { - return new JsonReader(inputRowSchema, getFlattenSpec(), objectMapper); - } - @Override public boolean equals(Object o) { diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java index c0daabc75585..98cf8b95942e 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.TextReader; import org.apache.druid.java.util.common.parsers.JSONFlattenerMaker; @@ -49,7 +48,7 @@ public class JsonReader extends TextReader } @Override - public List readLine(String line) throws IOException, ParseException + public List parseInputRows(String line) throws IOException, ParseException { final JsonNode document = mapper.readValue(line, JsonNode.class); final Map flattened = flattener.flatten(document); @@ -63,24 +62,10 @@ public List readLine(String line) throws IOException, ParseException } @Override - public InputRowListPlusJson sampleLine(String line) throws IOException + public String toJson(String intermediateRow) throws IOException { - final JsonNode document = mapper.readValue(line, JsonNode.class); - final String rawJson = SAMPLER_JSON_WRITER.writeValueAsString(document); - final Map flattened = flattener.flatten(document); - try { - return InputRowListPlusJson.ofJson( - MapInputRowParser.parse( - getInputRowSchema().getTimestampSpec(), - getInputRowSchema().getDimensionsSpec(), - flattened - ), - rawJson - ); - } - catch (ParseException e) { - return InputRowListPlusJson.of(rawJson, e); - } + final JsonNode document = mapper.readValue(intermediateRow, JsonNode.class); + return DEFAULT_JSON_WRITER.writeValueAsString(document); } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java b/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java index debf62a82dd9..ef898a9ff944 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java @@ -70,7 +70,6 @@ public static InputRow parse( return parse(timestampSpec, dimensionsSpec.getDimensionNames(), dimensionsSpec.getDimensionExclusions(), theMap); } - // TODO: maybe prune columns with metrics + dimensions public static InputRow parse( TimestampSpec timestampSpec, List dimensions, diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java index 6db7abdeaae4..8520671645b9 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java @@ -20,7 +20,6 @@ package org.apache.druid.data.input.impl; import org.apache.druid.data.input.InputEntityReader; -import org.apache.druid.data.input.InputEntitySampler; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRowSchema; @@ -38,12 +37,6 @@ public InputEntityReader createReader(InputRowSchema inputRowSchema) throw new UnsupportedOperationException(); } - @Override - public InputEntitySampler createSampler(InputRowSchema inputRowSchema) - { - throw new UnsupportedOperationException(); - } - @Override public String toString() { From 87b83fa5d69e603a0e35e2190d99ecf2287540a8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 13 Nov 2019 17:09:17 -0800 Subject: [PATCH 23/29] Address comments --- .../druid/data/input/AbstractInputSource.java | 6 +-- .../FirehoseFactoryToInputSourceAdaptor.java | 2 +- .../apache/druid/data/input/InputEntity.java | 4 +- .../druid/data/input/InputEntityReader.java | 7 ++-- .../apache/druid/data/input/InputFormat.java | 4 +- .../apache/druid/data/input/InputSource.java | 4 +- .../druid/data/input/InputSourceReader.java | 4 +- .../guice/annotations/ExtensionPoint.java | 1 + .../druid/guice/annotations/PublicApi.java | 1 + .../druid/guice/annotations/UnstableApi.java | 41 +++++++++++++++++++ .../indexing/overlord/TaskLifecycleTest.java | 4 +- 11 files changed, 61 insertions(+), 17 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/guice/annotations/UnstableApi.java diff --git a/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java b/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java index 20936eab9a24..94ddcd6b79e7 100644 --- a/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java @@ -26,7 +26,7 @@ /** * Abstract class for {@link InputSource}. This class provides a default implementation of {@link #reader} with - * a sanity check. Child classes should implement one of {@link #formattableReader} or {@link #unformattableReader} + * a sanity check. Child classes should implement one of {@link #formattableReader} or {@link #fixedFormatReader} * depending on {@link #needsFormat()}. */ public abstract class AbstractInputSource implements InputSource @@ -45,7 +45,7 @@ public InputSourceReader reader( temporaryDirectory ); } else { - return unformattableReader(inputRowSchema, temporaryDirectory); + return fixedFormatReader(inputRowSchema, temporaryDirectory); } } @@ -58,7 +58,7 @@ protected InputSourceReader formattableReader( throw new UnsupportedOperationException("Implement this method properly if needsFormat() = true"); } - protected InputSourceReader unformattableReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) + protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) { throw new UnsupportedOperationException("Implement this method properly if needsFormat() = false"); } diff --git a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java index 8ffc88581475..11466ad6583f 100644 --- a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java +++ b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java @@ -92,7 +92,7 @@ public boolean needsFormat() } @Override - protected InputSourceReader unformattableReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) + protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) { return new FirehoseToInputSourceReaderAdaptor(firehoseFactory, inputRowParser, temporaryDirectory); } diff --git a/core/src/main/java/org/apache/druid/data/input/InputEntity.java b/core/src/main/java/org/apache/druid/data/input/InputEntity.java index 62cc7fc2d2ac..007c20459b4f 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputEntity.java +++ b/core/src/main/java/org/apache/druid/data/input/InputEntity.java @@ -20,7 +20,7 @@ package org.apache.druid.data.input; import com.google.common.base.Predicate; -import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.guice.annotations.UnstableApi; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; @@ -33,7 +33,7 @@ /** * InputEntity abstracts an object and knows how to read bytes from the given object. */ -@ExtensionPoint +@UnstableApi public interface InputEntity { Logger LOG = new Logger(InputEntity.class); diff --git a/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java index 6b10728c3914..d3a9225aa1e9 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java +++ b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; -import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.guice.annotations.UnstableApi; import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.File; @@ -31,9 +31,10 @@ * InputEntityReader knows how to parse data into {@link InputRow}. * This class is stateful and a new InputEntityReader should be created per {@link InputEntity}. * - * @see TextReader for text format readers + * @see IntermediateRowParsingReader + * @see TextReader */ -@ExtensionPoint +@UnstableApi public interface InputEntityReader { /** diff --git a/core/src/main/java/org/apache/druid/data/input/InputFormat.java b/core/src/main/java/org/apache/druid/data/input/InputFormat.java index a638e0b5eb5d..575905e667bc 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/InputFormat.java @@ -27,7 +27,7 @@ import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.NestedInputFormat; import org.apache.druid.data.input.impl.SplittableInputSource; -import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.guice.annotations.UnstableApi; /** * InputFormat abstracts the file format of input data. @@ -36,7 +36,7 @@ * * @see NestedInputFormat for nested input formats such as JSON. */ -@ExtensionPoint +@UnstableApi @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @Type(name = "csv", value = CsvInputFormat.class), diff --git a/core/src/main/java/org/apache/druid/data/input/InputSource.java b/core/src/main/java/org/apache/druid/data/input/InputSource.java index 4ce64e3f0b88..5e52924a8f9b 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSource.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.data.input.impl.HttpInputSource; import org.apache.druid.data.input.impl.LocalInputSource; -import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.guice.annotations.UnstableApi; import javax.annotation.Nullable; import java.io.File; @@ -44,7 +44,7 @@ * } * }

*/ -@ExtensionPoint +@UnstableApi @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @Type(name = "local", value = LocalInputSource.class), diff --git a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java index 8086a75a9fa1..efd81013c26d 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java @@ -20,7 +20,7 @@ package org.apache.druid.data.input; import org.apache.druid.data.input.impl.InputEntityIteratingReader; -import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.guice.annotations.UnstableApi; import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.IOException; @@ -34,7 +34,7 @@ * * See {@link InputEntityIteratingReader} as an example. */ -@ExtensionPoint +@UnstableApi public interface InputSourceReader { CloseableIterator read() throws IOException; diff --git a/core/src/main/java/org/apache/druid/guice/annotations/ExtensionPoint.java b/core/src/main/java/org/apache/druid/guice/annotations/ExtensionPoint.java index e0c71e3b8637..5f122b022db7 100644 --- a/core/src/main/java/org/apache/druid/guice/annotations/ExtensionPoint.java +++ b/core/src/main/java/org/apache/druid/guice/annotations/ExtensionPoint.java @@ -42,6 +42,7 @@ * update of Druid. * * @see PublicApi + * @see UnstableApi */ @Target({ElementType.TYPE}) @Retention(RetentionPolicy.SOURCE) diff --git a/core/src/main/java/org/apache/druid/guice/annotations/PublicApi.java b/core/src/main/java/org/apache/druid/guice/annotations/PublicApi.java index 9a4987197801..ec9ea54afa40 100644 --- a/core/src/main/java/org/apache/druid/guice/annotations/PublicApi.java +++ b/core/src/main/java/org/apache/druid/guice/annotations/PublicApi.java @@ -43,6 +43,7 @@ * as well). * * @see ExtensionPoint + * @see UnstableApi */ @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.CONSTRUCTOR}) @Retention(RetentionPolicy.SOURCE) diff --git a/core/src/main/java/org/apache/druid/guice/annotations/UnstableApi.java b/core/src/main/java/org/apache/druid/guice/annotations/UnstableApi.java new file mode 100644 index 000000000000..75a17758a0b8 --- /dev/null +++ b/core/src/main/java/org/apache/druid/guice/annotations/UnstableApi.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.guice.annotations; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Signifies that the annotated entity is an unstable API for extension authors. Unstable APIs may change at any time + * in breaking ways even between minor Druid release lines (e.g., 0.16.0 -> 0.16.1). + * + * All public and protected fields, methods, and constructors of annotated classes and interfaces are considered + * unstable in this sense. + * + * @see PublicApi + * @see ExtensionPoint + */ +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.SOURCE) +public @interface UnstableApi +{ +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 740560e141d6..305585a13754 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -284,7 +284,7 @@ private static InputRow ir(String dt, String dim1, String dim2, float met) private static class MockExceptionInputSource extends AbstractInputSource { @Override - protected InputSourceReader unformattableReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) + protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) { return new InputSourceReader() { @@ -336,7 +336,7 @@ public boolean needsFormat() private static class MockInputSource extends AbstractInputSource { @Override - protected InputSourceReader unformattableReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) + protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) { return new InputSourceReader() { From 169ab49383bafe8568ff93ce84f47d3fdd04aeba Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 13 Nov 2019 17:30:20 -0800 Subject: [PATCH 24/29] move csv reader test --- .../druid/data/input/impl/CsvReaderTest.java | 111 ++++++++++++++++++ .../util/common/parsers/CSVParserTest.java | 97 --------------- 2 files changed, 111 insertions(+), 97 deletions(-) delete mode 100644 core/src/test/java/org/apache/druid/java/util/common/parsers/CSVParserTest.java diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java index e01d639af19a..752f32429119 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java @@ -20,10 +20,12 @@ package org.apache.druid.data.input.impl; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -34,6 +36,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.stream.Collectors; @@ -140,6 +143,114 @@ public void testMultiValues() throws IOException } } + @Test + public void testQuotes() throws IOException + { + final ByteSource source = writeData( + ImmutableList.of( + "3,\"Lets do some \"\"normal\"\" quotes\",2018-05-05T10:00:00Z", + "34,\"Lets do some \"\"normal\"\", quotes with comma\",2018-05-06T10:00:00Z", + "343,\"Lets try \\\"\"it\\\"\" with slash quotes\",2018-05-07T10:00:00Z", + "545,\"Lets try \\\"\"it\\\"\", with slash quotes and comma\",2018-05-08T10:00:00Z", + "65,Here I write \\n slash n,2018-05-09T10:00:00Z" + ) + ); + final List expectedResults = ImmutableList.of( + new MapBasedInputRow( + DateTimes.of("2018-05-05T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableMap.of( + "Value", + "3", + "Comment", + "Lets do some \"normal\" quotes", + "Timestamp", + "2018-05-05T10:00:00Z" + ) + ), + new MapBasedInputRow( + DateTimes.of("2018-05-06T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableMap.of( + "Value", + "34", + "Comment", + "Lets do some \"normal\", quotes with comma", + "Timestamp", + "2018-05-06T10:00:00Z" + ) + ), + new MapBasedInputRow( + DateTimes.of("2018-05-07T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableMap.of( + "Value", + "343", + "Comment", + "Lets try \\\"it\\\" with slash quotes", + "Timestamp", + "2018-05-07T10:00:00Z" + ) + ), + new MapBasedInputRow( + DateTimes.of("2018-05-08T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableMap.of( + "Value", + "545", + "Comment", + "Lets try \\\"it\\\", with slash quotes and comma", + "Timestamp", + "2018-05-08T10:00:00Z" + ) + ), + new MapBasedInputRow( + DateTimes.of("2018-05-09T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableMap.of("Value", "65", "Comment", "Here I write \\n slash n", "Timestamp", "2018-05-09T10:00:00Z") + ) + ); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("Value", "Comment", "Timestamp"), null, false, 0); + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("Timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("Timestamp"))), + Collections.emptyList() + ) + ); + + try (CloseableIterator iterator = reader.read(source, null)) { + final Iterator expectedRowIterator = expectedResults.iterator(); + while (iterator.hasNext()) { + Assert.assertTrue(expectedRowIterator.hasNext()); + Assert.assertEquals(expectedRowIterator.next(), iterator.next()); + } + } + } + + @Test + public void testRussianTextMess() throws IOException + { + final ByteSource source = writeData( + ImmutableList.of( + "2019-01-01T00:00:10Z,name_1,\"Как говорится: \\\"\"всё течет, всё изменяется\\\"\". Украина как всегда обвиняет Россию в собственных проблемах. #ПровокацияКиева\"" + ) + ); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "Comment"), null, false, 0); + final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA); + try (CloseableIterator iterator = reader.read(source, null)) { + Assert.assertTrue(iterator.hasNext()); + final InputRow row = iterator.next(); + Assert.assertEquals(DateTimes.of("2019-01-01T00:00:10Z"), row.getTimestamp()); + Assert.assertEquals("name_1", Iterables.getOnlyElement(row.getDimension("name"))); + Assert.assertEquals( + "Как говорится: \\\"всё течет, всё изменяется\\\". Украина как всегда обвиняет Россию в собственных проблемах. #ПровокацияКиева", + Iterables.getOnlyElement(row.getDimension("Comment")) + ); + Assert.assertFalse(iterator.hasNext()); + } + } + private ByteSource writeData(List lines) throws IOException { final List byteLines = lines.stream() diff --git a/core/src/test/java/org/apache/druid/java/util/common/parsers/CSVParserTest.java b/core/src/test/java/org/apache/druid/java/util/common/parsers/CSVParserTest.java deleted file mode 100644 index 5fa6ae51dbd6..000000000000 --- a/core/src/test/java/org/apache/druid/java/util/common/parsers/CSVParserTest.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.java.util.common.parsers; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -public class CSVParserTest -{ - @Test - public void testBasic() - { - CSVParser parser = new CSVParser(null, ImmutableList.of("Value", "Comment", "Timestamp"), false, 0); - - final List inputs = ImmutableList.of( - "3,\"Lets do some \"\"normal\"\" quotes\",2018-05-05T10:00:00Z", - "34,\"Lets do some \"\"normal\"\", quotes with comma\",2018-05-06T10:00:00Z", - "343,\"Lets try \\\"\"it\\\"\" with slash quotes\",2018-05-07T10:00:00Z", - "545,\"Lets try \\\"\"it\\\"\", with slash quotes and comma\",2018-05-08T10:00:00Z", - "65,Here I write \\n slash n,2018-05-09T10:00:00Z" - ); - final List> expectedResult = ImmutableList.of( - ImmutableMap.of("Value", "3", "Comment", "Lets do some \"normal\" quotes", "Timestamp", "2018-05-05T10:00:00Z"), - ImmutableMap.of( - "Value", - "34", - "Comment", - "Lets do some \"normal\", quotes with comma", - "Timestamp", - "2018-05-06T10:00:00Z" - ), - ImmutableMap.of( - "Value", - "343", - "Comment", - "Lets try \\\"it\\\" with slash quotes", - "Timestamp", - "2018-05-07T10:00:00Z" - ), - ImmutableMap.of( - "Value", - "545", - "Comment", - "Lets try \\\"it\\\", with slash quotes and comma", - "Timestamp", - "2018-05-08T10:00:00Z" - ), - ImmutableMap.of("Value", "65", "Comment", "Here I write \\n slash n", "Timestamp", "2018-05-09T10:00:00Z") - ); - final List> parsedResult = new ArrayList<>(); - - for (String input : inputs) { - Map parsedLineList = parser.parseToMap(input); - parsedResult.add(parsedLineList); - } - - Assert.assertEquals(expectedResult, parsedResult); - } - - @Test - public void testRussianTextMess() - { - CSVParser parser = new CSVParser(null, ImmutableList.of("Comment"), false, 0); - final String input = "\"Как говорится: \\\"\"всё течет, всё изменяется\\\"\". Украина как всегда обвиняет Россию в собственных проблемах. #ПровокацияКиева\""; - final Map expect = ImmutableMap.of( - "Comment", - "Как говорится: \\\"всё течет, всё изменяется\\\". Украина как всегда обвиняет Россию в собственных проблемах. #ПровокацияКиева" - ); - final Map parsedInput = parser.parseToMap(input); - - Assert.assertEquals(String.class, parsedInput.get("Comment").getClass()); - Assert.assertEquals(expect, parsedInput); - } -} From a9d167a0caac99ccd0d641923805946c8a16d90d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 13 Nov 2019 17:32:46 -0800 Subject: [PATCH 25/29] remove test for verify --- .../data/input/impl/CsvInputFormatTest.java | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java index 893687f3b7e0..8d2d688e9868 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java @@ -21,14 +21,12 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.data.input.InputFormat; -import org.apache.druid.data.input.InputRowSchema; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import java.io.IOException; -import java.util.Arrays; import java.util.Collections; public class CsvInputFormatTest @@ -53,19 +51,4 @@ public void testComma() expectedException.expectMessage("Column[a,] has a comma, it cannot"); new CsvInputFormat(Collections.singletonList("a,"), ",", false, 0); } - - @Test - public void testComma2() - { - final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), ",", false, 0); - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("column[a,] not in columns"); - format.createReader( - new InputRowSchema( - new TimestampSpec("timestamp", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("a,", "b"))), - Collections.emptyList() - ) - ); - } } From 42a6965a4dc19e17c3c85f4a119b9fb76158e3f4 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 13 Nov 2019 17:56:34 -0800 Subject: [PATCH 26/29] adjust comments --- .../java/org/apache/druid/data/input/InputEntity.java | 6 ++++-- .../java/org/apache/druid/data/input/InputFormat.java | 6 +++--- .../org/apache/druid/data/input/InputRowSchema.java | 3 +++ .../java/org/apache/druid/data/input/InputSource.java | 2 +- .../org/apache/druid/data/input/InputSourceReader.java | 2 +- .../druid/data/input/IntermediateRowParsingReader.java | 6 +++--- .../data/input/impl/InputEntityIteratingReader.java | 2 +- .../apache/druid/data/input/impl/LocalInputSource.java | 2 +- .../org/apache/druid/java/util/common/FileUtils.java | 10 +++++----- 9 files changed, 22 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/InputEntity.java b/core/src/main/java/org/apache/druid/data/input/InputEntity.java index 007c20459b4f..eaecd609afdf 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputEntity.java +++ b/core/src/main/java/org/apache/druid/data/input/InputEntity.java @@ -32,6 +32,8 @@ /** * InputEntity abstracts an object and knows how to read bytes from the given object. + * + * @param Type of input entity */ @UnstableApi public interface InputEntity @@ -39,7 +41,7 @@ public interface InputEntity Logger LOG = new Logger(InputEntity.class); int DEFAULT_FETCH_BUFFER_SIZE = 4 * 1024; // 4 KB - int DEFAULT_MAX_FETCH_RETRY = 2; // 3 tries including the initial try + int DEFAULT_MAX_NUM_FETCH_TRIES = 3; // 3 tries including the initial try /** * CleanableFile is the result type of {@link #fetch}. @@ -83,7 +85,7 @@ default CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) throws tempFile, fetchBuffer, getFetchRetryCondition(), - DEFAULT_MAX_FETCH_RETRY, + DEFAULT_MAX_NUM_FETCH_TRIES, StringUtils.format("Failed to fetch into [%s]", tempFile.getAbsolutePath()) ); } diff --git a/core/src/main/java/org/apache/druid/data/input/InputFormat.java b/core/src/main/java/org/apache/druid/data/input/InputFormat.java index 575905e667bc..8ac986ecb999 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/InputFormat.java @@ -32,9 +32,9 @@ /** * InputFormat abstracts the file format of input data. * It creates a {@link InputEntityReader} to read data and parse it into {@link InputRow}. - * The created SplitReader is used by {@link InputSourceReader}. + * The created InputEntityReader is used by {@link InputSourceReader}. * - * @see NestedInputFormat for nested input formats such as JSON. + * See {@link NestedInputFormat} for nested input formats such as JSON. */ @UnstableApi @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @@ -48,7 +48,7 @@ public interface InputFormat * Trait to indicate that a file can be split into multiple {@link InputSplit}s. * * This method is not being used anywhere for now, but should be considered - * in {@link SplittableInputSource#createSplits}. + * in {@link SplittableInputSource#createSplits} in the future. */ @JsonIgnore boolean isSplittable(); diff --git a/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java b/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java index b4b7fc477fd4..c908187962e7 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java +++ b/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java @@ -24,6 +24,9 @@ import java.util.List; +/** + * Schema of {@link InputRow}. + */ public class InputRowSchema { private final TimestampSpec timestampSpec; diff --git a/core/src/main/java/org/apache/druid/data/input/InputSource.java b/core/src/main/java/org/apache/druid/data/input/InputSource.java index 5e52924a8f9b..6109c592604c 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSource.java @@ -67,7 +67,7 @@ public interface InputSource boolean needsFormat(); /** - * Create an {@link InputSourceReader}. + * Creates an {@link InputSourceReader}. * * @param inputRowSchema for {@link InputRow} * @param inputFormat to parse data. It can be null if {@link #needsFormat()} = true diff --git a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java index efd81013c26d..0a8921422c39 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java @@ -27,7 +27,7 @@ /** * InputSourceReader reads data from {@link InputSource} and returns a {@link CloseableIterator} of - * {@link InputRow}. See {@link InputSource} for an example usage. + * {@link InputRow}s. See {@link InputSource} for an example usage. * * Implementations of this class can use {@link InputEntity} and {@link InputEntityReader}. {@link InputFormat} * can be useful to understand how to create an InputEntityReader. diff --git a/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java b/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java index 0693831b9ef6..5ebfa276052b 100644 --- a/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java @@ -28,9 +28,9 @@ import java.util.List; /** - * {@link InputEntityReader} that parses bytes into some intermediate rows first, and then parse them - * into {@link InputRow}s. For example, {@link org.apache.druid.data.input.impl.CsvReader} parses bytes - * into string lines, and then parses those lines into InputRows. + * {@link InputEntityReader} that parses bytes into some intermediate rows first, and then into {@link InputRow}s. + * For example, {@link org.apache.druid.data.input.impl.CsvReader} parses bytes into string lines, and then parses + * those lines into InputRows. * * @param type of intermediate row. For example, it can be {@link String} for text formats. */ diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java index 9790b7589c31..cd23cbf010ce 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java @@ -78,7 +78,7 @@ public CloseableIterator read() public CloseableIterator sample() { return createIterator(entity -> { - // InputEntitySampler is stateful and so a new one should be created per entity. + // InputEntityReader is stateful and so a new one should be created per entity. final InputEntityReader reader = inputFormat.createReader(inputRowSchema); try { return reader.sample(sourceIterator.next(), temporaryDirectory); diff --git a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java index 022e5d7ed842..5149fb4f3323 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java @@ -114,7 +114,7 @@ protected InputSourceReader formattableReader( return new InputEntityIteratingReader<>( inputRowSchema, inputFormat, - // reader() is supposed to be called in each task that creates segments. + // formattableReader() is supposed to be called in each task that actually creates segments. // The task should already have only one split in parallel indexing, // while there's no need to make splits using splitHintSpec in sequential indexing. createSplits(inputFormat, null).map(split -> new FileSource(split.get())), diff --git a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java index aedf67166b0e..1964aea71264 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java @@ -272,7 +272,7 @@ public void close() * @param outFile file to write data * @param fetchBuffer a buffer to copy data from the input stream to the file * @param retryCondition condition which should be satisfied for retry - * @param numRetries max number of retries + * @param numTries max number of retries * @param messageOnRetry log message on retry * * @return the number of bytes copied @@ -283,7 +283,7 @@ public static long copyLarge( File outFile, byte[] fetchBuffer, Predicate retryCondition, - int numRetries, + int numTries, String messageOnRetry ) throws IOException { @@ -297,7 +297,7 @@ public static long copyLarge( }, retryCondition, outFile::delete, - numRetries, + numTries, messageOnRetry ); } @@ -311,7 +311,7 @@ public static long copyLarge( File outFile, byte[] fetchBuffer, Predicate retryCondition, - int numRetries, + int numTries, String messageOnRetry ) throws IOException { @@ -324,7 +324,7 @@ public static long copyLarge( }, retryCondition, outFile::delete, - numRetries, + numTries, messageOnRetry ); } From 230803bb73c8b80c781b2142bf2df3b96045e1ec Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 13 Nov 2019 21:57:28 -0800 Subject: [PATCH 27/29] Fix InputEntityIteratingReader --- .../druid/data/input/impl/InputEntityIteratingReader.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java index cd23cbf010ce..720feeaa894c 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java @@ -66,7 +66,7 @@ public CloseableIterator read() // InputEntityReader is stateful and so a new one should be created per entity. final InputEntityReader reader = inputFormat.createReader(inputRowSchema); try { - return reader.read(sourceIterator.next(), temporaryDirectory); + return reader.read(entity, temporaryDirectory); } catch (IOException e) { throw new RuntimeException(e); @@ -81,7 +81,7 @@ public CloseableIterator sample() // InputEntityReader is stateful and so a new one should be created per entity. final InputEntityReader reader = inputFormat.createReader(inputRowSchema); try { - return reader.sample(sourceIterator.next(), temporaryDirectory); + return reader.sample(entity, temporaryDirectory); } catch (IOException e) { throw new RuntimeException(e); From 540759beecb1b28d3f7e9d5c50f1e5d6c0f17589 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 14 Nov 2019 12:10:13 -0800 Subject: [PATCH 28/29] rename source -> entity --- .../impl/{ByteSource.java => ByteEntity.java} | 6 +++--- .../impl/{FileSource.java => FileEntity.java} | 4 ++-- .../impl/{HttpSource.java => HttpEntity.java} | 4 ++-- .../data/input/impl/HttpInputSource.java | 2 +- .../data/input/impl/LocalInputSource.java | 2 +- .../druid/data/input/impl/CsvReaderTest.java | 20 +++++++++---------- .../impl/InputEntityIteratingReaderTest.java | 2 +- .../druid/data/input/impl/JsonReaderTest.java | 4 ++-- .../firehose/HttpFirehoseFactory.java | 4 ++-- 9 files changed, 24 insertions(+), 24 deletions(-) rename core/src/main/java/org/apache/druid/data/input/impl/{ByteSource.java => ByteEntity.java} (91%) rename core/src/main/java/org/apache/druid/data/input/impl/{FileSource.java => FileEntity.java} (95%) rename core/src/main/java/org/apache/druid/data/input/impl/{HttpSource.java => HttpEntity.java} (97%) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java b/core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java similarity index 91% rename from core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java rename to core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java index 6c8cc0edbaab..a995b9a0a4e9 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/ByteSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java @@ -27,16 +27,16 @@ import java.io.InputStream; import java.nio.ByteBuffer; -public class ByteSource implements InputEntity +public class ByteEntity implements InputEntity { private final ByteBuffer buffer; - public ByteSource(ByteBuffer buffer) + public ByteEntity(ByteBuffer buffer) { this.buffer = buffer.duplicate(); } - public ByteSource(byte[] bytes) + public ByteEntity(byte[] bytes) { this(ByteBuffer.wrap(bytes)); } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java b/core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java similarity index 95% rename from core/src/main/java/org/apache/druid/data/input/impl/FileSource.java rename to core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java index 0d1299781d24..3837135e9d71 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FileSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java @@ -29,11 +29,11 @@ import java.io.IOException; import java.io.InputStream; -public class FileSource implements InputEntity +public class FileEntity implements InputEntity { private final File file; - FileSource(File file) + FileEntity(File file) { this.file = file; } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java similarity index 97% rename from core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java rename to core/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java index c332ef466d3e..d9138a8fd116 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/HttpSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java @@ -33,7 +33,7 @@ import java.net.URLConnection; import java.util.Base64; -public class HttpSource implements InputEntity +public class HttpEntity implements InputEntity { private final URI split; @Nullable @@ -41,7 +41,7 @@ public class HttpSource implements InputEntity @Nullable private final PasswordProvider httpAuthenticationPasswordProvider; - HttpSource( + HttpEntity( URI split, @Nullable String httpAuthenticationUsername, @Nullable PasswordProvider httpAuthenticationPasswordProvider diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java index 666626725fee..201d656001bd 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java @@ -111,7 +111,7 @@ protected InputSourceReader formattableReader( return new InputEntityIteratingReader<>( inputRowSchema, inputFormat, - createSplits(inputFormat, null).map(split -> new HttpSource( + createSplits(inputFormat, null).map(split -> new HttpEntity( split.get(), httpAuthenticationUsername, httpAuthenticationPasswordProvider diff --git a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java index 5149fb4f3323..900020d7e630 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java @@ -117,7 +117,7 @@ protected InputSourceReader formattableReader( // formattableReader() is supposed to be called in each task that actually creates segments. // The task should already have only one split in parallel indexing, // while there's no need to make splits using splitHintSpec in sequential indexing. - createSplits(inputFormat, null).map(split -> new FileSource(split.get())), + createSplits(inputFormat, null).map(split -> new FileEntity(split.get())), temporaryDirectory ); } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java index 752f32429119..c988b1200267 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java @@ -51,7 +51,7 @@ public class CsvReaderTest @Test public void testWithoutHeaders() throws IOException { - final ByteSource source = writeData( + final ByteEntity source = writeData( ImmutableList.of( "2019-01-01T00:00:10Z,name_1,5", "2019-01-01T00:00:20Z,name_2,10", @@ -65,7 +65,7 @@ public void testWithoutHeaders() throws IOException @Test public void testFindColumn() throws IOException { - final ByteSource source = writeData( + final ByteEntity source = writeData( ImmutableList.of( "ts,name,score", "2019-01-01T00:00:10Z,name_1,5", @@ -80,7 +80,7 @@ public void testFindColumn() throws IOException @Test public void testSkipHeaders() throws IOException { - final ByteSource source = writeData( + final ByteEntity source = writeData( ImmutableList.of( "this,is,a,row,to,skip", "2019-01-01T00:00:10Z,name_1,5", @@ -95,7 +95,7 @@ public void testSkipHeaders() throws IOException @Test public void testFindColumnAndSkipHeaders() throws IOException { - final ByteSource source = writeData( + final ByteEntity source = writeData( ImmutableList.of( "this,is,a,row,to,skip", "ts,name,score", @@ -111,7 +111,7 @@ public void testFindColumnAndSkipHeaders() throws IOException @Test public void testMultiValues() throws IOException { - final ByteSource source = writeData( + final ByteEntity source = writeData( ImmutableList.of( "ts,name,score", "2019-01-01T00:00:10Z,name_1,5|1", @@ -146,7 +146,7 @@ public void testMultiValues() throws IOException @Test public void testQuotes() throws IOException { - final ByteSource source = writeData( + final ByteEntity source = writeData( ImmutableList.of( "3,\"Lets do some \"\"normal\"\" quotes\",2018-05-05T10:00:00Z", "34,\"Lets do some \"\"normal\"\", quotes with comma\",2018-05-06T10:00:00Z", @@ -231,7 +231,7 @@ public void testQuotes() throws IOException @Test public void testRussianTextMess() throws IOException { - final ByteSource source = writeData( + final ByteEntity source = writeData( ImmutableList.of( "2019-01-01T00:00:10Z,name_1,\"Как говорится: \\\"\"всё течет, всё изменяется\\\"\". Украина как всегда обвиняет Россию в собственных проблемах. #ПровокацияКиева\"" ) @@ -251,7 +251,7 @@ public void testRussianTextMess() throws IOException } } - private ByteSource writeData(List lines) throws IOException + private ByteEntity writeData(List lines) throws IOException { final List byteLines = lines.stream() .map(line -> StringUtils.toUtf8(line + "\n")) @@ -262,10 +262,10 @@ private ByteSource writeData(List lines) throws IOException for (byte[] bytes : byteLines) { outputStream.write(bytes); } - return new ByteSource(outputStream.toByteArray()); + return new ByteEntity(outputStream.toByteArray()); } - private void assertResult(ByteSource source, CsvInputFormat format) throws IOException + private void assertResult(ByteEntity source, CsvInputFormat format) throws IOException { final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA); int numResults = 0; diff --git a/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java index af5ba136c7f3..bc8c0333c746 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java @@ -72,7 +72,7 @@ public void test() throws IOException false, 0 ), - files.stream().flatMap(file -> ImmutableList.of(new FileSource(file)).stream()), + files.stream().flatMap(file -> ImmutableList.of(new FileEntity(file)).stream()), temporaryFolder.newFolder() ); diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java index a364e21d86b1..f91fe2d37ba3 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java @@ -56,7 +56,7 @@ public void testParseRow() throws IOException null ); - final ByteSource source = new ByteSource( + final ByteEntity source = new ByteEntity( StringUtils.toUtf8("{\"timestamp\":\"2019-01-01\",\"bar\":null,\"foo\":\"x\",\"baz\":4,\"o\":{\"mg\":1}}") ); @@ -103,7 +103,7 @@ public void testParseRowWithConditional() throws IOException null ); - final ByteSource source = new ByteSource( + final ByteEntity source = new ByteEntity( StringUtils.toUtf8("{\"timestamp\":\"2019-01-01\",\"something_else\": {\"foo\": \"test\"}}") ); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java index 202804195121..3d536d3b00c0 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java @@ -26,7 +26,7 @@ import com.google.common.net.HttpHeaders; import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.HttpSource; +import org.apache.druid.data.input.impl.HttpEntity; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; import org.apache.druid.java.util.common.StringUtils; @@ -108,7 +108,7 @@ protected InputStream openObjectStream(URI object) throws IOException @Override protected InputStream openObjectStream(URI object, long start) throws IOException { - URLConnection urlConnection = HttpSource.openURLConnection( + URLConnection urlConnection = HttpEntity.openURLConnection( object, httpAuthenticationUsername, httpAuthenticationPasswordProvider From ce88049c6969fbee65078064949ad0122be9565c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 14 Nov 2019 12:43:46 -0800 Subject: [PATCH 29/29] address comments --- .../apache/druid/data/input/InputEntity.java | 27 +++++++++++-------- .../druid/data/input/InputEntityReader.java | 4 +-- .../input/IntermediateRowParsingReader.java | 6 ++--- .../apache/druid/data/input/TextReader.java | 2 +- .../druid/data/input/impl/ByteEntity.java | 9 ++++--- .../druid/data/input/impl/CsvReader.java | 8 +----- .../druid/data/input/impl/FileEntity.java | 7 ++--- .../druid/data/input/impl/HttpEntity.java | 16 +++++------ .../data/input/impl/HttpInputSource.java | 2 +- .../impl/InputEntityIteratingReader.java | 8 +++--- .../druid/data/input/impl/JsonReader.java | 8 +----- .../data/input/impl/LocalInputSource.java | 2 +- .../data/input/impl/MapInputRowParser.java | 6 +++++ .../druid/guice/annotations/UnstableApi.java | 3 +++ .../impl/InputEntityIteratingReaderTest.java | 2 +- .../druid/segment/transform/Transformer.java | 9 ++++++- 16 files changed, 66 insertions(+), 53 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/InputEntity.java b/core/src/main/java/org/apache/druid/data/input/InputEntity.java index eaecd609afdf..6309ffad8522 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputEntity.java +++ b/core/src/main/java/org/apache/druid/data/input/InputEntity.java @@ -25,18 +25,18 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.net.URI; /** - * InputEntity abstracts an object and knows how to read bytes from the given object. - * - * @param Type of input entity + * InputEntity abstracts an input entity and knows how to read bytes from the given entity. */ @UnstableApi -public interface InputEntity +public interface InputEntity { Logger LOG = new Logger(InputEntity.class); @@ -52,18 +52,23 @@ interface CleanableFile extends Closeable File file(); } - T getObject(); + /** + * Returns an URI to identify the input entity. Implementations can return null if they don't have + * an unique URI. + */ + @Nullable + URI getUri(); /** - * Opens an {@link InputStream} on the object directly. - * This is the basic way to read the given object. + * Opens an {@link InputStream} on the input entity directly. + * This is the basic way to read the given entity. * * @see #fetch as an alternative way to read data. */ InputStream open() throws IOException; /** - * Fetches the object into the local storage. + * Fetches the input entity into the local storage. * This method might be preferred instead of {@link #open()}, for example * * - {@link InputFormat} requires expensive random access on remote storage. @@ -71,14 +76,14 @@ interface CleanableFile extends Closeable * * @param temporaryDirectory to store temp data. This directory will be removed automatically once * the task finishes. - * @param fetchBuffer is used to fetch remote object into local storage. + * @param fetchBuffer is used to fetch remote entity into local storage. * * @see FileUtils#copyLarge */ default CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) throws IOException { - final File tempFile = File.createTempFile("druid-object-source", ".tmp", temporaryDirectory); - LOG.debug("Fetching object into file[%s]", tempFile.getAbsolutePath()); + final File tempFile = File.createTempFile("druid-input-entity", ".tmp", temporaryDirectory); + LOG.debug("Fetching entity into file[%s]", tempFile.getAbsolutePath()); try (InputStream is = open()) { FileUtils.copyLarge( is, diff --git a/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java index d3a9225aa1e9..fbef80506e8a 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java +++ b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java @@ -44,7 +44,7 @@ public interface InputEntityReader */ ObjectWriter DEFAULT_JSON_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter(); - CloseableIterator read(InputEntity source, File temporaryDirectory) throws IOException; + CloseableIterator read(InputEntity source, File temporaryDirectory) throws IOException; - CloseableIterator sample(InputEntity source, File temporaryDirectory) throws IOException; + CloseableIterator sample(InputEntity source, File temporaryDirectory) throws IOException; } diff --git a/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java b/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java index 5ebfa276052b..41a7df47dd02 100644 --- a/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java @@ -37,7 +37,7 @@ public abstract class IntermediateRowParsingReader implements InputEntityReader { @Override - public CloseableIterator read(InputEntity source, File temporaryDirectory) throws IOException + public CloseableIterator read(InputEntity source, File temporaryDirectory) throws IOException { return intermediateRowIterator(source, temporaryDirectory).flatMap(row -> { try { @@ -56,7 +56,7 @@ public CloseableIterator read(InputEntity source, File temporaryDir } @Override - public CloseableIterator sample(InputEntity source, File temporaryDirectory) + public CloseableIterator sample(InputEntity source, File temporaryDirectory) throws IOException { return intermediateRowIterator(source, temporaryDirectory).map(row -> { @@ -83,7 +83,7 @@ public CloseableIterator sample(InputEntity source, Fil * Creates an iterator of intermediate rows. The returned rows will be consumed by {@link #parseInputRows} and * {@link #toJson}. */ - protected abstract CloseableIterator intermediateRowIterator(InputEntity source, File temporaryDirectory) + protected abstract CloseableIterator intermediateRowIterator(InputEntity source, File temporaryDirectory) throws IOException; /** diff --git a/core/src/main/java/org/apache/druid/data/input/TextReader.java b/core/src/main/java/org/apache/druid/data/input/TextReader.java index d967ef8a3993..beb748e978c3 100644 --- a/core/src/main/java/org/apache/druid/data/input/TextReader.java +++ b/core/src/main/java/org/apache/druid/data/input/TextReader.java @@ -50,7 +50,7 @@ public InputRowSchema getInputRowSchema() } @Override - public CloseableIterator intermediateRowIterator(InputEntity source, File temporaryDirectory) + public CloseableIterator intermediateRowIterator(InputEntity source, File temporaryDirectory) throws IOException { final LineIterator delegate = new LineIterator( diff --git a/core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java b/core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java index a995b9a0a4e9..f356b2084dd0 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java @@ -24,10 +24,12 @@ import org.apache.druid.data.input.InputEntity; import org.apache.druid.io.ByteBufferInputStream; +import javax.annotation.Nullable; import java.io.InputStream; +import java.net.URI; import java.nio.ByteBuffer; -public class ByteEntity implements InputEntity +public class ByteEntity implements InputEntity { private final ByteBuffer buffer; @@ -42,9 +44,10 @@ public ByteEntity(byte[] bytes) } @Override - public ByteBuffer getObject() + @Nullable + public URI getUri() { - return buffer; + return null; } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java index 3efd77af108d..f21afe153847 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java @@ -81,13 +81,7 @@ public class CsvReader extends TextReader public List parseInputRows(String line) throws IOException, ParseException { final Map zipped = parseLine(line); - return Collections.singletonList( - MapInputRowParser.parse( - getInputRowSchema().getTimestampSpec(), - getInputRowSchema().getDimensionsSpec(), - zipped - ) - ); + return Collections.singletonList(MapInputRowParser.parse(getInputRowSchema(), zipped)); } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java b/core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java index 3837135e9d71..d48345710a3e 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java @@ -28,8 +28,9 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.net.URI; -public class FileEntity implements InputEntity +public class FileEntity implements InputEntity { private final File file; @@ -58,9 +59,9 @@ public void close() } @Override - public File getObject() + public URI getUri() { - return file; + return file.toURI(); } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java index d9138a8fd116..220b5c99dc42 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java @@ -33,37 +33,37 @@ import java.net.URLConnection; import java.util.Base64; -public class HttpEntity implements InputEntity +public class HttpEntity implements InputEntity { - private final URI split; + private final URI uri; @Nullable private final String httpAuthenticationUsername; @Nullable private final PasswordProvider httpAuthenticationPasswordProvider; HttpEntity( - URI split, + URI uri, @Nullable String httpAuthenticationUsername, @Nullable PasswordProvider httpAuthenticationPasswordProvider ) { - this.split = split; + this.uri = uri; this.httpAuthenticationUsername = httpAuthenticationUsername; this.httpAuthenticationPasswordProvider = httpAuthenticationPasswordProvider; } @Override - public URI getObject() + public URI getUri() { - return split; + return uri; } @Override public InputStream open() throws IOException { return CompressionUtils.decompress( - openURLConnection(split, httpAuthenticationUsername, httpAuthenticationPasswordProvider).getInputStream(), - split.toString() + openURLConnection(uri, httpAuthenticationUsername, httpAuthenticationPasswordProvider).getInputStream(), + uri.toString() ); } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java index 201d656001bd..7a822a8e9618 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java @@ -108,7 +108,7 @@ protected InputSourceReader formattableReader( @Nullable File temporaryDirectory ) { - return new InputEntityIteratingReader<>( + return new InputEntityIteratingReader( inputRowSchema, inputFormat, createSplits(inputFormat, null).map(split -> new HttpEntity( diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java index 720feeaa894c..b726ec78af3b 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java @@ -39,17 +39,17 @@ * InputSourceReader iterating multiple {@link InputEntity}s. This class could be used for * most of {@link org.apache.druid.data.input.InputSource}s. */ -public class InputEntityIteratingReader implements InputSourceReader +public class InputEntityIteratingReader implements InputSourceReader { private final InputRowSchema inputRowSchema; private final InputFormat inputFormat; - private final Iterator> sourceIterator; + private final Iterator sourceIterator; private final File temporaryDirectory; InputEntityIteratingReader( InputRowSchema inputRowSchema, InputFormat inputFormat, - Stream> sourceStream, + Stream sourceStream, File temporaryDirectory ) { @@ -89,7 +89,7 @@ public CloseableIterator sample() }); } - private CloseableIterator createIterator(Function, CloseableIterator> rowPopulator) + private CloseableIterator createIterator(Function> rowPopulator) { return CloseableIterators.withEmptyBaggage(sourceIterator).flatMap(rowPopulator); } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java index 98cf8b95942e..61526efe6912 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java @@ -52,13 +52,7 @@ public List parseInputRows(String line) throws IOException, ParseExcep { final JsonNode document = mapper.readValue(line, JsonNode.class); final Map flattened = flattener.flatten(document); - return Collections.singletonList( - MapInputRowParser.parse( - getInputRowSchema().getTimestampSpec(), - getInputRowSchema().getDimensionsSpec(), - flattened - ) - ); + return Collections.singletonList(MapInputRowParser.parse(getInputRowSchema(), flattened)); } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java index 900020d7e630..54e56136fcfa 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java @@ -111,7 +111,7 @@ protected InputSourceReader formattableReader( @Nullable File temporaryDirectory ) { - return new InputEntityIteratingReader<>( + return new InputEntityIteratingReader( inputRowSchema, inputFormat, // formattableReader() is supposed to be called in each task that actually creates segments. diff --git a/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java b/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java index ef898a9ff944..f7c34434b0fc 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; @@ -61,6 +62,11 @@ public List parseBatch(Map theMap) ); } + public static InputRow parse(InputRowSchema inputRowSchema, Map theMap) throws ParseException + { + return parse(inputRowSchema.getTimestampSpec(), inputRowSchema.getDimensionsSpec(), theMap); + } + public static InputRow parse( TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec, diff --git a/core/src/main/java/org/apache/druid/guice/annotations/UnstableApi.java b/core/src/main/java/org/apache/druid/guice/annotations/UnstableApi.java index 75a17758a0b8..c7b7cd8f11be 100644 --- a/core/src/main/java/org/apache/druid/guice/annotations/UnstableApi.java +++ b/core/src/main/java/org/apache/druid/guice/annotations/UnstableApi.java @@ -31,6 +31,9 @@ * All public and protected fields, methods, and constructors of annotated classes and interfaces are considered * unstable in this sense. * + * Unstable APIs can become {@link PublicApi}s or {@link ExtensionPoint}s once they settle down. This change can happen + * only between major Druid release lines (e.g., 0.16.0 -> 0.17.0). + * * @see PublicApi * @see ExtensionPoint */ diff --git a/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java index bc8c0333c746..6d55f14d103c 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java @@ -58,7 +58,7 @@ public void test() throws IOException writer.write(StringUtils.format("%d,%s,%d", 20190102 + i, "name_" + (i + 1), i + 1)); } } - final InputEntityIteratingReader firehose = new InputEntityIteratingReader<>( + final InputEntityIteratingReader firehose = new InputEntityIteratingReader( new InputRowSchema( new TimestampSpec("time", "yyyyMMdd", null), new DimensionsSpec( diff --git a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java index 109195f38242..ce34da044879 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.transform; +import com.google.common.base.Preconditions; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.Row; @@ -30,6 +31,7 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -105,7 +107,12 @@ public InputRowListPlusJson transform(@Nullable final InputRowListPlusJson row) if (transforms.isEmpty()) { transformedRow = row; } else { - transformedRow = InputRowListPlusJson.of(new TransformedInputRow(row.getInputRow(), transforms), row.getRaw()); + final List originalRows = Preconditions.checkNotNull(row.getInputRows(), "rows before transform"); + final List transformedRows = new ArrayList<>(originalRows.size()); + for (InputRow originalRow : originalRows) { + transformedRows.add(new TransformedInputRow(originalRow, transforms)); + } + transformedRow = InputRowListPlusJson.of(transformedRows, row.getRawJson()); } if (valueMatcher != null) {