Create splits of multiple files for parallel indexing (#9360)

* Create splits of multiple files for parallel indexing

* fix wrong import and npe in test

* use the single file split in tests

* rename

* import order

* Remove specific local input source

* Update docs/ingestion/native-batch.md

Co-Authored-By: sthetland <steve.hetland@imply.io>

* Update docs/ingestion/native-batch.md

Co-Authored-By: sthetland <steve.hetland@imply.io>

* doc and error msg

* fix build

* fix a test and address comments

Co-authored-by: sthetland <steve.hetland@imply.io>
This commit is contained in:
Jihoon Son 2020-02-24 17:34:39 -08:00 committed by GitHub
parent 806dfe6de6
commit 3bc7ae782c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
40 changed files with 1019 additions and 254 deletions

View File

@ -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;
}
}

View File

@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
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<T>

View File

@ -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 <T> Iterator<List<T>> split(Iterator<T> inputIterator, Function<T, InputFileAttribute> inputAttributeExtractor)
{
return new Iterator<List<T>>()
{
private T peeking;
@Override
public boolean hasNext()
{
return peeking != null || inputIterator.hasNext();
}
@Override
public List<T> next()
{
if (!hasNext()) {
throw new NoSuchElementException();
}
final List<T> 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);
}
}

View File

@ -23,10 +23,19 @@ import com.fasterxml.jackson.annotation.JsonCreator;
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 class SegmentsSplitHintSpec implements SplitHintSpec
return maxInputSegmentBytesPerTask;
}
@Override
public <T> Iterator<List<T>> split(Iterator<T> inputIterator, Function<T, InputFileAttribute> 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)
{

View File

@ -24,6 +24,10 @@ import com.fasterxml.jackson.annotation.JsonSubTypes.Type;
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 @@ import org.apache.druid.data.input.impl.SplittableInputSource;
*/
@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.
*/
<T> Iterator<List<T>> split(Iterator<T> inputIterator, Function<T, InputFileAttribute> inputAttributeExtractor);
}

View File

@ -33,12 +33,13 @@ import org.apache.druid.utils.CollectionUtils;
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<T extends InputEntity> extends AbstractInputSource
implements SplittableInputSource<CloudObjectLocation>
public abstract class CloudObjectInputSource extends AbstractInputSource
implements SplittableInputSource<List<CloudObjectLocation>>
{
private final List<URI> uris;
private final List<URI> prefixes;
@ -90,7 +91,7 @@ public abstract class CloudObjectInputSource<T extends InputEntity> extends Abst
* 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<CloudObjectLocation> 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 abstract class CloudObjectInputSource<T extends InputEntity> extends Abst
* 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<InputSplit<CloudObjectLocation>> getPrefixesSplitStream();
protected abstract Stream<InputSplit<List<CloudObjectLocation>>> getPrefixesSplitStream(SplitHintSpec splitHintSpec);
@Override
public Stream<InputSplit<CloudObjectLocation>> createSplits(
public Stream<InputSplit<List<CloudObjectLocation>>> 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 abstract class CloudObjectInputSource<T extends InputEntity> extends Abst
return uris.size();
}
return Ints.checkedCast(getPrefixesSplitStream().count());
return Ints.checkedCast(getPrefixesSplitStream(getSplitHintSpecOrDefault(splitHintSpec)).count());
}
@Override
@ -146,7 +149,7 @@ public abstract class CloudObjectInputSource<T extends InputEntity> extends Abst
return new InputEntityIteratingReader(
inputRowSchema,
inputFormat,
createSplits(inputFormat, null).map(this::createEntity),
createSplits(inputFormat, null).flatMap(split -> split.get().stream()).map(this::createEntity).iterator(),
temporaryDirectory
);
}

View File

@ -115,7 +115,7 @@ public class HttpInputSource extends AbstractInputSource implements SplittableIn
split.get(),
httpAuthenticationUsername,
httpAuthenticationPasswordProvider
)),
)).iterator(),
temporaryDirectory
);
}

View File

@ -69,7 +69,7 @@ public class InlineInputSource extends AbstractInputSource
return new InputEntityIteratingReader(
inputRowSchema,
inputFormat,
Stream.of(new ByteEntity(StringUtils.toUtf8(data))),
Stream.of(new ByteEntity(StringUtils.toUtf8(data))).iterator(),
temporaryDirectory
);
}

View File

