diff --git a/core/src/main/java/org/apache/druid/data/input/InputFileAttribute.java b/core/src/main/java/org/apache/druid/data/input/InputFileAttribute.java new file mode 100644 index 000000000000..5cb90cf57b66 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/InputFileAttribute.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; + +/** + * A class storing some attributes of an input file. + * This information is used to make splits in the parallel indexing. + * + * @see SplitHintSpec + * @see org.apache.druid.data.input.impl.SplittableInputSource + */ +public class InputFileAttribute +{ + /** + * The size of the input file. + */ + private final long size; + + public InputFileAttribute(long size) + { + this.size = size; + } + + public long getSize() + { + return size; + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/InputSplit.java b/core/src/main/java/org/apache/druid/data/input/InputSplit.java index 7ceb2b0410e9..a7db34505b94 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSplit.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSplit.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; /** - * Input unit for distributed batch ingestion. Used in {@link FiniteFirehoseFactory}. + * Input unit for distributed batch ingestion. Used in {@link org.apache.druid.data.input.impl.SplittableInputSource}. * An {@link InputSplit} represents the input data processed by a {@code org.apache.druid.indexing.common.task.Task}. */ public class InputSplit diff --git a/core/src/main/java/org/apache/druid/data/input/MaxSizeSplitHintSpec.java b/core/src/main/java/org/apache/druid/data/input/MaxSizeSplitHintSpec.java new file mode 100644 index 000000000000..4f810e2c629e --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/MaxSizeSplitHintSpec.java @@ -0,0 +1,119 @@ +/* + * 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.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Objects; +import java.util.function.Function; + +/** + * A SplitHintSpec that can create splits of multiple files. + * A split created by this class can have one or more input files. + * If there is only one file in the split, its size can be larger than {@link #maxSplitSize}. + * If there are two or more files in the split, their total size cannot be larger than {@link #maxSplitSize}. + */ +public class MaxSizeSplitHintSpec implements SplitHintSpec +{ + public static final String TYPE = "maxSize"; + + @VisibleForTesting + static final long DEFAULT_MAX_SPLIT_SIZE = 512 * 1024 * 1024; + + private final long maxSplitSize; + + @JsonCreator + public MaxSizeSplitHintSpec(@JsonProperty("maxSplitSize") @Nullable Long maxSplitSize) + { + this.maxSplitSize = maxSplitSize == null ? DEFAULT_MAX_SPLIT_SIZE : maxSplitSize; + } + + @JsonProperty + public long getMaxSplitSize() + { + return maxSplitSize; + } + + @Override + public Iterator> split(Iterator inputIterator, Function inputAttributeExtractor) + { + return new Iterator>() + { + private T peeking; + + @Override + public boolean hasNext() + { + return peeking != null || inputIterator.hasNext(); + } + + @Override + public List next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final List current = new ArrayList<>(); + long splitSize = 0; + while (splitSize < maxSplitSize && (peeking != null || inputIterator.hasNext())) { + if (peeking == null) { + peeking = inputIterator.next(); + } + final long size = inputAttributeExtractor.apply(peeking).getSize(); + if (current.isEmpty() || splitSize + size < maxSplitSize) { + current.add(peeking); + splitSize += size; + peeking = null; + } else { + break; + } + } + assert !current.isEmpty(); + return current; + } + }; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MaxSizeSplitHintSpec that = (MaxSizeSplitHintSpec) o; + return maxSplitSize == that.maxSplitSize; + } + + @Override + public int hashCode() + { + return Objects.hash(maxSplitSize); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/SegmentsSplitHintSpec.java b/core/src/main/java/org/apache/druid/data/input/SegmentsSplitHintSpec.java index 6cca8ab71915..bfb4e8efa791 100644 --- a/core/src/main/java/org/apache/druid/data/input/SegmentsSplitHintSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/SegmentsSplitHintSpec.java @@ -23,10 +23,19 @@ import com.fasterxml.jackson.annotation.JsonProperty; import javax.annotation.Nullable; +import java.util.Iterator; +import java.util.List; import java.util.Objects; +import java.util.function.Function; /** - * {@link SplitHintSpec} for IngestSegmentFirehoseFactory. + * {@link SplitHintSpec} for IngestSegmentFirehoseFactory and DruidInputSource. + * + * In DruidInputSource, this spec is converted into {@link MaxSizeSplitHintSpec}. As a result, its {@link #split} + * method is never called (IngestSegmentFirehoseFactory creates splits on its own instead of calling the + * {@code split()} method). This doesn't necessarily mean this class is deprecated in favor of the MaxSizeSplitHintSpec. + * We may want to create more optimized splits in the future. For example, segments can be split to maximize the rollup + * ratio if the segments have different sets of columns or even different value ranges of columns. */ public class SegmentsSplitHintSpec implements SplitHintSpec { @@ -41,9 +50,7 @@ public class SegmentsSplitHintSpec implements SplitHintSpec private final long maxInputSegmentBytesPerTask; @JsonCreator - public SegmentsSplitHintSpec( - @JsonProperty("maxInputSegmentBytesPerTask") @Nullable Long maxInputSegmentBytesPerTask - ) + public SegmentsSplitHintSpec(@JsonProperty("maxInputSegmentBytesPerTask") @Nullable Long maxInputSegmentBytesPerTask) { this.maxInputSegmentBytesPerTask = maxInputSegmentBytesPerTask == null ? DEFAULT_MAX_INPUT_SEGMENT_BYTES_PER_TASK @@ -56,6 +63,13 @@ public long getMaxInputSegmentBytesPerTask() return maxInputSegmentBytesPerTask; } + @Override + public Iterator> split(Iterator inputIterator, Function inputAttributeExtractor) + { + // This method is not supported currently, but we may want to implement in the future to create optimized splits. + throw new UnsupportedOperationException(); + } + @Override public boolean equals(Object o) { 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 be4a07274e69..3c40ff840a77 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 @@ -24,6 +24,10 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.data.input.impl.SplittableInputSource; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; + /** * In native parallel indexing, the supervisor task partitions input data into splits and assigns each of them * to a single sub task. How to create splits could mainly depend on the input file format, but sometimes druid users @@ -37,8 +41,16 @@ */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { - @Type(name = SegmentsSplitHintSpec.TYPE, value = SegmentsSplitHintSpec.class) + @Type(name = SegmentsSplitHintSpec.TYPE, value = SegmentsSplitHintSpec.class), + @Type(name = MaxSizeSplitHintSpec.TYPE, value = MaxSizeSplitHintSpec.class) }) public interface SplitHintSpec { + /** + * Returns an iterator of splits. A split has a list of files of the type {@link T}. + * + * @param inputIterator that returns input files. + * @param inputAttributeExtractor to create {@link InputFileAttribute} for each input file. + */ + Iterator> split(Iterator inputIterator, Function inputAttributeExtractor); } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CloudObjectInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/CloudObjectInputSource.java index c8b7a82bf344..d27bf4faa01d 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CloudObjectInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CloudObjectInputSource.java @@ -33,12 +33,13 @@ 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 abstract class CloudObjectInputSource extends AbstractInputSource - implements SplittableInputSource +public abstract class CloudObjectInputSource extends AbstractInputSource + implements SplittableInputSource> { private final List uris; private final List prefixes; @@ -90,7 +91,7 @@ public List getObjects() * Create the correct {@link InputEntity} for this input source given a split on a {@link CloudObjectLocation}. This * is called internally by {@link #formattableReader} and operates on the output of {@link #createSplits}. */ - protected abstract T createEntity(InputSplit split); + protected abstract InputEntity createEntity(CloudObjectLocation location); /** * Create a stream of {@link CloudObjectLocation} splits by listing objects that appear under {@link #prefixes} using @@ -98,22 +99,24 @@ public List getObjects() * only if {@link #prefixes} is set, otherwise the splits are created directly from {@link #uris} or {@link #objects}. * Calling if {@link #prefixes} is not set is likely to either lead to an empty iterator or null pointer exception. */ - protected abstract Stream> getPrefixesSplitStream(); + protected abstract Stream>> getPrefixesSplitStream(SplitHintSpec splitHintSpec); @Override - public Stream> createSplits( + public Stream>> createSplits( InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec ) { if (!CollectionUtils.isNullOrEmpty(objects)) { - return objects.stream().map(InputSplit::new); + return objects.stream().map(object -> new InputSplit<>(Collections.singletonList(object))); } if (!CollectionUtils.isNullOrEmpty(uris)) { - return uris.stream().map(CloudObjectLocation::new).map(InputSplit::new); + return uris.stream() + .map(CloudObjectLocation::new) + .map(object -> new InputSplit<>(Collections.singletonList(object))); } - return getPrefixesSplitStream(); + return getPrefixesSplitStream(getSplitHintSpecOrDefault(splitHintSpec)); } @Override @@ -127,7 +130,7 @@ public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec sp return uris.size(); } - return Ints.checkedCast(getPrefixesSplitStream().count()); + return Ints.checkedCast(getPrefixesSplitStream(getSplitHintSpecOrDefault(splitHintSpec)).count()); } @Override @@ -146,7 +149,7 @@ protected InputSourceReader formattableReader( return new InputEntityIteratingReader( inputRowSchema, inputFormat, - createSplits(inputFormat, null).map(this::createEntity), + createSplits(inputFormat, null).flatMap(split -> split.get().stream()).map(this::createEntity).iterator(), temporaryDirectory ); } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java index 99780caf0408..21480fd1b1d8 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java @@ -115,7 +115,7 @@ protected InputSourceReader formattableReader( split.get(), httpAuthenticationUsername, httpAuthenticationPasswordProvider - )), + )).iterator(), temporaryDirectory ); } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InlineInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/InlineInputSource.java index 244facec5501..a99f7cdc4447 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InlineInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InlineInputSource.java @@ -69,7 +69,7 @@ protected InputSourceReader formattableReader( return new InputEntityIteratingReader( inputRowSchema, inputFormat, - Stream.of(new ByteEntity(StringUtils.toUtf8(data))), + Stream.of(new ByteEntity(StringUtils.toUtf8(data))).iterator(), temporaryDirectory ); } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java index 64581763b22a..d52097c3182c 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java @@ -31,8 +31,8 @@ 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 @@ -48,23 +48,23 @@ public class InputEntityIteratingReader implements InputSourceReader public InputEntityIteratingReader( InputRowSchema inputRowSchema, InputFormat inputFormat, - Stream sourceStream, + Iterator sourceIterator, File temporaryDirectory ) { - this(inputRowSchema, inputFormat, CloseableIterators.withEmptyBaggage(sourceStream.iterator()), temporaryDirectory); + this(inputRowSchema, inputFormat, CloseableIterators.withEmptyBaggage(sourceIterator), temporaryDirectory); } public InputEntityIteratingReader( InputRowSchema inputRowSchema, InputFormat inputFormat, - CloseableIterator sourceIterator, + CloseableIterator sourceCloseableIterator, File temporaryDirectory ) { this.inputRowSchema = inputRowSchema; this.inputFormat = inputFormat; - this.sourceIterator = sourceIterator; + this.sourceIterator = (CloseableIterator) sourceCloseableIterator; this.temporaryDirectory = temporaryDirectory; } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java index c6447a5842d6..65b0f6195b14 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java @@ -21,81 +21,153 @@ 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.Iterators; import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOCase; +import org.apache.commons.io.filefilter.AndFileFilter; +import org.apache.commons.io.filefilter.IOFileFilter; +import org.apache.commons.io.filefilter.NameFileFilter; +import org.apache.commons.io.filefilter.NotFileFilter; 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.InputFileAttribute; 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.java.util.common.IAE; +import org.apache.druid.utils.CollectionUtils; +import org.apache.druid.utils.Streams; import javax.annotation.Nullable; import java.io.File; +import java.util.Collections; +import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Objects; -import java.util.Spliterator; -import java.util.Spliterators; +import java.util.Set; +import java.util.stream.Collectors; import java.util.stream.Stream; -import java.util.stream.StreamSupport; -public class LocalInputSource extends AbstractInputSource implements SplittableInputSource +public class LocalInputSource extends AbstractInputSource implements SplittableInputSource> { + @Nullable private final File baseDir; + @Nullable private final String filter; + private final Set files; @JsonCreator public LocalInputSource( - @JsonProperty("baseDir") File baseDir, - @JsonProperty("filter") String filter + @JsonProperty("baseDir") @Nullable File baseDir, + @JsonProperty("filter") @Nullable String filter, + @JsonProperty("files") @Nullable Set files ) { this.baseDir = baseDir; - this.filter = filter; + this.filter = baseDir != null ? Preconditions.checkNotNull(filter, "filter") : filter; + this.files = files == null ? Collections.emptySet() : files; + + if (baseDir == null && CollectionUtils.isNullOrEmpty(files)) { + throw new IAE("At least one of baseDir or files should be specified"); + } + } + + public LocalInputSource(File baseDir, String filter) + { + this(baseDir, filter, null); } + @Nullable @JsonProperty public File getBaseDir() { return baseDir; } + @Nullable @JsonProperty public String getFilter() { return filter; } + @JsonProperty + public Set getFiles() + { + return files; + } + @Override - public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + public Stream>> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) { - return StreamSupport.stream(Spliterators.spliteratorUnknownSize(getFileIterator(), Spliterator.DISTINCT), false) - .map(InputSplit::new); + return Streams.sequentialStreamFrom(getSplitFileIterator(getSplitHintSpecOrDefault(splitHintSpec))) + .map(InputSplit::new); } @Override public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) { - return Iterators.size(getFileIterator()); + return Iterators.size(getSplitFileIterator(getSplitHintSpecOrDefault(splitHintSpec))); + } + + private Iterator> getSplitFileIterator(SplitHintSpec splitHintSpec) + { + final Iterator fileIterator = getFileIterator(); + return splitHintSpec.split(fileIterator, file -> new InputFileAttribute(file.length())); } - private Iterator getFileIterator() + @VisibleForTesting + Iterator getFileIterator() { - return FileUtils.iterateFiles( - Preconditions.checkNotNull(baseDir).getAbsoluteFile(), - new WildcardFileFilter(filter), - TrueFileFilter.INSTANCE + return Iterators.concat( + getDirectoryListingIterator(), + getFilesListIterator() ); } + private Iterator getDirectoryListingIterator() + { + if (baseDir == null) { + return Collections.emptyIterator(); + } else { + final IOFileFilter fileFilter; + if (files == null) { + fileFilter = new WildcardFileFilter(filter); + } else { + fileFilter = new AndFileFilter( + new WildcardFileFilter(filter), + new NotFileFilter( + new NameFileFilter(files.stream().map(File::getName).collect(Collectors.toList()), IOCase.SENSITIVE) + ) + ); + } + return FileUtils.iterateFiles( + baseDir.getAbsoluteFile(), + fileFilter, + TrueFileFilter.INSTANCE + ); + } + } + + private Iterator getFilesListIterator() + { + if (files == null) { + return Collections.emptyIterator(); + } else { + return files.iterator(); + } + } + @Override - public SplittableInputSource withSplit(InputSplit split) + public SplittableInputSource> withSplit(InputSplit> split) { - final File file = split.get(); - return new LocalInputSource(file.getParentFile(), file.getName()); + return new LocalInputSource(null, null, new HashSet<>(split.get())); } @Override @@ -111,13 +183,11 @@ protected InputSourceReader formattableReader( @Nullable File temporaryDirectory ) { + //noinspection ConstantConditions 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())), + Iterators.transform(getFileIterator(), FileEntity::new), temporaryDirectory ); } @@ -131,14 +201,15 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - LocalInputSource source = (LocalInputSource) o; - return Objects.equals(baseDir, source.baseDir) && - Objects.equals(filter, source.filter); + LocalInputSource that = (LocalInputSource) o; + return Objects.equals(baseDir, that.baseDir) && + Objects.equals(filter, that.filter) && + Objects.equals(files, that.files); } @Override public int hashCode() { - return Objects.hash(baseDir, filter); + return Objects.hash(baseDir, filter, files); } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java index c2172b890429..28b625687a83 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/SplittableInputSource.java @@ -23,6 +23,7 @@ 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.MaxSizeSplitHintSpec; import org.apache.druid.data.input.SplitHintSpec; import javax.annotation.Nullable; @@ -34,6 +35,8 @@ */ public interface SplittableInputSource extends InputSource { + SplitHintSpec DEFAULT_SPLIT_HINT_SPEC = new MaxSizeSplitHintSpec(null); + @JsonIgnore @Override default boolean isSplittable() @@ -70,5 +73,10 @@ default boolean isSplittable() * Helper method for ParallelIndexSupervisorTask. * Most of implementations can simply create a new instance with the given split. */ - SplittableInputSource withSplit(InputSplit split); + InputSource withSplit(InputSplit split); + + default SplitHintSpec getSplitHintSpecOrDefault(@Nullable SplitHintSpec splitHintSpec) + { + return splitHintSpec == null ? DEFAULT_SPLIT_HINT_SPEC : splitHintSpec; + } } diff --git a/core/src/main/java/org/apache/druid/utils/CollectionUtils.java b/core/src/main/java/org/apache/druid/utils/CollectionUtils.java index aacf43df19df..39f6fc259b7f 100644 --- a/core/src/main/java/org/apache/druid/utils/CollectionUtils.java +++ b/core/src/main/java/org/apache/druid/utils/CollectionUtils.java @@ -31,7 +31,6 @@ import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; import java.util.Spliterator; import java.util.TreeSet; @@ -144,7 +143,7 @@ public static LinkedHashMap newLinkedHashMapWithExpectedSize(int ex return new LinkedHashMap<>(Integer.MAX_VALUE); } - public static boolean isNullOrEmpty(@Nullable List list) + public static boolean isNullOrEmpty(@Nullable Collection list) { return list == null || list.isEmpty(); } diff --git a/core/src/main/java/org/apache/druid/utils/Streams.java b/core/src/main/java/org/apache/druid/utils/Streams.java new file mode 100644 index 000000000000..d898d56c111f --- /dev/null +++ b/core/src/main/java/org/apache/druid/utils/Streams.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.utils; + +import java.util.Iterator; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +public final class Streams +{ + public static Stream sequentialStreamFrom(Iterator iterator) + { + final Spliterator spliterator = Spliterators.spliteratorUnknownSize(iterator, Spliterator.DISTINCT); + return StreamSupport.stream(spliterator, false); + } + + public static Stream sequentialStreamFrom(Iterable iterable) + { + return StreamSupport.stream(iterable.spliterator(), false); + } + + private Streams() + { + } +} diff --git a/core/src/test/java/org/apache/druid/data/input/MaxSizeSplitHintSpecTest.java b/core/src/test/java/org/apache/druid/data/input/MaxSizeSplitHintSpecTest.java new file mode 100644 index 000000000000..6e7db2992db1 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/MaxSizeSplitHintSpecTest.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; + +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.commons.compress.utils.Lists; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; +import java.util.function.Function; +import java.util.stream.IntStream; + +public class MaxSizeSplitHintSpecTest +{ + @Test + public void testSerde() throws IOException + { + final ObjectMapper mapper = new ObjectMapper(); + final MaxSizeSplitHintSpec original = new MaxSizeSplitHintSpec(1024L); + final byte[] bytes = mapper.writeValueAsBytes(original); + final MaxSizeSplitHintSpec fromJson = (MaxSizeSplitHintSpec) mapper.readValue(bytes, SplitHintSpec.class); + Assert.assertEquals(original, fromJson); + } + + @Test + public void testCreateWithNullReturningDefaultMaxSplitSize() + { + Assert.assertEquals(MaxSizeSplitHintSpec.DEFAULT_MAX_SPLIT_SIZE, new MaxSizeSplitHintSpec(null).getMaxSplitSize()); + } + + @Test + public void testSplitSmallInputsGroupingIntoLargerSplits() + { + final int eachInputSize = 3; + final MaxSizeSplitHintSpec splitHintSpec = new MaxSizeSplitHintSpec(10L); + final Function inputAttributeExtractor = InputFileAttribute::new; + final List> splits = Lists.newArrayList( + splitHintSpec.split(IntStream.generate(() -> eachInputSize).limit(10).iterator(), inputAttributeExtractor) + ); + Assert.assertEquals(4, splits.size()); + Assert.assertEquals(3, splits.get(0).size()); + Assert.assertEquals(3, splits.get(1).size()); + Assert.assertEquals(3, splits.get(2).size()); + Assert.assertEquals(1, splits.get(3).size()); + } + + @Test + public void testSplitLargeInputsReturningSplitsOfSingleInput() + { + final int eachInputSize = 15; + final MaxSizeSplitHintSpec splitHintSpec = new MaxSizeSplitHintSpec(10L); + final Function inputAttributeExtractor = InputFileAttribute::new; + final List> splits = Lists.newArrayList( + splitHintSpec.split(IntStream.generate(() -> eachInputSize).limit(10).iterator(), inputAttributeExtractor) + ); + Assert.assertEquals(10, splits.size()); + for (List split : splits) { + Assert.assertEquals(1, split.size()); + } + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(MaxSizeSplitHintSpec.class).withNonnullFields("maxSplitSize").usingGetClass().verify(); + } +} diff --git a/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java b/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java index 7bde81c3cb35..e202d152047a 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java @@ -73,7 +73,7 @@ public void test() throws IOException false, 0 ), - files.stream().flatMap(file -> ImmutableList.of(new FileEntity(file)).stream()), + files.stream().flatMap(file -> ImmutableList.of(new FileEntity(file)).stream()).iterator(), temporaryFolder.newFolder() ); diff --git a/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java b/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java index c4f0ab54ca05..6d7342c4180f 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/LocalInputSourceTest.java @@ -20,15 +20,31 @@ package org.apache.druid.data.input.impl; import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.MaxSizeSplitHintSpec; +import org.apache.druid.utils.Streams; +import org.easymock.EasyMock; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import java.io.File; import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; public class LocalInputSourceTest { + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Test public void testSerde() throws IOException { @@ -38,4 +54,98 @@ public void testSerde() throws IOException final LocalInputSource fromJson = (LocalInputSource) mapper.readValue(json, InputSource.class); Assert.assertEquals(source, fromJson); } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(LocalInputSource.class).usingGetClass().withNonnullFields("files").verify(); + } + + @Test + public void testCreateSplitsRespectingSplitHintSpec() + { + final long fileSize = 15; + final long maxSplitSize = 50; + final Set files = prepareFiles(10, fileSize); + final LocalInputSource inputSource = new LocalInputSource(null, null, files); + final List>> splits = inputSource + .createSplits(new NoopInputFormat(), new MaxSizeSplitHintSpec(maxSplitSize)) + .collect(Collectors.toList()); + Assert.assertEquals(4, splits.size()); + Assert.assertEquals(3, splits.get(0).get().size()); + Assert.assertEquals(3, splits.get(1).get().size()); + Assert.assertEquals(3, splits.get(2).get().size()); + Assert.assertEquals(1, splits.get(3).get().size()); + } + + @Test + public void testEstimateNumSplitsRespectingSplitHintSpec() + { + final long fileSize = 13; + final long maxSplitSize = 40; + final Set files = prepareFiles(10, fileSize); + final LocalInputSource inputSource = new LocalInputSource(null, null, files); + Assert.assertEquals( + 4, + inputSource.estimateNumSplits(new NoopInputFormat(), new MaxSizeSplitHintSpec(maxSplitSize)) + ); + } + + @Test + public void testGetFileIteratorWithBothBaseDirAndDuplicateFilesIteratingFilesOnlyOnce() throws IOException + { + File baseDir = temporaryFolder.newFolder(); + List filesInBaseDir = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + filesInBaseDir.add(File.createTempFile("local-input-source", ".data", baseDir)); + } + Set files = new HashSet<>(filesInBaseDir.subList(0, 5)); + for (int i = 0; i < 3; i++) { + files.add(File.createTempFile("local-input-source", ".data", baseDir)); + } + Set expectedFiles = new HashSet<>(filesInBaseDir); + expectedFiles.addAll(files); + File.createTempFile("local-input-source", ".filtered", baseDir); + Iterator fileIterator = new LocalInputSource(baseDir, "*.data", files).getFileIterator(); + Set actualFiles = Streams.sequentialStreamFrom(fileIterator).collect(Collectors.toSet()); + Assert.assertEquals(expectedFiles, actualFiles); + } + + @Test + public void testGetFileIteratorWithOnlyBaseDirIteratingAllFiles() throws IOException + { + File baseDir = temporaryFolder.newFolder(); + Set filesInBaseDir = new HashSet<>(); + for (int i = 0; i < 10; i++) { + filesInBaseDir.add(File.createTempFile("local-input-source", ".data", baseDir)); + } + Iterator fileIterator = new LocalInputSource(baseDir, "*", null).getFileIterator(); + Set actualFiles = Streams.sequentialStreamFrom(fileIterator).collect(Collectors.toSet()); + Assert.assertEquals(filesInBaseDir, actualFiles); + } + + @Test + public void testGetFileIteratorWithOnlyFilesIteratingAllFiles() throws IOException + { + File baseDir = temporaryFolder.newFolder(); + Set filesInBaseDir = new HashSet<>(); + for (int i = 0; i < 10; i++) { + filesInBaseDir.add(File.createTempFile("local-input-source", ".data", baseDir)); + } + Iterator fileIterator = new LocalInputSource(null, null, filesInBaseDir).getFileIterator(); + Set actualFiles = Streams.sequentialStreamFrom(fileIterator).collect(Collectors.toSet()); + Assert.assertEquals(filesInBaseDir, actualFiles); + } + + private static Set prepareFiles(int numFiles, long fileSize) + { + final Set files = new HashSet<>(); + for (int i = 0; i < numFiles; i++) { + final File file = EasyMock.niceMock(File.class); + EasyMock.expect(file.length()).andReturn(fileSize).anyTimes(); + EasyMock.replay(file); + files.add(file); + } + return files; + } } diff --git a/docs/ingestion/native-batch.md b/docs/ingestion/native-batch.md index e19f0efbed50..2195f83cd1f3 100644 --- a/docs/ingestion/native-batch.md +++ b/docs/ingestion/native-batch.md @@ -42,11 +42,12 @@ demonstrates the "simple" (single-task) mode. ## Parallel task The Parallel task (type `index_parallel`) is a task for parallel batch indexing. This task only uses Druid's resource and -doesn't depend on other external systems like Hadoop. `index_parallel` task is a supervisor task which basically creates -multiple worker tasks and submits them to the Overlord. Each worker task reads input data and creates segments. Once they -successfully generate segments for all input data, they report the generated segment list to the supervisor task. +doesn't depend on other external systems like Hadoop. The `index_parallel` task is a supervisor task that orchestrates +the whole indexing process. The supervisor task splits the input data and creates worker tasks to process those splits. +The created worker tasks are issued to the Overlord so that they can be scheduled and run on MiddleManagers or Indexers. +Once a worker task successfully processes the assigned input split, it reports the generated segment list to the supervisor task. The supervisor task periodically checks the status of worker tasks. If one of them fails, it retries the failed task -until the number of retries reaches to the configured limit. If all worker tasks succeed, then it publishes the reported segments at once and finalize the ingestion. +until the number of retries reaches the configured limit. If all worker tasks succeed, it publishes the reported segments at once and finalizes ingestion. The detailed behavior of the Parallel task is different depending on the [`partitionsSpec`](#partitionsspec). See each `partitionsSpec` for more details. @@ -69,15 +70,12 @@ with the `firehose`. - [`static-cloudfiles`](../development/extensions-contrib/cloudfiles.md#firehose) -The splittable `inputSource` (and `firehose`) types are responsible for generating _splits_. -The supervisor task generates _worker task specs_ containing a split -and submits worker tasks using those specs. As a result, the number of worker tasks depends on -the implementation of the splittable `inputSource`. For now, all implementations create one split per input file -except for the Druid Input Source. Please note that multiple worker tasks can be created for the same worker task spec -if one of them fails. - You may want to consider the below things: +- You may want to control the amount of input data each worker task processes. This can be + controlled using different configurations depending on the phase in parallel ingestion (see [`partitionsSpec`](#partitionsspec) for more details). + For the tasks that read data from the `inputSource`, you can set the [SplitHintSpec](#splithintspec) in the `tuningConfig`. + For the tasks that merge shuffled segments, you can set the `totalNumMergeTasks` in the `tuningConfig`. - The number of concurrent worker tasks in parallel ingestion is determined by `maxNumConcurrentSubTasks` in the `tuningConfig`. The supervisor task checks the number of current running worker tasks and creates more if it's smaller than `maxNumConcurrentSubTasks` no matter how many task slots are currently available. @@ -204,7 +202,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no| |maxTotalRows|Deprecated. Use `partitionsSpec` instead. Total number of rows in segments waiting for being pushed. Used in determining when intermediate pushing should occur.|20000000|no| |numShards|Deprecated. Use `partitionsSpec` instead. Directly specify the number of shards to create when using a `hashed` `partitionsSpec`. If this is specified and `intervals` is specified in the `granularitySpec`, the index task can skip the determine intervals/partitions pass through the data. `numShards` cannot be specified if `maxRowsPerSegment` is set.|null|no| -|splitHintSpec|Used to give a hint to control the amount of data that each first phase task reads. This hint could be ignored depending on the implementation of the input source. See [SplitHintSpec](#splithintspec) for more details.|null|no| +|splitHintSpec|Used to give a hint to control the amount of data that each first phase task reads. This hint could be ignored depending on the implementation of the input source. See [SplitHintSpec](#splithintspec) for more details.|null|`maxSize`| |partitionsSpec|Defines how to partition data in each timeChunk, see [PartitionsSpec](#partitionsspec)|`dynamic` if `forceGuaranteedRollup` = false, `hashed` or `single_dim` if `forceGuaranteedRollup` = true|no| |indexSpec|Defines segment storage format options to be used at indexing time, see [IndexSpec](index.md#indexspec)|null|no| |indexSpecForIntermediatePersists|Defines segment storage format options to be used at indexing time for intermediate persisted temporary segments. this can be used to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. however, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published, see [IndexSpec](index.md#indexspec) for possible values.|same as indexSpec|no| @@ -216,7 +214,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |maxNumConcurrentSubTasks|Maximum number of worker tasks which can be run in parallel at the same time. The supervisor task would spawn worker tasks up to `maxNumConcurrentSubTasks` regardless of the current available task slots. If this value is set to 1, the supervisor task processes data ingestion on its own instead of spawning worker tasks. If this value is set to too large, too many worker tasks can be created which might block other ingestion. Check [Capacity Planning](#capacity-planning) for more details.|1|no| |maxRetry|Maximum number of retries on task failures.|3|no| |maxNumSegmentsToMerge|Max limit for the number of segments that a single task can merge at the same time in the second phase. Used only `forceGuaranteedRollup` is set.|100|no| -|totalNumMergeTasks|Total number of tasks to merge segments in the second phase when `forceGuaranteedRollup` is set.|10|no| +|totalNumMergeTasks|Total number of tasks to merge segments in the merge phase when `partitionsSpec` is set to `hashed` or `single_dim`.|10|no| |taskStatusCheckPeriodMs|Polling period in milliseconds to check running task statuses.|1000|no| |chatHandlerTimeout|Timeout for reporting the pushed segments in worker tasks.|PT10S|no| |chatHandlerNumRetries|Retries for reporting the pushed segments in worker tasks.|5|no| @@ -226,7 +224,14 @@ The tuningConfig is optional and default parameters will be used if no tuningCon `SplitHintSpec` is used to give a hint when the supervisor task creates input splits. Note that each worker task processes a single input split. You can control the amount of data each worker task will read during the first phase. -Currently only one splitHintSpec, i.e., `segments`, is available. +#### `MaxSizeSplitHintSpec` + +`MaxSizeSplitHintSpec` is respected by all splittable input sources except for the HTTP input source. + +|property|description|default|required?| +|--------|-----------|-------|---------| +|type|This should always be `maxSize`.|none|yes| +|maxSplitSize|Maximum number of bytes of input files to process in a single task. If a single file is larger than this number, it will be processed by itself in a single task (Files are never split across tasks yet).|500MB|no| #### `SegmentsSplitHintSpec` @@ -235,7 +240,7 @@ Currently only one splitHintSpec, i.e., `segments`, is available. |property|description|default|required?| |--------|-----------|-------|---------| |type|This should always be `segments`.|none|yes| -|maxInputSegmentBytesPerTask|Maximum number of bytes of input segments to process in a single task. If a single segment is larger than this number, it will be processed by itself in a single task (input segments are never split across tasks).|150MB|no| +|maxInputSegmentBytesPerTask|Maximum number of bytes of input segments to process in a single task. If a single segment is larger than this number, it will be processed by itself in a single task (input segments are never split across tasks).|500MB|no| ### `partitionsSpec` @@ -289,8 +294,7 @@ How the worker task creates segments is: The Parallel task with hash-based partitioning is similar to [MapReduce](https://en.wikipedia.org/wiki/MapReduce). The task runs in 2 phases, i.e., `partial segment generation` and `partial segment merge`. - In the `partial segment generation` phase, just like the Map phase in MapReduce, -the Parallel task splits the input data (currently one split for -each input file or based on `splitHintSpec` for `DruidInputSource`) +the Parallel task splits the input data based on `splitHintSpec` and assigns each split to a worker task. Each worker task (type `partial_index_generate`) reads the assigned split, and partitions rows by the time chunk from `segmentGranularity` (primary partition key) in the `granularitySpec` and then by the hash value of `partitionDimensions` (secondary partition key) in the `partitionsSpec`. @@ -322,8 +326,7 @@ The first phase is to collect some statistics to find the best partitioning and the other 2 phases are to create partial segments and to merge them, respectively, as in hash-based partitioning. - In the `partial dimension distribution` phase, the Parallel task splits the input data and -assigns them to worker tasks (currently one split for -each input file or based on `splitHintSpec` for `DruidInputSource`). Each worker task (type `partial_dimension_distribution`) reads +assigns them to worker tasks based on `splitHintSpec`. Each worker task (type `partial_dimension_distribution`) reads the assigned split and builds a histogram for `partitionDimension`. The Parallel task collects those histograms from worker tasks and finds the best range partitioning based on `partitionDimension` to evenly @@ -776,7 +779,7 @@ Objects can be specified either via a list of S3 URI strings or a list of S3 location prefixes, which will attempt to list the contents and ingest all objects contained in the locations. The S3 input source is splittable and can be used by the [Parallel task](#parallel-task), -where each worker task of `index_parallel` will read a single object. +where each worker task of `index_parallel` will read one or multiple objects. Sample specs: @@ -853,7 +856,8 @@ S3 Object: The Google Cloud Storage input source is to support reading objects directly from Google Cloud Storage. Objects can be specified as list of Google Cloud Storage URI strings. The Google Cloud Storage input source is splittable -and can be used by the [Parallel task](#parallel-task), where each worker task of `index_parallel` will read a single object. +and can be used by the [Parallel task](#parallel-task), where each worker task of `index_parallel` will read +one or multiple objects. Sample specs: @@ -1007,7 +1011,7 @@ Azure Blob object: The HDFS input source is to support reading files directly from HDFS storage. File paths can be specified as an HDFS URI string or a list of HDFS URI strings. The HDFS input source is splittable and can be used by the [Parallel task](#parallel-task), -where each worker task of `index_parallel` will read a single file. +where each worker task of `index_parallel` will read one or multiple files. Sample specs: @@ -1089,7 +1093,7 @@ the [S3 input source](#s3-input-source) or the [Google Cloud Storage input sourc The HDFS input source is to support reading files directly from remote sites via HTTP. The HDFS input source is _splittable_ and can be used by the [Parallel task](#parallel-task), -where each worker task of `index_parallel` will read a file. +where each worker task of `index_parallel` will read only one file. Sample specs: @@ -1193,7 +1197,7 @@ Sample spec: The Local input source is to support reading files directly from local storage, and is mainly intended for proof-of-concept testing. The Local input source is _splittable_ and can be used by the [Parallel task](#parallel-task), -where each worker task of `index_parallel` will read a file. +where each worker task of `index_parallel` will read one or multiple files. Sample spec: @@ -1204,7 +1208,8 @@ Sample spec: "inputSource": { "type": "local", "filter" : "*.csv", - "baseDir": "/data/directory" + "baseDir": "/data/directory", + "files": ["/bar/foo", "/foo/bar"] }, "inputFormat": { "type": "csv" @@ -1217,8 +1222,9 @@ Sample spec: |property|description|required?| |--------|-----------|---------| |type|This should be "local".|yes| -|filter|A wildcard filter for files. See [here](http://commons.apache.org/proper/commons-io/apidocs/org/apache/commons/io/filefilter/WildcardFileFilter.html) for more information.|yes| -|baseDir|directory to search recursively for files to be ingested. |yes| +|filter|A wildcard filter for files. See [here](http://commons.apache.org/proper/commons-io/apidocs/org/apache/commons/io/filefilter/WildcardFileFilter.html) for more information.|yes if `baseDir` is specified| +|baseDir|Directory to search recursively for files to be ingested. |At least one of `baseDir` or `files` should be specified| +|files|File paths to ingest. Some files can be ignored to avoid ingesting duplicate files if they are located under the specified `baseDir`. |At least one of `baseDir` or `files` should be specified| ### Druid Input Source @@ -1381,7 +1387,7 @@ Google Blobs: This firehose ingests events from a predefined list of files from the HDFS storage. This firehose is _splittable_ and can be used by the [Parallel task](#parallel-task). -Since each split represents an HDFS file, each worker task of `index_parallel` will read a file. +Since each split represents an HDFS file, each worker task of `index_parallel` will read files. Sample spec: diff --git a/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/data/input/azure/AzureInputSource.java b/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/data/input/azure/AzureInputSource.java index 6bf94f4b82a8..d2382069bca1 100644 --- a/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/data/input/azure/AzureInputSource.java +++ b/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/data/input/azure/AzureInputSource.java @@ -23,8 +23,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputFileAttribute; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.CloudObjectInputSource; import org.apache.druid.data.input.impl.CloudObjectLocation; import org.apache.druid.data.input.impl.SplittableInputSource; @@ -33,19 +34,22 @@ import org.apache.druid.storage.azure.AzureInputDataConfig; import org.apache.druid.storage.azure.AzureStorage; import org.apache.druid.storage.azure.blob.CloudBlobHolder; +import org.apache.druid.utils.Streams; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.net.URI; +import java.util.Iterator; import java.util.List; import java.util.Objects; +import java.util.stream.Collectors; import java.util.stream.Stream; -import java.util.stream.StreamSupport; /** * Abstracts the Azure storage system where input data is stored. Allows users to retrieve entities in * the storage system that match either a particular uri, prefix, or object. */ -public class AzureInputSource extends CloudObjectInputSource +public class AzureInputSource extends CloudObjectInputSource { public static final String SCHEME = "azure"; @@ -75,11 +79,14 @@ public AzureInputSource( "AzureCloudBlobIterableFactory" ); this.inputDataConfig = Preconditions.checkNotNull(inputDataConfig, "AzureInputDataConfig"); - this.azureCloudBlobToLocationConverter = Preconditions.checkNotNull(azureCloudBlobToLocationConverter, "AzureCloudBlobToLocationConverter"); + this.azureCloudBlobToLocationConverter = Preconditions.checkNotNull( + azureCloudBlobToLocationConverter, + "AzureCloudBlobToLocationConverter" + ); } @Override - public SplittableInputSource withSplit(InputSplit split) + public SplittableInputSource> withSplit(InputSplit> split) { return new AzureInputSource( storage, @@ -89,32 +96,28 @@ public SplittableInputSource withSplit(InputSplit split) + protected InputEntity createEntity(CloudObjectLocation location) { - return new GoogleCloudStorageEntity(storage, split.get()); + return new GoogleCloudStorageEntity(storage, location); } @Override - protected Stream> getPrefixesSplitStream() + protected Stream>> getPrefixesSplitStream(@Nonnull SplitHintSpec splitHintSpec) { - return StreamSupport.stream(storageObjectIterable().spliterator(), false) - .map(this::byteSourceFromStorageObject) - .map(InputSplit::new); + final Iterator> splitIterator = splitHintSpec.split( + storageObjectIterable().iterator(), + storageObject -> { + final BigInteger sizeInBigInteger = storageObject.getSize(); + long sizeInLong; + if (sizeInBigInteger == null) { + sizeInLong = Long.MAX_VALUE; + } else { + try { + sizeInLong = sizeInBigInteger.longValueExact(); + } + catch (ArithmeticException e) { + LOG.warn( + e, + "The object [%s, %s] has a size [%s] out of the range of the long type. " + + "The max long value will be used for its size instead.", + storageObject.getBucket(), + storageObject.getName(), + sizeInBigInteger + ); + sizeInLong = Long.MAX_VALUE; + } + } + return new InputFileAttribute(sizeInLong); + } + ); + + return Streams.sequentialStreamFrom(splitIterator) + .map(objects -> objects.stream().map(this::byteSourceFromStorageObject).collect(Collectors.toList())) + .map(InputSplit::new); } @Override - public SplittableInputSource withSplit(InputSplit split) + public SplittableInputSource> withSplit(InputSplit> split) { - return new GoogleCloudStorageInputSource(storage, inputDataConfig, null, null, ImmutableList.of(split.get())); + return new GoogleCloudStorageInputSource(storage, inputDataConfig, null, null, split.get()); } private CloudObjectLocation byteSourceFromStorageObject(final StorageObject storageObject) @@ -86,7 +123,11 @@ private CloudObjectLocation byteSourceFromStorageObject(final StorageObject stor private Iterable storageObjectIterable() { return () -> - GoogleUtils.lazyFetchingStorageObjectsIterator(storage, getPrefixes().iterator(), inputDataConfig.getMaxListingLength()); + GoogleUtils.lazyFetchingStorageObjectsIterator( + storage, + getPrefixes().iterator(), + inputDataConfig.getMaxListingLength() + ); } @Override diff --git a/extensions-core/google-extensions/src/test/java/org/apache/druid/data/input/google/GoogleCloudStorageInputSourceTest.java b/extensions-core/google-extensions/src/test/java/org/apache/druid/data/input/google/GoogleCloudStorageInputSourceTest.java index 62b3b6281436..25fdcca804d2 100644 --- a/extensions-core/google-extensions/src/test/java/org/apache/druid/data/input/google/GoogleCloudStorageInputSourceTest.java +++ b/extensions-core/google-extensions/src/test/java/org/apache/druid/data/input/google/GoogleCloudStorageInputSourceTest.java @@ -35,6 +35,7 @@ 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.MaxSizeSplitHintSpec; import org.apache.druid.data.input.impl.CloudObjectLocation; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -58,9 +59,11 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.math.BigInteger; import java.net.URI; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -81,8 +84,10 @@ public class GoogleCloudStorageInputSourceTest extends InitializedNullHandlingTe URI.create("gs://bar/foo/file2.csv.gz") ); - private static final List EXPECTED_OBJECTS = - EXPECTED_URIS.stream().map(CloudObjectLocation::new).collect(Collectors.toList()); + private static final List> EXPECTED_OBJECTS = + EXPECTED_URIS.stream() + .map(uri -> Collections.singletonList(new CloudObjectLocation(uri))) + .collect(Collectors.toList()); private static final List PREFIXES = Arrays.asList( URI.create("gs://foo/bar"), @@ -142,7 +147,7 @@ public void testWithUrisSplit() GoogleCloudStorageInputSource inputSource = new GoogleCloudStorageInputSource(STORAGE, INPUT_DATA_CONFIG, EXPECTED_URIS, ImmutableList.of(), null); - Stream> splits = inputSource.createSplits( + Stream>> splits = inputSource.createSplits( new JsonInputFormat(JSONPathSpec.DEFAULT, null), null ); @@ -163,14 +168,39 @@ public void testWithPrefixesSplit() throws IOException GoogleCloudStorageInputSource inputSource = new GoogleCloudStorageInputSource(STORAGE, INPUT_DATA_CONFIG, null, PREFIXES, null); - Stream> splits = inputSource.createSplits( + Stream>> splits = inputSource.createSplits( new JsonInputFormat(JSONPathSpec.DEFAULT, null), - null + new MaxSizeSplitHintSpec(1L) // set maxSplitSize to 1 so that each inputSplit has only one object ); Assert.assertEquals(EXPECTED_OBJECTS, splits.map(InputSplit::get).collect(Collectors.toList())); } + @Test + public void testCreateSplitsWithSplitHintSpecRespectingHint() throws IOException + { + EasyMock.reset(STORAGE); + EasyMock.reset(INPUT_DATA_CONFIG); + addExpectedPrefixObjects(PREFIXES.get(0), ImmutableList.of(EXPECTED_URIS.get(0))); + addExpectedPrefixObjects(PREFIXES.get(1), ImmutableList.of(EXPECTED_URIS.get(1))); + EasyMock.expect(INPUT_DATA_CONFIG.getMaxListingLength()).andReturn(MAX_LISTING_LENGTH); + EasyMock.replay(STORAGE); + EasyMock.replay(INPUT_DATA_CONFIG); + + GoogleCloudStorageInputSource inputSource = + new GoogleCloudStorageInputSource(STORAGE, INPUT_DATA_CONFIG, null, PREFIXES, null); + + Stream>> splits = inputSource.createSplits( + new JsonInputFormat(JSONPathSpec.DEFAULT, null), + new MaxSizeSplitHintSpec(CONTENT.length * 3L) + ); + + Assert.assertEquals( + ImmutableList.of(EXPECTED_URIS.stream().map(CloudObjectLocation::new).collect(Collectors.toList())), + splits.map(InputSplit::get).collect(Collectors.toList()) + ); + } + @Test public void testReader() throws IOException { @@ -274,6 +304,7 @@ private static void addExpectedPrefixObjects(URI prefix, List uris) throws StorageObject s = new StorageObject(); s.setBucket(bucket); s.setName(uri.getPath()); + s.setSize(BigInteger.valueOf(CONTENT.length)); mockObjects.add(s); } Objects response = new Objects(); diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/inputsource/hdfs/HdfsInputSource.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/inputsource/hdfs/HdfsInputSource.java index fe4a18bdf76e..661a07894e21 100644 --- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/inputsource/hdfs/HdfsInputSource.java +++ b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/inputsource/hdfs/HdfsInputSource.java @@ -24,7 +24,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterators; import org.apache.druid.data.input.AbstractInputSource; +import org.apache.druid.data.input.InputFileAttribute; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; @@ -34,6 +36,7 @@ import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.guice.Hdfs; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.utils.Streams; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; @@ -49,11 +52,12 @@ import java.io.UncheckedIOException; import java.util.Collection; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.stream.Collectors; import java.util.stream.Stream; -public class HdfsInputSource extends AbstractInputSource implements SplittableInputSource +public class HdfsInputSource extends AbstractInputSource implements SplittableInputSource> { private static final String PROP_PATHS = "paths"; @@ -150,28 +154,38 @@ protected InputSourceReader formattableReader( @Nullable File temporaryDirectory ) { - final Stream> splits; try { - splits = createSplits(inputFormat, null); + cachePathsIfNeeded(); } catch (IOException e) { throw new UncheckedIOException(e); } - return new InputEntityIteratingReader( inputRowSchema, inputFormat, - splits.map(split -> new HdfsInputEntity(configuration, split.get())), + Iterators.transform(cachedPaths.iterator(), path -> new HdfsInputEntity(configuration, path)), temporaryDirectory ); } @Override - public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + public Stream>> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException { cachePathsIfNeeded(); - return cachedPaths.stream().map(InputSplit::new); + final Iterator> splitIterator = getSplitHintSpecOrDefault(splitHintSpec).split( + cachedPaths.iterator(), + path -> { + try { + final long size = path.getFileSystem(configuration).getFileStatus(path).getLen(); + return new InputFileAttribute(size); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + ); + return Streams.sequentialStreamFrom(splitIterator).map(InputSplit::new); } @Override @@ -182,7 +196,7 @@ public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec sp } @Override - public SplittableInputSource withSplit(InputSplit split) + public SplittableInputSource> withSplit(InputSplit> split) { return new HdfsInputSource(split.get().toString(), configuration); } diff --git a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/inputsource/hdfs/HdfsInputSourceTest.java b/extensions-core/hdfs-storage/src/test/java/org/apache/druid/inputsource/hdfs/HdfsInputSourceTest.java index dae315c0b7cb..92b4ad8ed405 100644 --- a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/inputsource/hdfs/HdfsInputSourceTest.java +++ b/extensions-core/hdfs-storage/src/test/java/org/apache/druid/inputsource/hdfs/HdfsInputSourceTest.java @@ -27,6 +27,8 @@ 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.InputSplit; +import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; @@ -246,16 +248,32 @@ public void readsSplitsCorrectly() throws IOException @Test public void hasCorrectSplits() throws IOException { - Set actualPaths = target.createSplits(null, null) - .map(split -> Path.getPathWithoutSchemeAndAuthority(split.get())) + // Set maxSplitSize to 1 so that each inputSplit has only one object + List>> splits = target.createSplits(null, new MaxSizeSplitHintSpec(1L)) + .collect(Collectors.toList()); + splits.forEach(split -> Assert.assertEquals(1, split.get().size())); + Set actualPaths = splits.stream() + .flatMap(split -> split.get().stream()) + .map(Path::getPathWithoutSchemeAndAuthority) .collect(Collectors.toSet()); Assert.assertEquals(paths, actualPaths); } + @Test + public void createSplitsRespectSplitHintSpec() throws IOException + { + List>> splits = target.createSplits(null, new MaxSizeSplitHintSpec(7L)) + .collect(Collectors.toList()); + Assert.assertEquals(2, splits.size()); + Assert.assertEquals(2, splits.get(0).get().size()); + Assert.assertEquals(1, splits.get(1).get().size()); + } + @Test public void hasCorrectNumberOfSplits() throws IOException { - int numSplits = target.estimateNumSplits(null, null); + // Set maxSplitSize to 1 so that each inputSplit has only one object + int numSplits = target.estimateNumSplits(null, new MaxSizeSplitHintSpec(1L)); Assert.assertEquals(NUM_FILE, numSplits); } } @@ -286,10 +304,9 @@ public void readsSplitsCorrectly() throws IOException @Test public void hasCorrectSplits() throws IOException { - List paths = target.createSplits(null, null) - .map(split -> Path.getPathWithoutSchemeAndAuthority(split.get())) - .collect(Collectors.toList()); - Assert.assertTrue(String.valueOf(paths), paths.isEmpty()); + List>> splits = target.createSplits(null, null) + .collect(Collectors.toList()); + Assert.assertTrue(String.valueOf(splits), splits.isEmpty()); } @Test diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/data/input/s3/S3InputSource.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/data/input/s3/S3InputSource.java index 829d56cdc718..659c68f72909 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/data/input/s3/S3InputSource.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/data/input/s3/S3InputSource.java @@ -24,8 +24,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputFileAttribute; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.CloudObjectInputSource; import org.apache.druid.data.input.impl.CloudObjectLocation; import org.apache.druid.data.input.impl.SplittableInputSource; @@ -33,14 +35,17 @@ import org.apache.druid.storage.s3.S3StorageDruidModule; import org.apache.druid.storage.s3.S3Utils; import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; +import org.apache.druid.utils.Streams; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.net.URI; +import java.util.Iterator; import java.util.List; +import java.util.stream.Collectors; import java.util.stream.Stream; -import java.util.stream.StreamSupport; -public class S3InputSource extends CloudObjectInputSource +public class S3InputSource extends CloudObjectInputSource { private final ServerSideEncryptingAmazonS3 s3Client; private final S3InputDataConfig inputDataConfig; @@ -60,23 +65,30 @@ public S3InputSource( } @Override - protected S3Entity createEntity(InputSplit split) + protected InputEntity createEntity(CloudObjectLocation location) { - return new S3Entity(s3Client, split.get()); + return new S3Entity(s3Client, location); } @Override - protected Stream> getPrefixesSplitStream() + protected Stream>> getPrefixesSplitStream(@Nonnull SplitHintSpec splitHintSpec) { - return StreamSupport.stream(getIterableObjectsFromPrefixes().spliterator(), false) - .map(S3Utils::summaryToCloudObjectLocation) - .map(InputSplit::new); + final Iterator> splitIterator = splitHintSpec.split( + getIterableObjectsFromPrefixes().iterator(), + object -> new InputFileAttribute(object.getSize()) + ); + + return Streams.sequentialStreamFrom(splitIterator) + .map(objects -> objects.stream() + .map(S3Utils::summaryToCloudObjectLocation) + .collect(Collectors.toList())) + .map(InputSplit::new); } @Override - public SplittableInputSource withSplit(InputSplit split) + public SplittableInputSource> withSplit(InputSplit> split) { - return new S3InputSource(s3Client, inputDataConfig, null, null, ImmutableList.of(split.get())); + return new S3InputSource(s3Client, inputDataConfig, null, null, split.get()); } @Override diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java index acedc0c62969..a7ab23bd9feb 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java @@ -43,6 +43,7 @@ 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.MaxSizeSplitHintSpec; import org.apache.druid.data.input.impl.CloudObjectLocation; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -75,6 +76,7 @@ import java.io.IOException; import java.net.URI; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -100,8 +102,10 @@ public class S3InputSourceTest extends InitializedNullHandlingTest URI.create("s3://bar/foo/file2.csv.gz") ); - private static final List EXPECTED_COORDS = - EXPECTED_URIS.stream().map(CloudObjectLocation::new).collect(Collectors.toList()); + private static final List> EXPECTED_COORDS = + EXPECTED_URIS.stream() + .map(uri -> Collections.singletonList(new CloudObjectLocation(uri))) + .collect(Collectors.toList()); private static final List PREFIXES = Arrays.asList( URI.create("s3://foo/bar"), @@ -221,7 +225,7 @@ public void testWithUrisSplit() { S3InputSource inputSource = new S3InputSource(SERVICE, INPUT_DATA_CONFIG, EXPECTED_URIS, null, null); - Stream> splits = inputSource.createSplits( + Stream>> splits = inputSource.createSplits( new JsonInputFormat(JSONPathSpec.DEFAULT, null), null ); @@ -239,15 +243,37 @@ public void testWithPrefixesSplit() S3InputSource inputSource = new S3InputSource(SERVICE, INPUT_DATA_CONFIG, null, PREFIXES, null); - Stream> splits = inputSource.createSplits( + Stream>> splits = inputSource.createSplits( new JsonInputFormat(JSONPathSpec.DEFAULT, null), - null + new MaxSizeSplitHintSpec(1L) // set maxSplitSize to 1 so that each inputSplit has only one object ); Assert.assertEquals(EXPECTED_COORDS, splits.map(InputSplit::get).collect(Collectors.toList())); EasyMock.verify(S3_CLIENT); } + @Test + public void testCreateSplitsWithSplitHintSpecRespectingHint() + { + EasyMock.reset(S3_CLIENT); + expectListObjects(PREFIXES.get(0), ImmutableList.of(EXPECTED_URIS.get(0))); + expectListObjects(PREFIXES.get(1), ImmutableList.of(EXPECTED_URIS.get(1))); + EasyMock.replay(S3_CLIENT); + + S3InputSource inputSource = new S3InputSource(SERVICE, INPUT_DATA_CONFIG, null, PREFIXES, null); + + Stream>> splits = inputSource.createSplits( + new JsonInputFormat(JSONPathSpec.DEFAULT, null), + new MaxSizeSplitHintSpec(CONTENT.length * 3L) + ); + + Assert.assertEquals( + ImmutableList.of(EXPECTED_URIS.stream().map(CloudObjectLocation::new).collect(Collectors.toList())), + splits.map(InputSplit::get).collect(Collectors.toList()) + ); + EasyMock.verify(S3_CLIENT); + } + @Test public void testAccessDeniedWhileListingPrefix() { @@ -370,6 +396,7 @@ private static void expectListObjects(URI prefix, List uris) final S3ObjectSummary objectSummary = new S3ObjectSummary(); objectSummary.setBucketName(bucket); objectSummary.setKey(key); + objectSummary.setSize(CONTENT.length); result.getObjectSummaries().add(objectSummary); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java index 3f5f9b8259d6..f23c513e59d6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java @@ -22,6 +22,7 @@ import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; +import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexing.common.TaskToolbox; @@ -33,7 +34,7 @@ /** * Base class for different implementations of {@link ParallelIndexTaskRunner} that operate on - * {@link org.apache.druid.data.input.InputSource} splits. + * {@link InputSource} splits. */ abstract class InputSourceSplitParallelIndexTaskRunner extends ParallelIndexPhaseRunner @@ -85,7 +86,7 @@ final int estimateTotalNumSubTasks() throws IOException final SubTaskSpec newTaskSpec(InputSplit split) { final FirehoseFactory firehoseFactory; - final SplittableInputSource inputSource; + final InputSource inputSource; if (baseInputSource instanceof FirehoseFactoryToInputSourceAdaptor) { firehoseFactory = ((FirehoseFactoryToInputSourceAdaptor) baseInputSource).getFirehoseFactory().withSplit(split); inputSource = null; 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 52ebb04a4b8b..10311a8a9c4e 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 @@ -23,6 +23,7 @@ import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; +import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexing.common.TaskToolbox; @@ -103,7 +104,7 @@ int estimateTotalNumSubTasks() throws IOException SubTaskSpec newTaskSpec(InputSplit split) { final FirehoseFactory firehoseFactory; - final SplittableInputSource inputSource; + final InputSource inputSource; if (baseInputSource instanceof FirehoseFactoryToInputSourceAdaptor) { firehoseFactory = ((FirehoseFactoryToInputSourceAdaptor) baseInputSource).getFirehoseFactory().withSplit(split); inputSource = null; 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 320b85980353..41c1c8f258ef 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 @@ -290,12 +290,14 @@ private void initializeSplitsIfNeeded(@Nullable SplitHintSpec splitHintSpec) return; } - splits = DruidInputSource.createSplits( - coordinatorClient, - retryPolicyFactory, - dataSource, - interval, - splitHintSpec == null ? new SegmentsSplitHintSpec(maxInputSegmentBytesPerTask) : splitHintSpec + splits = Lists.newArrayList( + DruidInputSource.createSplits( + coordinatorClient, + retryPolicyFactory, + dataSource, + interval, + splitHintSpec == null ? new SegmentsSplitHintSpec(maxInputSegmentBytesPerTask) : splitHintSpec + ) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/WindowedSegmentId.java b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/WindowedSegmentId.java index c3f04bbcb27e..b55510a7e219 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/WindowedSegmentId.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/WindowedSegmentId.java @@ -24,7 +24,9 @@ import com.google.common.base.Preconditions; import org.joda.time.Interval; +import java.util.Collections; import java.util.List; +import java.util.Objects; /** * A WindowedSegment represents a segment plus the list of intervals inside it which contribute to a timeline. @@ -47,6 +49,11 @@ public WindowedSegmentId( this.intervals = Preconditions.checkNotNull(intervals, "null intervals"); } + public void addInterval(Interval interval) + { + this.intervals.add(interval); + } + @JsonProperty public String getSegmentId() { @@ -56,6 +63,35 @@ public String getSegmentId() @JsonProperty public List getIntervals() { - return intervals; + return Collections.unmodifiableList(intervals); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WindowedSegmentId segmentId1 = (WindowedSegmentId) o; + return Objects.equals(segmentId, segmentId1.segmentId) && + Objects.equals(intervals, segmentId1.intervals); + } + + @Override + public int hashCode() + { + return Objects.hash(segmentId, intervals); + } + + @Override + public String toString() + { + return "WindowedSegmentId{" + + "segmentId='" + segmentId + '\'' + + ", intervals=" + intervals + + '}'; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java index bc07f02d008b..191f47e7357d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java @@ -25,13 +25,16 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import com.google.common.collect.Iterators; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.data.input.AbstractInputSource; import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputFileAttribute; 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.MaxSizeSplitHintSpec; import org.apache.druid.data.input.SegmentsSplitHintSpec; import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.InputEntityIteratingReader; @@ -53,6 +56,7 @@ import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionHolder; +import org.apache.druid.utils.Streams; import org.joda.time.Duration; import org.joda.time.Interval; @@ -62,6 +66,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.SortedMap; @@ -166,8 +171,7 @@ protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nu final SegmentLoader segmentLoader = segmentLoaderFactory.manufacturate(temporaryDirectory); final List> timeline = createTimeline(); - - final Stream entityStream = createTimeline() + final Stream entityStream = timeline .stream() .flatMap(holder -> { final PartitionHolder partitionHolder = holder.getObject(); @@ -205,7 +209,7 @@ protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nu return new InputEntityIteratingReader( inputRowSchema, inputFormat, - entityStream, + entityStream.iterator(), temporaryDirectory ); } @@ -228,13 +232,15 @@ public Stream>> createSplits( // segmentIds is supposed to be specified by the supervisor task during the parallel indexing. // If it's not null, segments are already split by the supervisor task and further split won't happen. if (segmentIds == null) { - return createSplits( - coordinatorClient, - retryPolicyFactory, - dataSource, - interval, - splitHintSpec == null ? new SegmentsSplitHintSpec(null) : splitHintSpec - ).stream(); + return Streams.sequentialStreamFrom( + createSplits( + coordinatorClient, + retryPolicyFactory, + dataSource, + interval, + splitHintSpec == null ? new MaxSizeSplitHintSpec(null) : splitHintSpec + ) + ); } else { return Stream.of(new InputSplit<>(segmentIds)); } @@ -246,13 +252,15 @@ public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec sp // segmentIds is supposed to be specified by the supervisor task during the parallel indexing. // If it's not null, segments are already split by the supervisor task and further split won't happen. if (segmentIds == null) { - return createSplits( - coordinatorClient, - retryPolicyFactory, - dataSource, - interval, - splitHintSpec == null ? new SegmentsSplitHintSpec(null) : splitHintSpec - ).size(); + return Iterators.size( + createSplits( + coordinatorClient, + retryPolicyFactory, + dataSource, + interval, + splitHintSpec == null ? new MaxSizeSplitHintSpec(null) : splitHintSpec + ) + ); } else { return 1; } @@ -281,7 +289,7 @@ public boolean needsFormat() return false; } - public static List>> createSplits( + public static Iterator>> createSplits( CoordinatorClient coordinatorClient, RetryPolicyFactory retryPolicyFactory, String dataSource, @@ -289,70 +297,52 @@ public static List>> createSplits( SplitHintSpec splitHintSpec ) { - final long maxInputSegmentBytesPerTask; - if (!(splitHintSpec instanceof SegmentsSplitHintSpec)) { - LOG.warn("Given splitHintSpec[%s] is not a SegmentsSplitHintSpec. Ignoring it.", splitHintSpec); - maxInputSegmentBytesPerTask = new SegmentsSplitHintSpec(null).getMaxInputSegmentBytesPerTask(); + final SplitHintSpec convertedSplitHintSpec; + if (splitHintSpec instanceof SegmentsSplitHintSpec) { + convertedSplitHintSpec = new MaxSizeSplitHintSpec( + ((SegmentsSplitHintSpec) splitHintSpec).getMaxInputSegmentBytesPerTask() + ); } else { - maxInputSegmentBytesPerTask = ((SegmentsSplitHintSpec) splitHintSpec).getMaxInputSegmentBytesPerTask(); + convertedSplitHintSpec = splitHintSpec; } - // isSplittable() ensures this is only called when we have an interval. final List> timelineSegments = getTimelineForInterval( coordinatorClient, retryPolicyFactory, dataSource, interval ); + final Map segmentIdToSize = createWindowedSegmentIdFromTimeline(timelineSegments); + //noinspection ConstantConditions + return Iterators.transform( + convertedSplitHintSpec.split( + segmentIdToSize.keySet().iterator(), + segmentId -> new InputFileAttribute( + Preconditions.checkNotNull(segmentIdToSize.get(segmentId), "segment size for [%s]", segmentId) + ) + ), + InputSplit::new + ); + } - // We do the simplest possible greedy algorithm here instead of anything cleverer. The general bin packing - // problem is NP-hard, and we'd like to get segments from the same interval into the same split so that their - // data can combine with each other anyway. - - List>> splits = new ArrayList<>(); - List currentSplit = new ArrayList<>(); + private static Map createWindowedSegmentIdFromTimeline( + List> timelineHolders + ) + { Map windowedSegmentIds = new HashMap<>(); - long bytesInCurrentSplit = 0; - for (TimelineObjectHolder timelineHolder : timelineSegments) { - for (PartitionChunk chunk : timelineHolder.getObject()) { - final DataSegment segment = chunk.getObject(); - final WindowedSegmentId existingWindowedSegmentId = windowedSegmentIds.get(segment); - if (existingWindowedSegmentId != null) { - // We've already seen this segment in the timeline, so just add this interval to it. It has already - // been placed into a split. - existingWindowedSegmentId.getIntervals().add(timelineHolder.getInterval()); - } else { - // It's the first time we've seen this segment, so create a new WindowedSegmentId. - List intervals = new ArrayList<>(); - // Use the interval that contributes to the timeline, not the entire segment's true interval. - intervals.add(timelineHolder.getInterval()); - final WindowedSegmentId newWindowedSegmentId = new WindowedSegmentId(segment.getId().toString(), intervals); - windowedSegmentIds.put(segment, newWindowedSegmentId); - - // Now figure out if it goes in the current split or not. - final long segmentBytes = segment.getSize(); - if (bytesInCurrentSplit + segmentBytes > maxInputSegmentBytesPerTask && !currentSplit.isEmpty()) { - // This segment won't fit in the current non-empty split, so this split is done. - splits.add(new InputSplit<>(currentSplit)); - currentSplit = new ArrayList<>(); - bytesInCurrentSplit = 0; - } - if (segmentBytes > maxInputSegmentBytesPerTask) { - // If this segment is itself bigger than our max, just put it in its own split. - Preconditions.checkState(currentSplit.isEmpty() && bytesInCurrentSplit == 0); - splits.add(new InputSplit<>(Collections.singletonList(newWindowedSegmentId))); - } else { - currentSplit.add(newWindowedSegmentId); - bytesInCurrentSplit += segmentBytes; - } - } + for (TimelineObjectHolder holder : timelineHolders) { + for (PartitionChunk chunk : holder.getObject()) { + windowedSegmentIds.computeIfAbsent( + chunk.getObject(), + segment -> new WindowedSegmentId(segment.getId().toString(), new ArrayList<>()) + ).addInterval(holder.getInterval()); } } - if (!currentSplit.isEmpty()) { - splits.add(new InputSplit<>(currentSplit)); - } - - return splits; + // It is important to create this map after windowedSegmentIds is completely filled + // because WindowedSegmentId can be updated. + Map segmentSizeMap = new HashMap<>(); + windowedSegmentIds.forEach((segment, segmentId) -> segmentSizeMap.put(segmentId, segment.getSize())); + return segmentSizeMap; } public static List> getTimelineForInterval( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index eb8383020f48..8ffa4779ecb3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.common.task; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.google.common.io.Files; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SegmentsSplitHintSpec; @@ -124,12 +125,14 @@ public void testDruidInputSourceCreateSplitsWithIndividualSplits() throws Except { runIndexTask(); - List>> splits = DruidInputSource.createSplits( - getCoordinatorClient(), - RETRY_POLICY_FACTORY, - DATA_SOURCE, - INTERVAL_TO_INDEX, - new SegmentsSplitHintSpec(1L) // each segment gets its own split with this config + List>> splits = Lists.newArrayList( + DruidInputSource.createSplits( + getCoordinatorClient(), + RETRY_POLICY_FACTORY, + DATA_SOURCE, + INTERVAL_TO_INDEX, + new SegmentsSplitHintSpec(1L) // each segment gets its own split with this config + ) ); List segments = new ArrayList<>( 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 e393d7536af9..84fff3141ef1 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 @@ -1637,15 +1637,16 @@ public void testOverwriteWithDifferentSegmentGranularity() throws Exception } @Test - public void testIndexTaskWitSingleDimPartitionsSpecThrowingException() throws Exception + public void testIndexTaskWithSingleDimPartitionsSpecThrowingException() throws Exception { + final File tmpDir = temporaryFolder.newFolder(); final IndexTask task = new IndexTask( null, null, createIngestionSpec( useInputFormatApi, jsonMapper, - null, + tmpDir, null, null, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index de8b4de96522..194a556a1291 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.common.task.batch.parallel; +import org.apache.druid.data.input.MaxSizeSplitHintSpec; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.StringInputRowParser; @@ -128,7 +129,7 @@ private ParallelIndexSupervisorTask newTask( null, null, null, - null, + new MaxSizeSplitHintSpec(1L), // set maxSplitSize to 1 so that each split has only one file. partitionsSpec, null, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/WindowedSegmentIdTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/WindowedSegmentIdTest.java new file mode 100644 index 000000000000..0586e98c11ba --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/WindowedSegmentIdTest.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.indexing.firehose; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class WindowedSegmentIdTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(WindowedSegmentId.class) + .usingGetClass() + .withNonnullFields("segmentId", "intervals") + .verify(); + } +} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java index 0b14f92dc561..18bc3fc7e9d3 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java @@ -37,6 +37,7 @@ @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITBestEffortRollupParallelIndexTest extends AbstractITBatchIndexTest { + // The task specs here use the MaxSizeSplitHintSpec with maxSplitSize of 1. This is to create splits per file. private static final String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json"; private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json"; private static final String REINDEX_TASK = "/indexer/wikipedia_parallel_reindex_task.json"; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java index 067c22455683..99bc1e180b38 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java @@ -38,6 +38,7 @@ @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITPerfectRollupParallelIndexTest extends AbstractITBatchIndexTest { + // The task specs here use the MaxSizeSplitHintSpec with maxSplitSize of 1. This is to create splits per file. private static final String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json"; private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json"; private static final String INDEX_DATASOURCE = "wikipedia_parallel_index_test"; diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json index e2eb4cb44b10..ff347d42219e 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json @@ -66,6 +66,10 @@ "type": "index_parallel", "maxNumConcurrentSubTasks": 10, "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%", + "splitHintSpec": { + "type": "maxSize", + "maxSplitSize": 1 + }, "partitionsSpec": %%PARTITIONS_SPEC%% } } diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json index e8b5db43491f..87cb498f3c4b 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json @@ -59,6 +59,10 @@ "type": "index_parallel", "maxNumConcurrentSubTasks": 10, "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%", + "splitHintSpec": { + "type": "maxSize", + "maxSplitSize": 1 + }, "partitionsSpec": %%PARTITIONS_SPEC%% } } diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_reindex_task.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_reindex_task.json index 685a9e21c741..e17913415f6c 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_parallel_reindex_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_reindex_task.json @@ -65,6 +65,10 @@ "type": "index_parallel", "maxNumConcurrentSubTasks": 10, "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%", + "splitHintSpec": { + "type": "maxSize", + "maxSplitSize": 1 + }, "partitionsSpec": %%PARTITIONS_SPEC%% } }