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..94ddcd6b79e7 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.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; + +import com.google.common.base.Preconditions; + +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 #fixedFormatReader} + * 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 fixedFormatReader(inputRowSchema, temporaryDirectory); + } + } + + protected InputSourceReader formattableReader( + InputRowSchema inputRowSchema, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) + { + throw new UnsupportedOperationException("Implement this method properly if needsFormat() = true"); + } + + 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/FiniteFirehoseFactory.java b/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java index 09f943bf3674..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,9 +30,12 @@ * {@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 */ +@Deprecated public interface FiniteFirehoseFactory extends FirehoseFactory { /** 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/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 new file mode 100644 index 000000000000..11466ad6583f --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java @@ -0,0 +1,99 @@ +/* + * 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.FirehoseToInputSourceReaderAdaptor; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.SplittableInputSource; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.stream.Stream; + +public class FirehoseFactoryToInputSourceAdaptor extends AbstractInputSource implements SplittableInputSource +{ + private final FiniteFirehoseFactory firehoseFactory; + private final 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() + { + return firehoseFactory.isSplittable(); + } + + @Override + public Stream createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + throws IOException + { + if (firehoseFactory.isSplittable()) { + return firehoseFactory.getSplits(splitHintSpec); + } else { + throw new UnsupportedOperationException(); + } + } + + @Override + public int getNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException + { + if (firehoseFactory.isSplittable()) { + return firehoseFactory.getNumSplits(splitHintSpec); + } else { + throw new UnsupportedOperationException(); + } + } + + @Override + public SplittableInputSource withSplit(InputSplit split) + { + if (firehoseFactory.isSplittable()) { + return new FirehoseFactoryToInputSourceAdaptor( + firehoseFactory.withSplit(split), + inputRowParser + ); + } else { + throw new UnsupportedOperationException(); + } + } + + @Override + public boolean needsFormat() + { + return false; + } + + @Override + 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 new file mode 100644 index 000000000000..6309ffad8522 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/InputEntity.java @@ -0,0 +1,120 @@ +/* + * 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.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; + +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 input entity and knows how to read bytes from the given entity. + */ +@UnstableApi +public interface InputEntity +{ + Logger LOG = new Logger(InputEntity.class); + + int DEFAULT_FETCH_BUFFER_SIZE = 4 * 1024; // 4 KB + int DEFAULT_MAX_NUM_FETCH_TRIES = 3; // 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(); + } + + /** + * 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 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 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. + * - 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 entity into local storage. + * + * @see FileUtils#copyLarge + */ + default CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) throws IOException + { + 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, + tempFile, + fetchBuffer, + getFetchRetryCondition(), + DEFAULT_MAX_NUM_FETCH_TRIES, + 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 matching to the returned predicate. + */ + Predicate getFetchRetryCondition(); +} 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 new file mode 100644 index 000000000000..fbef80506e8a --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java @@ -0,0 +1,50 @@ +/* + * 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.guice.annotations.UnstableApi; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import java.io.File; +import java.io.IOException; + +/** + * InputEntityReader knows how to parse data into {@link InputRow}. + * This class is stateful and a new InputEntityReader should be created per {@link InputEntity}. + * + * @see IntermediateRowParsingReader + * @see TextReader + */ +@UnstableApi +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/InputFormat.java b/core/src/main/java/org/apache/druid/data/input/InputFormat.java new file mode 100644 index 000000000000..8ac986ecb999 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/InputFormat.java @@ -0,0 +1,57 @@ +/* + * 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.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.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.UnstableApi; + +/** + * InputFormat abstracts the file format of input data. + * It creates a {@link InputEntityReader} to read data and parse it into {@link InputRow}. + * The created InputEntityReader is used by {@link InputSourceReader}. + * + * See {@link NestedInputFormat} for nested input formats such as JSON. + */ +@UnstableApi +@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 InputSplit}s. + * + * This method is not being used anywhere for now, but should be considered + * in {@link SplittableInputSource#createSplits} in the future. + */ + @JsonIgnore + boolean isSplittable(); + + InputEntityReader createReader(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 56% 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..27c42b3b06fa 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,32 +19,65 @@ 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; @Nullable private final byte[] raw; + @Nullable + private final String rawJson; + @Nullable private final ParseException parseException; - private InputRowPlusRaw(@Nullable InputRow inputRow, @Nullable byte[] raw, @Nullable ParseException parseException) + public static InputRowListPlusJson of(@Nullable InputRow inputRow, @Nullable byte[] raw) + { + return new InputRowListPlusJson(inputRow == null ? null : Collections.singletonList(inputRow), raw, null, null); + } + + public static InputRowListPlusJson of(@Nullable List inputRows, @Nullable String jsonRaw) + { + return new InputRowListPlusJson(inputRows, null, jsonRaw, null); + } + + 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); + } + + private InputRowListPlusJson(@Nullable List inputRows, @Nullable byte[] raw, @Nullable String rawJson, @Nullable ParseException parseException) { - this.inputRow = inputRow; + this.inputRows = inputRows; this.raw = raw; + this.rawJson = rawJson; this.parseException = parseException; } @Nullable public InputRow getInputRow() { - return inputRow; + return inputRows == null ? null : Iterables.getOnlyElement(inputRows); + } + + @Nullable + public List getInputRows() + { + return inputRows; } /** @@ -62,23 +95,19 @@ public byte[] getRaw() } @Nullable - public ParseException getParseException() - { - return parseException; - } - - public boolean isEmpty() + public String getRawJson() { - return inputRow == null && raw == null && parseException == null; + return rawJson; } - 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 && rawJson == null && parseException == null; } } 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 new file mode 100644 index 000000000000..c908187962e7 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/InputRowSchema.java @@ -0,0 +1,57 @@ +/* + * 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 java.util.List; + +/** + * Schema of {@link InputRow}. + */ +public class InputRowSchema +{ + private final TimestampSpec timestampSpec; + private final DimensionsSpec dimensionsSpec; + private final List metricNames; + + public InputRowSchema(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec, List metricNames) + { + this.timestampSpec = timestampSpec; + this.dimensionsSpec = dimensionsSpec; + this.metricNames = metricNames; + } + + public TimestampSpec getTimestampSpec() + { + return timestampSpec; + } + + public DimensionsSpec getDimensionsSpec() + { + return dimensionsSpec; + } + + public List getMetricNames() + { + return metricNames; + } +} 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..6109c592604c --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/InputSource.java @@ -0,0 +1,81 @@ +/* + * 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.HttpInputSource; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.guice.annotations.UnstableApi; + +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. + * 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);
+ *     }
+ *   }
+ * }
+ */ +@UnstableApi +@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. + */ + boolean isSplittable(); + + /** + * 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(); + + /** + * Creates 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 new file mode 100644 index 000000000000..0a8921422c39 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.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; + +import org.apache.druid.data.input.impl.InputEntityIteratingReader; +import org.apache.druid.guice.annotations.UnstableApi; +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}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. + * + * See {@link InputEntityIteratingReader} as an example. + */ +@UnstableApi +public interface InputSourceReader +{ + CloseableIterator read() throws IOException; + + CloseableIterator sample() 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 new file mode 100644 index 000000000000..41a7df47dd02 --- /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 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/SplitHintSpec.java b/core/src/main/java/org/apache/druid/data/input/SplitHintSpec.java index 69042a74d92f..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,6 +32,8 @@ * * @see FiniteFirehoseFactory#getSplits(SplitHintSpec) * @see FiniteFirehoseFactory#getNumSplits(SplitHintSpec) + * @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/TextReader.java b/core/src/main/java/org/apache/druid/data/input/TextReader.java new file mode 100644 index 000000000000..beb748e978c3 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/TextReader.java @@ -0,0 +1,132 @@ +/* + * 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.Strings; +import org.apache.commons.io.LineIterator; +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.apache.druid.java.util.common.parsers.ParserUtils; + +import java.io.File; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.ArrayList; +import java.util.List; + +/** + * Abstract {@link InputEntityReader} for text format readers such as CSV or JSON. + */ +public abstract class TextReader extends IntermediateRowParsingReader +{ + private final InputRowSchema inputRowSchema; + + public TextReader(InputRowSchema inputRowSchema) + { + this.inputRowSchema = inputRowSchema; + } + + public InputRowSchema getInputRowSchema() + { + return inputRowSchema; + } + + @Override + public CloseableIterator intermediateRowIterator(InputEntity source, File temporaryDirectory) + throws IOException + { + final LineIterator delegate = new LineIterator( + new InputStreamReader(source.open(), StringUtils.UTF8_STRING) + ); + final int numHeaderLines = getNumHeaderLinesToSkip(); + for (int i = 0; i < numHeaderLines && delegate.hasNext(); i++) { + delegate.nextLine(); // skip lines + } + if (needsToProcessHeaderLine() && delegate.hasNext()) { + 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 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}. + */ + @Override + public abstract List parseInputRows(String intermediateRow) 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 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 if {@link #needsToProcessHeaderLine()} = true. + */ + public abstract void processHeaderLine(String line) throws IOException; + + 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/ByteEntity.java b/core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java new file mode 100644 index 000000000000..f356b2084dd0 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java @@ -0,0 +1,64 @@ +/* + * 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.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 +{ + private final ByteBuffer buffer; + + public ByteEntity(ByteBuffer buffer) + { + this.buffer = buffer.duplicate(); + } + + public ByteEntity(byte[] bytes) + { + this(ByteBuffer.wrap(bytes)); + } + + @Override + @Nullable + public URI getUri() + { + return null; + } + + @Override + public InputStream open() + { + return new ByteBufferInputStream(buffer); + } + + @Override + 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 5bd9a2b1c320..5340a4274517 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; @@ -66,17 +67,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() { @@ -107,6 +97,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) { @@ -118,5 +114,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..79c2c6b657d4 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.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.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.InputEntityReader; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +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 InputEntityReader createReader(InputRowSchema inputRowSchema) + { + return new CsvReader( + inputRowSchema, + 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..f21afe153847 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java @@ -0,0 +1,126 @@ +/* + * 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.collect.Iterables; +import com.opencsv.RFC4180Parser; +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; +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.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class CsvReader extends TextReader +{ + private final RFC4180Parser parser = new RFC4180Parser(); + private final boolean findColumnsFromHeader; + private final int skipHeaderRows; + private final Function multiValueFunction; + @Nullable + private List columns; + + CsvReader( + InputRowSchema inputRowSchema, + @Nullable String listDelimiter, + @Nullable List columns, + boolean findColumnsFromHeader, + int skipHeaderRows + ) + { + super(inputRowSchema); + 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 = findColumnsFromHeader ? 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); + } + } else { + Preconditions.checkArgument( + findColumnsFromHeader, + "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 List parseInputRows(String line) throws IOException, ParseException + { + final Map zipped = parseLine(line); + return Collections.singletonList(MapInputRowParser.parse(getInputRowSchema(), zipped)); + } + + @Override + public String toJson(String intermediateRow) throws IOException + { + final Map zipped = parseLine(intermediateRow); + return DEFAULT_JSON_WRITER.writeValueAsString(zipped); + } + + private Map parseLine(String line) throws IOException + { + final String[] parsed = parser.parseLine(line); + return Utils.zipMapPartial( + Preconditions.checkNotNull(columns, "columns"), + Iterables.transform(Arrays.asList(parsed), multiValueFunction) + ); + } + + @Override + public int getNumHeaderLinesToSkip() + { + return skipHeaderRows; + } + + @Override + public boolean needsToProcessHeaderLine() + { + return findColumnsFromHeader; + } + + @Override + public void processHeaderLine(String line) throws IOException + { + if (!findColumnsFromHeader) { + throw new ISE("Don't call this if findColumnsFromHeader = false"); + } + 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/FileEntity.java b/core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java new file mode 100644 index 000000000000..d48345710a3e --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java @@ -0,0 +1,78 @@ +/* + * 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.InputEntity; +import org.apache.druid.utils.CompressionUtils; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; + +public class FileEntity implements InputEntity +{ + private final File file; + + FileEntity(File file) + { + this.file = file; + } + + @Override + public CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) + { + return new CleanableFile() + { + @Override + public File file() + { + return file; + } + + @Override + public void close() + { + // do nothing + } + }; + } + + @Override + public URI getUri() + { + return file.toURI(); + } + + @Override + public InputStream open() throws IOException + { + return CompressionUtils.decompress(new FileInputStream(file), file.getName()); + } + + @Override + public Predicate getFetchRetryCondition() + { + return Predicates.alwaysFalse(); + } +} 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 new file mode 100644 index 000000000000..2fb3cde38d6f --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.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.InputRowListPlusJson; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; + +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 UncheckedIOException(e); + } + } + + @Override + public InputRow next() + { + try { + return firehose.nextRow(); + } + catch (IOException e) { + throw new UncheckedIOException(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 UncheckedIOException(e); + } + } + + @Override + public InputRowListPlusJson next() + { + try { + return firehose.nextRowWithRaw(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public void close() throws IOException + { + firehose.close(); + } + }; + } +} 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 new file mode 100644 index 000000000000..220b5c99dc42 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java @@ -0,0 +1,87 @@ +/* + * 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.InputEntity; +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 HttpEntity implements InputEntity +{ + private final URI uri; + @Nullable + private final String httpAuthenticationUsername; + @Nullable + private final PasswordProvider httpAuthenticationPasswordProvider; + + HttpEntity( + URI uri, + @Nullable String httpAuthenticationUsername, + @Nullable PasswordProvider httpAuthenticationPasswordProvider + ) + { + this.uri = uri; + this.httpAuthenticationUsername = httpAuthenticationUsername; + this.httpAuthenticationPasswordProvider = httpAuthenticationPasswordProvider; + } + + @Override + public URI getUri() + { + return uri; + } + + @Override + public InputStream open() throws IOException + { + return CompressionUtils.decompress( + openURLConnection(uri, httpAuthenticationUsername, httpAuthenticationPasswordProvider).getInputStream(), + uri.toString() + ); + } + + @Override + public Predicate getFetchRetryCondition() + { + 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/HttpInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java new file mode 100644 index 000000000000..7a822a8e9618 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java @@ -0,0 +1,149 @@ +/* + * 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.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; +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 extends AbstractInputSource 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 + protected InputSourceReader formattableReader( + InputRowSchema inputRowSchema, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) + { + return new InputEntityIteratingReader( + inputRowSchema, + inputFormat, + createSplits(inputFormat, null).map(split -> new HttpEntity( + split.get(), + 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); + } + + @Override + public boolean needsFormat() + { + return true; + } +} 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 new file mode 100644 index 000000000000..b726ec78af3b --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.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.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.InputRowListPlusJson; +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.function.Function; +import java.util.stream.Stream; + +/** + * 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 +{ + private final InputRowSchema inputRowSchema; + private final InputFormat inputFormat; + private final Iterator sourceIterator; + private final File temporaryDirectory; + + InputEntityIteratingReader( + InputRowSchema inputRowSchema, + InputFormat inputFormat, + Stream sourceStream, + File temporaryDirectory + ) + { + this.inputRowSchema = inputRowSchema; + this.inputFormat = inputFormat; + this.sourceIterator = sourceStream.iterator(); + this.temporaryDirectory = temporaryDirectory; + } + + @Override + public CloseableIterator read() + { + 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(entity, temporaryDirectory); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + @Override + public CloseableIterator sample() + { + 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.sample(entity, temporaryDirectory); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + private CloseableIterator createIterator(Function> rowPopulator) + { + return CloseableIterators.withEmptyBaggage(sourceIterator).flatMap(rowPopulator); + } +} 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 3a7136b2f8cf..870076d4a7de 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; @@ -67,6 +68,12 @@ public Parser makeParser() return new JSONPathParser(getFlattenSpec(), objectMapper); } + @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..e335d9ba671a --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java @@ -0,0 +1,95 @@ +/* + * 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.InputEntityReader; +import org.apache.druid.data.input.InputRowSchema; +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") @Nullable 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 InputEntityReader createReader(InputRowSchema inputRowSchema) + { + return new JsonReader(inputRowSchema, 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..61526efe6912 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java @@ -0,0 +1,82 @@ +/* + * 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.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; +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.Collections; +import java.util.List; +import java.util.Map; + +public class JsonReader extends TextReader +{ + private final ObjectFlattener flattener; + private final ObjectMapper mapper; + + JsonReader(InputRowSchema inputRowSchema, JSONPathSpec flattenSpec, ObjectMapper mapper) + { + super(inputRowSchema); + this.flattener = ObjectFlatteners.create(flattenSpec, new JSONFlattenerMaker()); + this.mapper = mapper; + } + + @Override + public List parseInputRows(String line) throws IOException, ParseException + { + final JsonNode document = mapper.readValue(line, JsonNode.class); + final Map flattened = flattener.flatten(document); + return Collections.singletonList(MapInputRowParser.parse(getInputRowSchema(), flattened)); + } + + @Override + public String toJson(String intermediateRow) throws IOException + { + final JsonNode document = mapper.readValue(intermediateRow, JsonNode.class); + return DEFAULT_JSON_WRITER.writeValueAsString(document); + } + + @Override + public int getNumHeaderLinesToSkip() + { + return 0; + } + + @Override + public boolean needsToProcessHeaderLine() + { + return false; + } + + @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..54e56136fcfa --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.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.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; +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; +import org.apache.druid.data.input.SplitHintSpec; + +import javax.annotation.Nullable; +import java.io.File; +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 extends AbstractInputSource 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) + { + return Iterators.size(getFileIterator()); + } + + 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 boolean needsFormat() + { + return true; + } + + @Override + protected InputSourceReader formattableReader( + InputRowSchema inputRowSchema, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) + { + return new InputEntityIteratingReader( + inputRowSchema, + inputFormat, + // 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 FileEntity(split.get())), + 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..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 @@ -22,16 +22,18 @@ 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.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; 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 +52,47 @@ 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(InputRowSchema inputRowSchema, Map theMap) throws ParseException + { + return parse(inputRowSchema.getTimestampSpec(), inputRowSchema.getDimensionsSpec(), theMap); + } + + public static InputRow parse( + TimestampSpec timestampSpec, + DimensionsSpec dimensionsSpec, + Map theMap + ) throws ParseException + { + return parse(timestampSpec, dimensionsSpec.getDimensionNames(), dimensionsSpec.getDimensionExclusions(), theMap); + } + + public static InputRow parse( + TimestampSpec timestampSpec, + List dimensions, + Set dimensionExclusions, + Map theMap + ) throws ParseException + { + 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 +107,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..90faea6ce704 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/NestedInputFormat.java @@ -0,0 +1,68 @@ +/* + * 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.data.input.InputFormat; +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 +{ + @Nullable + private final JSONPathSpec flattenSpec; + + protected NestedInputFormat(@Nullable JSONPathSpec flattenSpec) + { + this.flattenSpec = flattenSpec == null ? JSONPathSpec.DEFAULT : flattenSpec; + } + + @Nullable + @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 a90bebbb6b35..adc529903384 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,10 +23,14 @@ 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; +import javax.annotation.Nullable; + +@Deprecated @ExtensionPoint @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "format") @JsonSubTypes(value = { @@ -67,6 +71,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/SplittableInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java new file mode 100644 index 000000000000..ed7de80d5469 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.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.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; + +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/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..c7b7cd8f11be --- /dev/null +++ b/core/src/main/java/org/apache/druid/guice/annotations/UnstableApi.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.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. + * + * 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 + */ +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.SOURCE) +public @interface UnstableApi +{ +} 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..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,34 @@ public static long copyLarge( }, retryCondition, outFile::delete, - numRetries, + numTries, + messageOnRetry + ); + } + catch (Exception e) { + throw new IOException(e); + } + } + + public static long copyLarge( + InputStream inputStream, + File outFile, + byte[] fetchBuffer, + Predicate retryCondition, + int numTries, + String messageOnRetry + ) throws IOException + { + try { + return RetryUtils.retry( + () -> { + try (OutputStream out = new FileOutputStream(outFile)) { + return IOUtils.copyLarge(inputStream, out, fetchBuffer); + } + }, + retryCondition, + outFile::delete, + numTries, messageOnRetry ); } 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..0015435f7027 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.TextReader; 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 = TextReader.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..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 @@ -20,10 +20,84 @@ package org.apache.druid.java.util.common.parsers; 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; /** */ 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(); + } + }; + } + + 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/core/src/main/java/org/apache/druid/timeline/DataSegment.java b/core/src/main/java/org/apache/druid/timeline/DataSegment.java index 1fb5efae3093..eec03a26e589 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..c163de1ecb3d --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java @@ -0,0 +1,180 @@ +/* + * 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.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.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Stream; + +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 UnimplementedInputFormatCsvParseSpec( + 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( + new InputRowSchema( + inputRowParser.getParseSpec().getTimestampSpec(), + inputRowParser.getParseSpec().getDimensionsSpec(), + Collections.emptyList() + ), + 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 UnimplementedInputFormatCsvParseSpec extends CSVParseSpec + { + private UnimplementedInputFormatCsvParseSpec( + 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 FiniteFirehoseFactory + { + 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 + } + }; + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec) + { + return null; + } + + @Override + public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) + { + return 0; + } + + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + return null; + } + } +} 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..8d2d688e9868 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.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.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.IOException; +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 testComma() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Column[a,] has a comma, it cannot"); + new CsvInputFormat(Collections.singletonList("a,"), ",", false, 0); + } +} 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..c988b1200267 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java @@ -0,0 +1,292 @@ +/* + * 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.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; +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +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; + +public class CsvReaderTest +{ + 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 + public void testWithoutHeaders() throws IOException + { + final ByteEntity 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 ByteEntity 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 ByteEntity 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 ByteEntity source = writeData( + ImmutableList.of( + "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" + ) + ); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), null, true, 1); + assertResult(source, format); + } + + @Test + public void testMultiValues() throws IOException + { + final ByteEntity 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 InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA); + 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); + } + } + + @Test + public void testQuotes() throws IOException + { + 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", + "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 ByteEntity 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 ByteEntity 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 ByteEntity(outputStream.toByteArray()); + } + + private void assertResult(ByteEntity source, CsvInputFormat format) throws IOException + { + final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA); + 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/InputEntityIteratingReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java new file mode 100644 index 000000000000..6d55f14d103c --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java @@ -0,0 +1,97 @@ +/* + * 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.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; +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.Collections; +import java.util.List; + +public class InputEntityIteratingReaderTest +{ + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Test + public void test() throws IOException + { + final int numFiles = 5; + final List files = new ArrayList<>(); + 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)) { + 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 InputEntityIteratingReader firehose = new InputEntityIteratingReader( + 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"), + null, + false, + 0 + ), + files.stream().flatMap(file -> ImmutableList.of(new FileEntity(file)).stream()), + 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++; + } + Assert.assertEquals(numFiles, i); + } + } +} 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..5530b946864d --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/JsonInputFormatTest.java @@ -0,0 +1,59 @@ +/* + * 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.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; +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..f91fe2d37ba3 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/JsonReaderTest.java @@ -0,0 +1,131 @@ +/* + * 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.InputEntityReader; +import org.apache.druid.data.input.InputRow; +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; +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 ByteEntity source = new ByteEntity( + StringUtils.toUtf8("{\"timestamp\":\"2019-01-01\",\"bar\":null,\"foo\":\"x\",\"baz\":4,\"o\":{\"mg\":1}}") + ); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), + 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()); + 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()); + numActualIterations++; + } + Assert.assertEquals(numExpectedIterations, numActualIterations); + } + } + + @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 ByteEntity source = new ByteEntity( + StringUtils.toUtf8("{\"timestamp\":\"2019-01-01\",\"something_else\": {\"foo\": \"test\"}}") + ); + + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), + 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("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/LocalInputSourceTest.java b/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java new file mode 100644 index 000000000000..c4f0ab54ca05 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.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.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..2669a968d045 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.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 org.apache.druid.data.input.FiniteFirehoseFactory; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; + +import javax.annotation.Nullable; +import java.util.stream.Stream; + +public class NoopFirehoseFactory implements FiniteFirehoseFactory +{ + @Override + public String toString() + { + return "NoopFirehoseFactory{}"; + } + + @Override + public Stream getSplits(@Nullable SplitHintSpec splitHintSpec) + { + throw new UnsupportedOperationException(); + } + + @Override + public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) + { + throw new UnsupportedOperationException(); + } + + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + throw new UnsupportedOperationException(); + } +} 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..8520671645b9 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputFormat.java @@ -0,0 +1,45 @@ +/* + * 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.InputEntityReader; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; + +public class NoopInputFormat implements InputFormat +{ + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public InputEntityReader createReader(InputRowSchema inputRowSchema) + { + 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..d1d18a827a07 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/NoopInputSource.java @@ -0,0 +1,59 @@ +/* + * 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.InputFormat; +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 +{ + @Override + public String toString() + { + return "NoopInputSource{}"; + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public boolean needsFormat() + { + return false; + } + + @Override + public InputSourceReader reader( + InputRowSchema inputRowSchema, + @Nullable InputFormat inputFormat, + @Nullable File temporaryDirectory + ) + { + throw new UnsupportedOperationException(); + } +} 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/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 2f95ebbd057d..000000000000 --- a/core/src/test/java/org/apache/druid/java/util/common/parsers/CSVParserTest.java +++ /dev/null @@ -1,98 +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.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -public class CSVParserTest -{ - @Test - public void testBasic() throws IOException - { - 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() throws IOException - { - 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); - } -} 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/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index bcde539f2603..1e9ad30f148c 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/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/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index 133c7c35bc82..348d0a528fe7 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 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/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..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 @@ -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,8 @@ 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().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 0861a915a7a3..338b857c05e9 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().getTimestampSpec()) + .withDimensionsSpec(config.getSchema().getDataSchema().getDimensionsSpec()) .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().getDimensionsSpec()); } @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().getDimensionsSpec()); } @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().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 3287925104b5..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 @@ -294,9 +294,9 @@ public IndexMergerV9 getIndexMergerV9() return indexMergerV9; } - public File getFirehoseTemporaryDir() + 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/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/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 51b44f18d117..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 @@ -22,22 +22,34 @@ 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.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.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 +65,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 +74,12 @@ 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.query.aggregation.AggregatorFactory; 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; @@ -106,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; @@ -116,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 { @@ -461,11 +478,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 +493,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 +517,8 @@ public TaskStatus runTask(final TaskToolbox toolbox) toolbox, dataSchema, allocateSpec, - firehoseFactory, - firehoseTempDir, + inputSource, + tmpDir, partitionsSpec ); } @@ -585,8 +604,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 +635,8 @@ private Map> determineShardSpecs( return createShardSpecsFromInput( jsonMapper, ingestionSchema, - firehoseFactory, - firehoseTempDir, + inputSource, + tmpDir, granularitySpec, nonNullPartitionsSpec, determineIntervals @@ -628,8 +647,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 +660,8 @@ private Map> createShardSpecsFromInput final Map> hllCollectors = collectIntervalsAndShardSpecs( jsonMapper, ingestionSchema, - firehoseFactory, - firehoseTempDir, + inputSource, + tmpDir, granularitySpec, nonNullPartitionsSpec, determineIntervals @@ -693,8 +712,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 +723,25 @@ 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( + new InputRowSchema( + ingestionSchema.getDataSchema().getTimestampSpec(), + ingestionSchema.getDataSchema().getDimensionsSpec(), + metricsNames + ), + 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 +861,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 +910,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 +1031,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 +1054,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 using inputFormat instead of parser."); + } + this.dataSchema = dataSchema; this.ioConfig = ioConfig; this.tuningConfig = tuningConfig == null ? new IndexTuningConfig() : tuningConfig; @@ -1043,30 +1092,90 @@ 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 using 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( + (FiniteFirehoseFactory) 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 82% 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..65c4291438ae 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,11 @@ 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.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.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.stats.RowIngestionMeters; @@ -30,7 +32,9 @@ 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.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; @@ -42,12 +46,15 @@ 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 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 +63,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 +79,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 +89,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 +100,24 @@ public SegmentsAndMetadata process( ? (DynamicPartitionsSpec) partitionsSpec : null; final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); - try ( - final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) - ) { - while (firehose.hasMore()) { + final List metricsNames = Arrays.stream(dataSchema.getAggregators()) + .map(AggregatorFactory::getName) + .collect(Collectors.toList()); + final InputSourceReader inputSourceReader = dataSchema.getTransformSpec().decorate( + inputSource.reader( + new InputRowSchema( + dataSchema.getTimestampSpec(), + dataSchema.getDimensionsSpec(), + metricsNames + ), + 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 +143,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..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,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.InputFormat; +import org.apache.druid.data.input.InputSource; 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..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 @@ -21,6 +21,11 @@ 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; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.IngestionSpec; @@ -39,6 +44,18 @@ 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) { + if (!(ioConfig.getInputSource() instanceof FirehoseFactoryToInputSourceAdaptor)) { + throw new IAE("Cannot use parser and inputSource together. Try using 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..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,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.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.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..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,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.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; 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..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,8 +21,10 @@ 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.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; import java.io.IOException; @@ -38,7 +40,7 @@ class PartialSegmentGenerateParallelIndexTaskRunner extends ParallelIndexPhaseRunner { private final ParallelIndexIngestionSpec ingestionSchema; - private final FiniteFirehoseFactory baseFirehoseFactory; + private final SplittableInputSource baseInputSource; PartialSegmentGenerateParallelIndexTaskRunner( TaskToolbox toolbox, @@ -58,7 +60,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 +74,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,17 +96,28 @@ ParallelIndexIngestionSpec getIngestionSchema() } @VisibleForTesting - FiniteFirehoseFactory getBaseFirehoseFactory() + SplittableInputSource getBaseInputSource() { - return baseFirehoseFactory; + return baseInputSource; } 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( - baseFirehoseFactory.withSplit(split), + firehoseFactory, + inputSource, + 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..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,8 +21,10 @@ 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.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; import java.io.IOException; @@ -40,7 +42,7 @@ class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner { private final ParallelIndexIngestionSpec ingestionSchema; - private final FiniteFirehoseFactory baseFirehoseFactory; + private final SplittableInputSource baseInputSource; SinglePhaseParallelIndexTaskRunner( TaskToolbox toolbox, @@ -60,7 +62,9 @@ class SinglePhaseParallelIndexTaskRunner indexingServiceClient ); this.ingestionSchema = ingestionSchema; - this.baseFirehoseFactory = (FiniteFirehoseFactory) ingestionSchema.getIOConfig().getFirehoseFactory(); + this.baseInputSource = (SplittableInputSource) ingestionSchema.getIOConfig().getNonNullInputSource( + ingestionSchema.getDataSchema().getParser() + ); } @Override @@ -79,18 +83,33 @@ 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 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(), @@ -98,7 +117,9 @@ SubTaskSpec newTaskSpec(InputSplit split) new ParallelIndexIngestionSpec( ingestionSchema.getDataSchema(), new ParallelIndexIOConfig( - baseFirehoseFactory.withSplit(split), + firehoseFactory, + inputSource, + 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..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 @@ -26,9 +26,10 @@ 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.InputRowSchema; +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,8 +52,10 @@ 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.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; @@ -79,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; @@ -208,11 +212,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 +230,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 +392,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 +426,33 @@ 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( + new InputRowSchema( + ingestionSchema.getDataSchema().getTimestampSpec(), + ingestionSchema.getDataSchema().getDimensionsSpec(), + metricsNames + ), + 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/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 8b59dabc2f37..dd4911a33704 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 @@ -228,7 +228,6 @@ public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) throw new RuntimeException(e); } }); - } } 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/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/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 6deef192aa27..b1a15d37cdb8 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 @@ -103,16 +103,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 5f25458f4bd4..c76bc9c0fbf4 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 @@ -37,11 +37,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; @@ -104,7 +102,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; @@ -1062,13 +1059,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 1372f2a7abe5..f0594724d325 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,12 @@ 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.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; @@ -116,47 +119,48 @@ 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, 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 @@ -199,10 +203,12 @@ public void testDeterminePartitions() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, null, + null, createTuningConfigWithMaxRowsPerSegment(2, true), false ), @@ -247,6 +253,7 @@ public void testTransformSpec() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, new CSVParseSpec( @@ -366,6 +373,7 @@ public void testWithArbitraryGranularity() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -404,6 +412,7 @@ public void testIntervalBucketing() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -443,10 +452,12 @@ public void testNumShardsProvided() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, null, + null, createTuningConfigWithNumShards(1, null, true), false ), @@ -483,10 +494,12 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, null, + null, createTuningConfigWithNumShards(2, ImmutableList.of("dim"), true), false ), @@ -559,10 +572,12 @@ public void testAppendToExisting() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, null, + null, createTuningConfigWithMaxRowsPerSegment(2, false), true ), @@ -607,6 +622,7 @@ public void testIntervalNotSpecified() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -661,19 +677,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, @@ -715,19 +724,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, @@ -775,6 +777,7 @@ public void testWithSmallMaxTotalRows() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -821,6 +824,7 @@ public void testPerfectRollup() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -867,6 +871,7 @@ public void testBestEffortRollup() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -932,19 +937,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, @@ -987,19 +985,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, @@ -1058,7 +1049,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, @@ -1082,22 +1073,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 @@ -1183,7 +1169,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, @@ -1207,22 +1193,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"), @@ -1264,7 +1245,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 ) ); @@ -1301,7 +1282,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, @@ -1325,22 +1306,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"), @@ -1375,7 +1351,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( @@ -1402,7 +1378,6 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc Assert.assertEquals(expectedUnparseables, reportData.getUnparseableEvents()); } - @Test public void testCsvWithHeaderOfEmptyColumns() throws Exception { @@ -1430,19 +1405,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, @@ -1502,19 +1470,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, @@ -1566,6 +1527,7 @@ public void testOverwriteWithSameSegmentGranularity() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -1634,6 +1596,7 @@ public void testOverwriteWithDifferentSegmentGranularity() throws Exception null, null, createIngestionSpec( + useInputFormatApi, jsonMapper, tmpDir, null, @@ -1730,7 +1693,7 @@ static IndexTuningConfig createTuningConfig( boolean reportParseException ) { - return new IndexTask.IndexTuningConfig( + return new IndexTuningConfig( null, maxRowsPerSegment, maxRowsInMemory, @@ -1767,16 +1730,38 @@ private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOExcep ); } - public static IndexTask.IndexIngestionSpec createIngestionSpec( + static IndexIngestionSpec createIngestionSpec( + ObjectMapper objectMapper, + File baseDir, + @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, - ParseSpec parseSpec, + @Nullable ParseSpec parseSpec, GranularitySpec granularitySpec, IndexTuningConfig tuningConfig, boolean appendToExisting ) { return createIngestionSpec( + useInputFormatApi, objectMapper, baseDir, parseSpec, @@ -1787,46 +1772,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/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 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..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,6 +28,7 @@ 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.impl.ParseSpec; @@ -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..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 @@ -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,13 @@ @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, true} ); } @@ -103,11 +105,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 +161,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 +175,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 +215,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 +422,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..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 @@ -21,10 +21,12 @@ 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.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.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.NoopInputFormat; +import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; @@ -90,8 +92,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 +128,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 +167,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 +183,7 @@ private ParallelIndexSupervisorTask newTask( Granularities.MINUTE, interval == null ? null : Collections.singletonList(interval) ), - null, - getObjectMapper() + null ), ioConfig, new ParallelIndexTuningConfig( @@ -236,12 +236,12 @@ private TestInput(long runTime, TaskState finalState) } } - private static class TestFirehoseFactory implements FiniteFirehoseFactory + private static class TestInputSource extends AbstractInputSource implements SplittableInputSource { private final List> splits; @SafeVarargs - private TestFirehoseFactory(Pair... inputSpecs) + private TestInputSource(Pair... inputSpecs) { splits = new ArrayList<>(); for (Pair inputSpec : inputSpecs) { @@ -252,27 +252,33 @@ 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 boolean needsFormat() { - return new TestFirehoseFactory(split); + return false; } } @@ -331,9 +337,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 +347,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 +431,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..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 @@ -22,11 +22,13 @@ 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.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.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.NoopInputFormat; +import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; @@ -85,19 +87,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 +138,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 +163,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 +205,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 +382,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 +417,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 +427,7 @@ private TestSupervisorTask newTask( Granularities.MINUTE, interval == null ? null : Collections.singletonList(interval) ), - null, - getObjectMapper() + null ), ioConfig, new ParallelIndexTuningConfig( @@ -457,31 +470,37 @@ private TestSupervisorTask newTask( ); } - private static class TestFirehose implements FiniteFirehoseFactory + private static class TestInputSource extends AbstractInputSource 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 boolean needsFormat() { - return new TestFirehose(Collections.singletonList(split.get())); + return false; } } @@ -539,9 +558,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 +568,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 +618,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 26747cbcc10a..0b12fbba6a40 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 @@ -22,8 +22,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; 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; @@ -70,21 +71,25 @@ @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, 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 @@ -121,13 +126,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); @@ -160,14 +159,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); @@ -234,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); @@ -260,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); @@ -282,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, @@ -342,21 +313,27 @@ public void testAppendToExisting() throws Exception Assert.assertEquals(new HashSet<>(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, @@ -391,36 +368,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( @@ -476,9 +490,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(), @@ -486,7 +500,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 5e4099eac7b0..38d25dd2b1da 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; @@ -35,11 +33,20 @@ 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.InputRowListPlusJson; +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.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; @@ -270,52 +281,99 @@ private static InputRow ir(String dt, String dim1, String dim2, float met) ); } - private static class MockExceptionalFirehoseFactory implements FirehoseFactory + private static class MockExceptionInputSource extends AbstractInputSource { @Override - public Firehose connect(InputRowParser parser, File temporaryDirectory) + protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) { - return new Firehose() + return new InputSourceReader() { @Override - public boolean hasMore() + public CloseableIterator read() { - return true; + return new CloseableIterator() + { + @Override + public void close() + { + } + + @Override + public boolean hasNext() + { + return true; + } + + @Override + public InputRow next() + { + throw new RuntimeException("HA HA HA"); + } + }; } - @Nullable @Override - public InputRow nextRow() + public CloseableIterator sample() { - throw new RuntimeException("HA HA HA"); + throw new UnsupportedOperationException(); } + }; + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public boolean needsFormat() + { + return false; + } + } + private static class MockInputSource extends AbstractInputSource + { + @Override + protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) + { + return new InputSourceReader() + { @Override - public void close() + 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 - { - @JsonProperty - private boolean usedByRealtimeIdxTask; + @Override + public boolean isSplittable() + { + return false; + } - @JsonCreator - public MockFirehoseFactory(@JsonProperty("usedByRealtimeIdxTask") boolean usedByRealtimeIdxTask) + @Override + public boolean needsFormat() { - this.usedByRealtimeIdxTask = usedByRealtimeIdxTask; + return false; } + } + 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() { @@ -396,8 +454,9 @@ 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(MockFirehoseFactory.class, "mockFirehoseFactory"), + new NamedType(MockInputSource.class, "mockInputSource"), + new NamedType(NoopInputFormat.class, "noopInputFormat") ); testDerbyConnector.createTaskTables(); testDerbyConnector.createSegmentTable(); @@ -667,17 +726,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, @@ -761,7 +820,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, @@ -1179,17 +1238,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, @@ -1285,17 +1344,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, @@ -1393,14 +1452,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/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/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..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,17 +19,19 @@ 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; 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; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -44,7 +46,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 +57,7 @@ public class Transformer .makeMatcher( RowBasedColumnSelectorFactory.create( rowSupplierForValueMatcher::get, - rowSignature + null ) ); } else { @@ -93,6 +95,36 @@ public InputRow transform(@Nullable final InputRow row) return transformedRow; } + @Nullable + public InputRowListPlusJson transform(@Nullable final InputRowListPlusJson row) + { + if (row == null) { + return null; + } + + final InputRowListPlusJson transformedRow; + + if (transforms.isEmpty()) { + transformedRow = row; + } else { + 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) { + 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/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) { 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..25944f6b6c27 --- /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.InputRowListPlusJson; +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..f964055bed23 --- /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.InputFormat; +import org.apache.druid.data.input.InputSource; + +/** + * 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..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 @@ -21,14 +21,17 @@ 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.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Strings; 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 +40,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 +58,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 +117,36 @@ 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; + @VisibleForTesting + 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 +160,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("timestampSpec") + private TimestampSpec getGivenTimestampSpec() { - return parser; + return timestampSpec; } - @JsonIgnore - public InputRowParser getParser() + public TimestampSpec getTimestampSpec() { - 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("dimensionsSpec") + private DimensionsSpec getGivenDimensionsSpec() + { + 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 getDimensionsSpec() + { + if (dimensionsSpec == null) { + dimensionsSpec = computeDimensionsSpec( + getTimestampSpec(), + Preconditions.checkNotNull(getParser(), "inputRowParser").getParseSpec().getDimensionsSpec(), + aggregators + ); } - - return cachedParser; + return dimensionsSpec; } @JsonProperty("metricsSpec") @@ -212,14 +249,65 @@ 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(); + parseSpec = parseSpec.withDimensionsSpec( + computeDimensionsSpec(parseSpec.getTimestampSpec(), parseSpec.getDimensionsSpec(), aggregators) + ); + 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 +315,13 @@ public String toString() { return "DataSchema{" + "dataSource='" + dataSource + '\'' + - ", parser=" + parser + ", aggregators=" + Arrays.toString(aggregators) + ", granularitySpec=" + granularitySpec + ", transformSpec=" + transformSpec + + ", parserMap=" + parserMap + + ", timestampSpec=" + timestampSpec + + ", dimensionsSpec=" + dimensionsSpec + + ", inputRowParser=" + inputRowParser + '}'; } } 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..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 @@ -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.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; @@ -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 = HttpEntity.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/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/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..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.getParser()) + .withTimestampSpec(schema.getTimestampSpec()) .withQueryGranularity(schema.getGranularitySpec().getQueryGranularity()) - .withDimensionsSpec(schema.getParser()) + .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 437efd313ce0..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.getParser().getParseSpec().getDimensionsSpec().getDimensionExclusions() + schema.getDimensionsSpec().getDimensionExclusions() ); } @@ -361,7 +361,6 @@ private void testInvalidWhitespaceDatasourceHelper(String dataSource, String inv } } - @Test public void testSerde() throws Exception { @@ -471,13 +470,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/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()); 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");