@ -31,8 +31,8 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator;
import java.io.File;
import java.io.IOException;
import java.util.Iterator;
import java.util.function.Function;
import java.util.stream.Stream;
/**
* InputSourceReader iterating multiple {@link InputEntity}s. This class could be used for
@ -48,23 +48,23 @@ public class InputEntityIteratingReader implements InputSourceReader
public InputEntityIteratingReader(
InputRowSchema inputRowSchema,
InputFormat inputFormat,
Stream<InputEntity> sourceStream,
Iterator<? extends InputEntity> 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<InputEntity> sourceIterator,
CloseableIterator<? extends InputEntity> sourceCloseableIterator,
File temporaryDirectory
)
{
this.inputRowSchema = inputRowSchema;
this.inputFormat = inputFormat;
this.sourceIterator = sourceIterator;
this.sourceIterator = (CloseableIterator<InputEntity>) sourceCloseableIterator;
this.temporaryDirectory = temporaryDirectory;
}

View File

@ -21,81 +21,153 @@ package org.apache.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.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<File>
public class LocalInputSource extends AbstractInputSource implements SplittableInputSource<List<File>>
{
@Nullable
private final File baseDir;
@Nullable
private final String filter;
private final Set<File> 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<File> 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;
}
@Override
public Stream<InputSplit<File>> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec)
@JsonProperty
public Set<File> getFiles()
{
return StreamSupport.stream(Spliterators.spliteratorUnknownSize(getFileIterator(), Spliterator.DISTINCT), false)
.map(InputSplit::new);
return files;
}
@Override
public Stream<InputSplit<List<File>>> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec)
{
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<File> getFileIterator()
private Iterator<List<File>> getSplitFileIterator(SplitHintSpec splitHintSpec)
{
return FileUtils.iterateFiles(
Preconditions.checkNotNull(baseDir).getAbsoluteFile(),
new WildcardFileFilter(filter),
TrueFileFilter.INSTANCE
final Iterator<File> fileIterator = getFileIterator();
return splitHintSpec.split(fileIterator, file -> new InputFileAttribute(file.length()));
}
@VisibleForTesting
Iterator<File> getFileIterator()
{
return Iterators.concat(
getDirectoryListingIterator(),
getFilesListIterator()
);
}
@Override
public SplittableInputSource<File> withSplit(InputSplit<File> split)
private Iterator<File> getDirectoryListingIterator()
{
final File file = split.get();
return new LocalInputSource(file.getParentFile(), file.getName());
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<File> getFilesListIterator()
{
if (files == null) {
return Collections.emptyIterator();
} else {
return files.iterator();
}
}
@Override
public SplittableInputSource<List<File>> withSplit(InputSplit<List<File>> split)
{
return new LocalInputSource(null, null, new HashSet<>(split.get()));
}
@Override
@ -111,13 +183,11 @@ public class LocalInputSource extends AbstractInputSource implements SplittableI
@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 class LocalInputSource extends AbstractInputSource implements SplittableI
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);
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.MaxSizeSplitHintSpec;
import org.apache.druid.data.input.SplitHintSpec;
import javax.annotation.Nullable;
@ -34,6 +35,8 @@ import java.util.stream.Stream;
*/
public interface SplittableInputSource<T> extends InputSource
{
SplitHintSpec DEFAULT_SPLIT_HINT_SPEC = new MaxSizeSplitHintSpec(null);
@JsonIgnore
@Override
default boolean isSplittable()
@ -70,5 +73,10 @@ public interface SplittableInputSource<T> extends InputSource
* Helper method for ParallelIndexSupervisorTask.
* Most of implementations can simply create a new instance with the given split.
*/
SplittableInputSource<T> withSplit(InputSplit<T> split);
InputSource withSplit(InputSplit<T> split);
default SplitHintSpec getSplitHintSpecOrDefault(@Nullable SplitHintSpec splitHintSpec)
{
return splitHintSpec == null ? DEFAULT_SPLIT_HINT_SPEC : splitHintSpec;
}
}

View File

@ -31,7 +31,6 @@ import java.util.Comparator;
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 final class CollectionUtils
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();
}

View File

@ -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 <T> Stream<T> sequentialStreamFrom(Iterator<T> iterator)
{
final Spliterator<T> spliterator = Spliterators.spliteratorUnknownSize(iterator, Spliterator.DISTINCT);
return StreamSupport.stream(spliterator, false);
}
public static <T> Stream<T> sequentialStreamFrom(Iterable<T> iterable)
{
return StreamSupport.stream(iterable.spliterator(), false);
}
private Streams()
{
}
}

View File

@ -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<Integer, InputFileAttribute> inputAttributeExtractor = InputFileAttribute::new;
final List<List<Integer>> 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<Integer, InputFileAttribute> inputAttributeExtractor = InputFileAttribute::new;
final List<List<Integer>> splits = Lists.newArrayList(
splitHintSpec.split(IntStream.generate(() -> eachInputSize).limit(10).iterator(), inputAttributeExtractor)
);
Assert.assertEquals(10, splits.size());
for (List<Integer> split : splits) {
Assert.assertEquals(1, split.size());
}
}
@Test
public void testEquals()
{
EqualsVerifier.forClass(MaxSizeSplitHintSpec.class).withNonnullFields("maxSplitSize").usingGetClass().verify();
}
}

View File

@ -73,7 +73,7 @@ public class InputEntityIteratingReaderTest
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()
);

View File

@ -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 class LocalInputSourceTest
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<File> files = prepareFiles(10, fileSize);
final LocalInputSource inputSource = new LocalInputSource(null, null, files);
final List<InputSplit<List<File>>> 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<File> 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<File> filesInBaseDir = new ArrayList<>();
for (int i = 0; i < 10; i++) {
filesInBaseDir.add(File.createTempFile("local-input-source", ".data", baseDir));
}
Set<File> files = new HashSet<>(filesInBaseDir.subList(0, 5));
for (int i = 0; i < 3; i++) {
files.add(File.createTempFile("local-input-source", ".data", baseDir));
}
Set<File> expectedFiles = new HashSet<>(filesInBaseDir);
expectedFiles.addAll(files);
File.createTempFile("local-input-source", ".filtered", baseDir);
Iterator<File> fileIterator = new LocalInputSource(baseDir, "*.data", files).getFileIterator();
Set<File> actualFiles = Streams.sequentialStreamFrom(fileIterator).collect(Collectors.toSet());
Assert.assertEquals(expectedFiles, actualFiles);
}
@Test
public void testGetFileIteratorWithOnlyBaseDirIteratingAllFiles() throws IOException
{
File baseDir = temporaryFolder.newFolder();
Set<File> filesInBaseDir = new HashSet<>();
for (int i = 0; i < 10; i++) {
filesInBaseDir.add(File.createTempFile("local-input-source", ".data", baseDir));
}
Iterator<File> fileIterator = new LocalInputSource(baseDir, "*", null).getFileIterator();
Set<File> actualFiles = Streams.sequentialStreamFrom(fileIterator).collect(Collectors.toSet());
Assert.assertEquals(filesInBaseDir, actualFiles);
}
@Test
public void testGetFileIteratorWithOnlyFilesIteratingAllFiles() throws IOException
{
File baseDir = temporaryFolder.newFolder();
Set<File> filesInBaseDir = new HashSet<>();
for (int i = 0; i < 10; i++) {
filesInBaseDir.add(File.createTempFile("local-input-source", ".data", baseDir));
}
Iterator<File> fileIterator = new LocalInputSource(null, null, filesInBaseDir).getFileIterator();
Set<File> actualFiles = Streams.sequentialStreamFrom(fileIterator).collect(Collectors.toSet());
Assert.assertEquals(filesInBaseDir, actualFiles);
}
private static Set<File> prepareFiles(int numFiles, long fileSize)
{
final Set<File> 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;
}
}

View File

@ -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:

View File

@ -23,8 +23,9 @@ import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.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.AzureCloudBlobIterableFactory;
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<AzureEntity>
public class AzureInputSource extends CloudObjectInputSource
{
public static final String SCHEME = "azure";
@ -75,11 +79,14 @@ public class AzureInputSource extends CloudObjectInputSource<AzureEntity>
"AzureCloudBlobIterableFactory"
);
this.inputDataConfig = Preconditions.checkNotNull(inputDataConfig, "AzureInputDataConfig");
this.azureCloudBlobToLocationConverter = Preconditions.checkNotNull(azureCloudBlobToLocationConverter, "AzureCloudBlobToLocationConverter");
this.azureCloudBlobToLocationConverter = Preconditions.checkNotNull(
azureCloudBlobToLocationConverter,
"AzureCloudBlobToLocationConverter"
);
}
@Override
public SplittableInputSource<CloudObjectLocation> withSplit(InputSplit<CloudObjectLocation> split)
public SplittableInputSource<List<CloudObjectLocation>> withSplit(InputSplit<List<CloudObjectLocation>> split)
{
return new AzureInputSource(
storage,
@ -89,32 +96,28 @@ public class AzureInputSource extends CloudObjectInputSource<AzureEntity>
inputDataConfig,
null,
null,
ImmutableList.of(split.get())
split.get()
);
}
@Override
public String toString()
protected AzureEntity createEntity(CloudObjectLocation location)
{
return "AzureInputSource{" +
"uris=" + getUris() +
", prefixes=" + getPrefixes() +
", objects=" + getObjects() +
'}';
return entityFactory.create(location);
}
@Override
protected AzureEntity createEntity(InputSplit<CloudObjectLocation> split)
protected Stream<InputSplit<List<CloudObjectLocation>>> getPrefixesSplitStream(@Nonnull SplitHintSpec splitHintSpec)
{
return entityFactory.create(split.get());
}
@Override
protected Stream<InputSplit<CloudObjectLocation>> getPrefixesSplitStream()
{
return StreamSupport.stream(getIterableObjectsFromPrefixes().spliterator(), false)
.map(o -> azureCloudBlobToLocationConverter.createCloudObjectLocation(o))
.map(InputSplit::new);
final Iterator<List<CloudBlobHolder>> splitIterator = splitHintSpec.split(
getIterableObjectsFromPrefixes().iterator(),
blobHolder -> new InputFileAttribute(blobHolder.getBlobLength())
);
return Streams.sequentialStreamFrom(splitIterator)
.map(objects -> objects.stream()
.map(azureCloudBlobToLocationConverter::createCloudObjectLocation)
.collect(Collectors.toList()))
.map(InputSplit::new);
}
private Iterable<CloudBlobHolder> getIterableObjectsFromPrefixes()
@ -154,4 +157,14 @@ public class AzureInputSource extends CloudObjectInputSource<AzureEntity>
azureCloudBlobToLocationConverter.equals(that.azureCloudBlobToLocationConverter) &&
inputDataConfig.equals(that.inputDataConfig);
}
@Override
public String toString()
{
return "AzureInputSource{" +
"uris=" + getUris() +
", prefixes=" + getPrefixes() +
", objects=" + getObjects() +
'}';
}
}

View File

@ -46,4 +46,9 @@ public class CloudBlobHolder
{
return delegate.getName();
}
public long getBlobLength()
{
return delegate.getProperties().getLength();
}
}

View File

@ -22,6 +22,7 @@ package org.apache.druid.data.input.azure;
import com.google.common.collect.ImmutableList;
import nl.jqno.equalsverifier.EqualsVerifier;
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.SplittableInputSource;
import org.apache.druid.java.util.common.logger.Logger;
@ -41,7 +42,6 @@ import org.junit.Test;
import java.net.URI;
import java.util.Iterator;
import java.util.List;
import java.util.Spliterators;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@ -64,7 +64,7 @@ public class AzureInputSourceTest extends EasyMockSupport
private AzureCloudBlobHolderToCloudObjectLocationConverter azureCloudBlobToLocationConverter;
private AzureInputDataConfig inputDataConfig;
private InputSplit<CloudObjectLocation> inputSplit;
private InputSplit<List<CloudObjectLocation>> inputSplit;
private AzureEntity azureEntity1;
private CloudBlobHolder cloudBlobDruid1;
private AzureCloudBlobIterable azureCloudBlobIterable;
@ -114,7 +114,7 @@ public class AzureInputSourceTest extends EasyMockSupport
public void test_createEntity_returnsExpectedEntity()
{
EasyMock.expect(entityFactory.create(CLOUD_OBJECT_LOCATION_1)).andReturn(azureEntity1);
EasyMock.expect(inputSplit.get()).andReturn(CLOUD_OBJECT_LOCATION_1);
EasyMock.expect(inputSplit.get()).andReturn(ImmutableList.of(CLOUD_OBJECT_LOCATION_1)).times(2);
replayAll();
List<CloudObjectLocation> objects = ImmutableList.of(CLOUD_OBJECT_LOCATION_1);
@ -129,7 +129,8 @@ public class AzureInputSourceTest extends EasyMockSupport
objects
);
AzureEntity actualAzureEntity = azureInputSource.createEntity(inputSplit);
Assert.assertEquals(1, inputSplit.get().size());
AzureEntity actualAzureEntity = azureInputSource.createEntity(inputSplit.get().get(0));
Assert.assertSame(azureEntity1, actualAzureEntity);
verifyAll();
}
@ -138,16 +139,16 @@ public class AzureInputSourceTest extends EasyMockSupport
public void test_getPrefixesSplitStream_successfullyCreatesCloudLocation_returnsExpectedLocations()
{
List<URI> prefixes = ImmutableList.of(PREFIX_URI);
List<CloudObjectLocation> expectedCloudLocations = ImmutableList.of(CLOUD_OBJECT_LOCATION_1);
List<List<CloudObjectLocation>> expectedCloudLocations = ImmutableList.of(ImmutableList.of(CLOUD_OBJECT_LOCATION_1));
List<CloudBlobHolder> expectedCloudBlobs = ImmutableList.of(cloudBlobDruid1);
Iterator<CloudBlobHolder> expectedCloudBlobsIterator = expectedCloudBlobs.iterator();
EasyMock.expect(inputDataConfig.getMaxListingLength()).andReturn(MAX_LISTING_LENGTH);
EasyMock.expect(azureCloudBlobIterableFactory.create(prefixes, MAX_LISTING_LENGTH)).andReturn(
azureCloudBlobIterable);
EasyMock.expect(azureCloudBlobIterable.spliterator())
.andReturn(Spliterators.spliteratorUnknownSize(expectedCloudBlobsIterator, 0));
EasyMock.expect(azureCloudBlobIterable.iterator()).andReturn(expectedCloudBlobsIterator);
EasyMock.expect(azureCloudBlobToLocationConverter.createCloudObjectLocation(cloudBlobDruid1))
.andReturn(CLOUD_OBJECT_LOCATION_1);
EasyMock.expect(cloudBlobDruid1.getBlobLength()).andReturn(100L);
replayAll();
azureInputSource = new AzureInputSource(
@ -161,10 +162,12 @@ public class AzureInputSourceTest extends EasyMockSupport
EMPTY_OBJECTS
);
Stream<InputSplit<CloudObjectLocation>> cloudObjectStream = azureInputSource.getPrefixesSplitStream();
Stream<InputSplit<List<CloudObjectLocation>>> cloudObjectStream = azureInputSource.getPrefixesSplitStream(
new MaxSizeSplitHintSpec(1L)
);
List<CloudObjectLocation> actualCloudLocationList = cloudObjectStream.map(split -> split.get())
.collect(Collectors.toList());
List<List<CloudObjectLocation>> actualCloudLocationList = cloudObjectStream.map(InputSplit::get)
.collect(Collectors.toList());
verifyAll();
Assert.assertEquals(expectedCloudLocations, actualCloudLocationList);
}
@ -173,7 +176,7 @@ public class AzureInputSourceTest extends EasyMockSupport
public void test_withSplit_constructsExpectedInputSource()
{
List<URI> prefixes = ImmutableList.of(PREFIX_URI);
EasyMock.expect(inputSplit.get()).andReturn(CLOUD_OBJECT_LOCATION_1);
EasyMock.expect(inputSplit.get()).andReturn(ImmutableList.of(CLOUD_OBJECT_LOCATION_1));
replayAll();
azureInputSource = new AzureInputSource(
@ -187,7 +190,7 @@ public class AzureInputSourceTest extends EasyMockSupport
EMPTY_OBJECTS
);
SplittableInputSource<CloudObjectLocation> newInputSource = azureInputSource.withSplit(inputSplit);
SplittableInputSource<List<CloudObjectLocation>> newInputSource = azureInputSource.withSplit(inputSplit);
Assert.assertTrue(newInputSource.isSplittable());
verifyAll();
}

View File

@ -23,24 +23,34 @@ import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.api.services.storage.model.StorageObject;
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;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.storage.google.GoogleInputDataConfig;
import org.apache.druid.storage.google.GoogleStorage;
import org.apache.druid.storage.google.GoogleUtils;
import org.apache.druid.utils.Streams;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.math.BigInteger;
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 GoogleCloudStorageInputSource extends CloudObjectInputSource<GoogleCloudStorageEntity>
public class GoogleCloudStorageInputSource extends CloudObjectInputSource
{
static final String SCHEME = "gs";
private static final Logger LOG = new Logger(GoogleCloudStorageInputSource.class);
private final GoogleStorage storage;
private final GoogleInputDataConfig inputDataConfig;
@ -59,23 +69,50 @@ public class GoogleCloudStorageInputSource extends CloudObjectInputSource<Google
}
@Override
protected GoogleCloudStorageEntity createEntity(InputSplit<CloudObjectLocation> split)
protected InputEntity createEntity(CloudObjectLocation location)
{
return new GoogleCloudStorageEntity(storage, split.get());
return new GoogleCloudStorageEntity(storage, location);
}
@Override
protected Stream<InputSplit<CloudObjectLocation>> getPrefixesSplitStream()
protected Stream<InputSplit<List<CloudObjectLocation>>> getPrefixesSplitStream(@Nonnull SplitHintSpec splitHintSpec)
{
return StreamSupport.stream(storageObjectIterable().spliterator(), false)
.map(this::byteSourceFromStorageObject)
.map(InputSplit::new);
final Iterator<List<StorageObject>> 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<CloudObjectLocation> withSplit(InputSplit<CloudObjectLocation> split)
public SplittableInputSource<List<CloudObjectLocation>> withSplit(InputSplit<List<CloudObjectLocation>> 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 @@ public class GoogleCloudStorageInputSource extends CloudObjectInputSource<Google
private Iterable<StorageObject> storageObjectIterable()
{
return () ->
GoogleUtils.lazyFetchingStorageObjectsIterator(storage, getPrefixes().iterator(), inputDataConfig.getMaxListingLength());
GoogleUtils.lazyFetchingStorageObjectsIterator(
storage,
getPrefixes().iterator(),
inputDataConfig.getMaxListingLength()
);
}
@Override

View File

@ -35,6 +35,7 @@ import org.apache.druid.data.input.InputRow;
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 org.junit.Test;
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<CloudObjectLocation> EXPECTED_OBJECTS =
EXPECTED_URIS.stream().map(CloudObjectLocation::new).collect(Collectors.toList());
private static final List<List<CloudObjectLocation>> EXPECTED_OBJECTS =
EXPECTED_URIS.stream()
.map(uri -> Collections.singletonList(new CloudObjectLocation(uri)))
.collect(Collectors.toList());
private static final List<URI> PREFIXES = Arrays.asList(
URI.create("gs://foo/bar"),
@ -142,7 +147,7 @@ public class GoogleCloudStorageInputSourceTest extends InitializedNullHandlingTe
GoogleCloudStorageInputSource inputSource =
new GoogleCloudStorageInputSource(STORAGE, INPUT_DATA_CONFIG, EXPECTED_URIS, ImmutableList.of(), null);
Stream<InputSplit<CloudObjectLocation>> splits = inputSource.createSplits(
Stream<InputSplit<List<CloudObjectLocation>>> splits = inputSource.createSplits(
new JsonInputFormat(JSONPathSpec.DEFAULT, null),
null
);
@ -163,14 +168,39 @@ public class GoogleCloudStorageInputSourceTest extends InitializedNullHandlingTe
GoogleCloudStorageInputSource inputSource =
new GoogleCloudStorageInputSource(STORAGE, INPUT_DATA_CONFIG, null, PREFIXES, null);
Stream<InputSplit<CloudObjectLocation>> splits = inputSource.createSplits(
Stream<InputSplit<List<CloudObjectLocation>>> 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<InputSplit<List<CloudObjectLocation>>> 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 @@ public class GoogleCloudStorageInputSourceTest extends InitializedNullHandlingTe
StorageObject s = new StorageObject();
s.setBucket(bucket);
s.setName(uri.getPath());
s.setSize(BigInteger.valueOf(CONTENT.length));
mockObjects.add(s);
}
Objects response = new Objects();

View File

@ -24,7 +24,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 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.InputEntityIteratingReader;
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.IOException;
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<Path>
public class HdfsInputSource extends AbstractInputSource implements SplittableInputSource<List<Path>>
{
private static final String PROP_PATHS = "paths";
@ -150,28 +154,38 @@ public class HdfsInputSource extends AbstractInputSource implements SplittableIn
@Nullable File temporaryDirectory
)
{
final Stream<InputSplit<Path>> 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<InputSplit<Path>> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec)
public Stream<InputSplit<List<Path>>> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec)
throws IOException
{
cachePathsIfNeeded();
return cachedPaths.stream().map(InputSplit::new);
final Iterator<List<Path>> 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 class HdfsInputSource extends AbstractInputSource implements SplittableIn
}
@Override
public SplittableInputSource<Path> withSplit(InputSplit<Path> split)
public SplittableInputSource<List<Path>> withSplit(InputSplit<List<Path>> split)
{
return new HdfsInputSource(split.get().toString(), configuration);
}

View File

@ -27,6 +27,8 @@ import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputRowSchema;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.InputSourceReader;
import org.apache.druid.data.input.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 class HdfsInputSourceTest
@Test
public void hasCorrectSplits() throws IOException
{
Set<Path> actualPaths = target.createSplits(null, null)
.map(split -> Path.getPathWithoutSchemeAndAuthority(split.get()))
// Set maxSplitSize to 1 so that each inputSplit has only one object
List<InputSplit<List<Path>>> splits = target.createSplits(null, new MaxSizeSplitHintSpec(1L))
.collect(Collectors.toList());
splits.forEach(split -> Assert.assertEquals(1, split.get().size()));
Set<Path> 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<InputSplit<List<Path>>> 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 class HdfsInputSourceTest
@Test
public void hasCorrectSplits() throws IOException
{
List<Path> paths = target.createSplits(null, null)
.map(split -> Path.getPathWithoutSchemeAndAuthority(split.get()))
.collect(Collectors.toList());
Assert.assertTrue(String.valueOf(paths), paths.isEmpty());
List<InputSplit<List<Path>>> splits = target.createSplits(null, null)
.collect(Collectors.toList());
Assert.assertTrue(String.valueOf(splits), splits.isEmpty());
}
@Test

View File

@ -24,8 +24,10 @@ import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.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.S3InputDataConfig;
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<S3Entity>
public class S3InputSource extends CloudObjectInputSource
{
private final ServerSideEncryptingAmazonS3 s3Client;
private final S3InputDataConfig inputDataConfig;
@ -60,23 +65,30 @@ public class S3InputSource extends CloudObjectInputSource<S3Entity>
}
@Override
protected S3Entity createEntity(InputSplit<CloudObjectLocation> split)
protected InputEntity createEntity(CloudObjectLocation location)
{
return new S3Entity(s3Client, split.get());
return new S3Entity(s3Client, location);
}
@Override
protected Stream<InputSplit<CloudObjectLocation>> getPrefixesSplitStream()
protected Stream<InputSplit<List<CloudObjectLocation>>> getPrefixesSplitStream(@Nonnull SplitHintSpec splitHintSpec)
{
return StreamSupport.stream(getIterableObjectsFromPrefixes().spliterator(), false)
.map(S3Utils::summaryToCloudObjectLocation)
.map(InputSplit::new);
final Iterator<List<S3ObjectSummary>> 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<CloudObjectLocation> withSplit(InputSplit<CloudObjectLocation> split)
public SplittableInputSource<List<CloudObjectLocation>> withSplit(InputSplit<List<CloudObjectLocation>> split)
{
return new S3InputSource(s3Client, inputDataConfig, null, null, ImmutableList.of(split.get()));
return new S3InputSource(s3Client, inputDataConfig, null, null, split.get());
}
@Override

View File

@ -43,6 +43,7 @@ import org.apache.druid.data.input.InputRow;
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.ByteArrayOutputStream;
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<CloudObjectLocation> EXPECTED_COORDS =
EXPECTED_URIS.stream().map(CloudObjectLocation::new).collect(Collectors.toList());
private static final List<List<CloudObjectLocation>> EXPECTED_COORDS =
EXPECTED_URIS.stream()
.map(uri -> Collections.singletonList(new CloudObjectLocation(uri)))
.collect(Collectors.toList());
private static final List<URI> PREFIXES = Arrays.asList(
URI.create("s3://foo/bar"),
@ -221,7 +225,7 @@ public class S3InputSourceTest extends InitializedNullHandlingTest
{
S3InputSource inputSource = new S3InputSource(SERVICE, INPUT_DATA_CONFIG, EXPECTED_URIS, null, null);
Stream<InputSplit<CloudObjectLocation>> splits = inputSource.createSplits(
Stream<InputSplit<List<CloudObjectLocation>>> splits = inputSource.createSplits(
new JsonInputFormat(JSONPathSpec.DEFAULT, null),
null
);
@ -239,15 +243,37 @@ public class S3InputSourceTest extends InitializedNullHandlingTest
S3InputSource inputSource = new S3InputSource(SERVICE, INPUT_DATA_CONFIG, null, PREFIXES, null);
Stream<InputSplit<CloudObjectLocation>> splits = inputSource.createSplits(
Stream<InputSplit<List<CloudObjectLocation>>> 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<InputSplit<List<CloudObjectLocation>>> 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 @@ public class S3InputSourceTest extends InitializedNullHandlingTest
final S3ObjectSummary objectSummary = new S3ObjectSummary();
objectSummary.setBucketName(bucket);
objectSummary.setKey(key);
objectSummary.setSize(CONTENT.length);
result.getObjectSummaries().add(objectSummary);
}

View File

@ -22,6 +22,7 @@ package org.apache.druid.indexing.common.task.batch.parallel;
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 @@ import java.util.Map;
/**
* 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<T extends Task, R extends SubTaskReport>
extends ParallelIndexPhaseRunner<T, R>
@ -85,7 +86,7 @@ abstract class InputSourceSplitParallelIndexTaskRunner<T extends Task, R extends
final SubTaskSpec<T> newTaskSpec(InputSplit split)
{
final FirehoseFactory firehoseFactory;
final SplittableInputSource inputSource;
final InputSource inputSource;
if (baseInputSource instanceof FirehoseFactoryToInputSourceAdaptor) {
firehoseFactory = ((FirehoseFactoryToInputSourceAdaptor) baseInputSource).getFirehoseFactory().withSplit(split);
inputSource = null;

View File

@ -23,6 +23,7 @@ import com.google.common.annotations.VisibleForTesting;
import org.apache.druid.client.indexing.IndexingServiceClient;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
import org.apache.druid.data.input.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 @@ class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner<Single
SubTaskSpec<SinglePhaseSubTask> newTaskSpec(InputSplit split)
{
final FirehoseFactory firehoseFactory;
final SplittableInputSource inputSource;
final InputSource inputSource;
if (baseInputSource instanceof FirehoseFactoryToInputSourceAdaptor) {
firehoseFactory = ((FirehoseFactoryToInputSourceAdaptor) baseInputSource).getFirehoseFactory().withSplit(split);
inputSource = null;

View File

@ -290,12 +290,14 @@ public class IngestSegmentFirehoseFactory implements FiniteFirehoseFactory<Input
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
)
);
}

View File

@ -24,7 +24,9 @@ import com.fasterxml.jackson.annotation.JsonProperty;
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 class 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 class WindowedSegmentId
@JsonProperty
public List<Interval> 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 +
'}';
}
}

View File

@ -25,13 +25,16 @@ import com.fasterxml.jackson.annotation.JsonInclude;
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.TimelineObjectHolder;
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.ArrayList;
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 @@ public class DruidInputSource extends AbstractInputSource implements SplittableI
final SegmentLoader segmentLoader = segmentLoaderFactory.manufacturate(temporaryDirectory);
final List<TimelineObjectHolder<String, DataSegment>> timeline = createTimeline();
final Stream<InputEntity> entityStream = createTimeline()
final Stream<InputEntity> entityStream = timeline
.stream()
.flatMap(holder -> {
final PartitionHolder<DataSegment> partitionHolder = holder.getObject();
@ -205,7 +209,7 @@ public class DruidInputSource extends AbstractInputSource implements SplittableI
return new InputEntityIteratingReader(
inputRowSchema,
inputFormat,
entityStream,
entityStream.iterator(),
temporaryDirectory
);
}
@ -228,13 +232,15 @@ public class DruidInputSource extends AbstractInputSource implements SplittableI
// 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 class DruidInputSource extends AbstractInputSource implements SplittableI
// 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 class DruidInputSource extends AbstractInputSource implements SplittableI
return false;
}
public static List<InputSplit<List<WindowedSegmentId>>> createSplits(
public static Iterator<InputSplit<List<WindowedSegmentId>>> createSplits(
CoordinatorClient coordinatorClient,
RetryPolicyFactory retryPolicyFactory,
String dataSource,
@ -289,70 +297,52 @@ public class DruidInputSource extends AbstractInputSource implements SplittableI
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<TimelineObjectHolder<String, DataSegment>> timelineSegments = getTimelineForInterval(
coordinatorClient,
retryPolicyFactory,
dataSource,
interval
);
final Map<WindowedSegmentId, Long> 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<InputSplit<List<WindowedSegmentId>>> splits = new ArrayList<>();
List<WindowedSegmentId> currentSplit = new ArrayList<>();
private static Map<WindowedSegmentId, Long> createWindowedSegmentIdFromTimeline(
List<TimelineObjectHolder<String, DataSegment>> timelineHolders
)
{
Map<DataSegment, WindowedSegmentId> windowedSegmentIds = new HashMap<>();
long bytesInCurrentSplit = 0;
for (TimelineObjectHolder<String, DataSegment> timelineHolder : timelineSegments) {
for (PartitionChunk<DataSegment> 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<Interval> 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<String, DataSegment> holder : timelineHolders) {
for (PartitionChunk<DataSegment> 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<WindowedSegmentId, Long> segmentSizeMap = new HashMap<>();
windowedSegmentIds.forEach((segment, segmentId) -> segmentSizeMap.put(segmentId, segment.getSize()));
return segmentSizeMap;
}
public static List<TimelineObjectHolder<String, DataSegment>> getTimelineForInterval(

View File

@ -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 class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
{
runIndexTask();
List<InputSplit<List<WindowedSegmentId>>> 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<InputSplit<List<WindowedSegmentId>>> 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<DataSegment> segments = new ArrayList<>(

View File

@ -1637,15 +1637,16 @@ public class IndexTaskTest extends IngestionTestBase
}
@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,

View File

@ -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 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
null,
null,
null,
null,
new MaxSizeSplitHintSpec(1L), // set maxSplitSize to 1 so that each split has only one file.
partitionsSpec,
null,
null,

View File

@ -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();
}
}

View File

@ -37,6 +37,7 @@ import java.util.function.Function;
@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";

View File

@ -38,6 +38,7 @@ import java.util.function.Function;
@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";

View File

@ -66,6 +66,10 @@
"type": "index_parallel",
"maxNumConcurrentSubTasks": 10,
"forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%",
"splitHintSpec": {
"type": "maxSize",
"maxSplitSize": 1
},
"partitionsSpec": %%PARTITIONS_SPEC%%
}
}

View File

@ -59,6 +59,10 @@
"type": "index_parallel",
"maxNumConcurrentSubTasks": 10,
"forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%",
"splitHintSpec": {
"type": "maxSize",
"maxSplitSize": 1
},
"partitionsSpec": %%PARTITIONS_SPEC%%
}
}

View File

@ -65,6 +65,10 @@
"type": "index_parallel",
"maxNumConcurrentSubTasks": 10,
"forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%",
"splitHintSpec": {
"type": "maxSize",
"maxSplitSize": 1
},
"partitionsSpec": %%PARTITIONS_SPEC%%
}
}