fetchFuture = fetchExecutor.submit(() -> {
- fetch();
- return null;
- });
- fetchFutures.add(fetchFuture);
- }
- }
-
- /**
- * Fetch objects to a local disk up to {@link FetchConfig#maxFetchCapacityBytes}.
- * This method is not thread safe and must be called by a single thread. Note that even
- * {@link FetchConfig#maxFetchCapacityBytes} is 0, at least 1 file is always fetched.
- * This is for simplifying design, and should be improved when our client implementations for cloud storages
- * like S3 support range scan.
- *
- * This method is called by {@link #fetchExecutor} if prefetch is enabled. Otherwise, it is called by the same
- * thread.
- */
- private void fetch() throws Exception
- {
- for (; nextFetchIndex < objects.size()
- && fetchedBytes.get() <= fetchConfig.getMaxFetchCapacityBytes(); nextFetchIndex++) {
- final T object = objects.get(nextFetchIndex);
- LOG.info("Fetching [%d]th object[%s], fetchedBytes[%d]", nextFetchIndex, object, fetchedBytes.get());
- final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, temporaryDirectory);
- fetchedBytes.addAndGet(download(object, outFile));
- fetchedFiles.put(new FetchedFile<>(object, outFile, getFileCloser(outFile, fetchedBytes)));
- }
- }
-
- /**
- * Downloads an object into a file. The download process could be retried depending on the object source.
- *
- * @param object an object to be downloaded
- * @param outFile a file which the object data is stored
- *
- * @return number of downloaded bytes
- */
- protected abstract long download(T object, File outFile) throws IOException;
-
- /**
- * Generates an instance of {@link OpenObject} for the given object.
- */
- protected abstract OpenObject generateOpenObject(T object) throws IOException;
-
- protected FetchConfig getFetchConfig()
- {
- return fetchConfig;
- }
-
- @Override
- public boolean hasNext()
- {
- return numRemainingObjects > 0;
- }
-
- @Override
- public OpenObject next()
- {
- if (!hasNext()) {
- throw new NoSuchElementException();
- }
-
- // If fetch() fails, hasNext() always returns true and next() is always called. The below method checks that
- // fetch() threw an exception and propagates it if exists.
- checkFetchException(false);
-
- try {
- final OpenObject openObject = prefetchEnabled ? openObjectFromLocal() : openObjectFromRemote();
- numRemainingObjects--;
- return openObject;
- }
- catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- private void checkFetchException(boolean wait)
- {
- try {
- for (Future fetchFuture; (fetchFuture = fetchFutures.poll()) != null; ) {
- if (wait) {
- fetchFuture.get(fetchConfig.getFetchTimeout(), TimeUnit.MILLISECONDS);
- } else {
- if (fetchFuture.isDone()) {
- fetchFuture.get();
- } else {
- fetchFutures.addFirst(fetchFuture);
- break;
- }
- }
- }
- }
- catch (InterruptedException | ExecutionException e) {
- throw new RuntimeException(e);
- }
- catch (TimeoutException e) {
- throw new ISE(e, "Failed to fetch, but cannot check the reason in [%d] ms", fetchConfig.getFetchTimeout());
- }
- }
-
- private OpenObject openObjectFromLocal() throws IOException
- {
- final FetchedFile fetchedFile;
-
- if (!fetchedFiles.isEmpty()) {
- // If there are already fetched files, use them
- fetchedFile = fetchedFiles.poll();
- } else {
- // Otherwise, wait for fetching
- try {
- fetchIfNeeded(fetchedBytes.get());
- fetchedFile = fetchedFiles.poll(fetchConfig.getFetchTimeout(), TimeUnit.MILLISECONDS);
- if (fetchedFile == null) {
- // Check the latest fetch is failed
- checkFetchException(true);
- // Or throw a timeout exception
- throw new RuntimeException(new TimeoutException());
- }
- }
- catch (InterruptedException e) {
- throw new RuntimeException(e);
- }
- }
- final FetchedFile maybeCached = cacheIfPossible(fetchedFile);
- // trigger fetch again for subsequent next() calls
- fetchIfNeeded(fetchedBytes.get());
- return new OpenObject<>(maybeCached);
- }
-
- private OpenObject openObjectFromRemote() throws IOException
- {
- if (fetchedFiles.size() > 0) {
- // If fetchedFiles is not empty even though prefetching is disabled, they should be cached files.
- // We use them first. See (*).
- return new OpenObject<>(fetchedFiles.poll());
- } else if (cacheManager.cacheable()) {
- // If cache is enabled, first download an object to local storage and cache it.
- try {
- // Since maxFetchCapacityBytes is 0, at most one file is fetched.
- fetch();
- FetchedFile fetchedFile = fetchedFiles.poll();
- if (fetchedFile == null) {
- throw new ISE("Cannot fetch object[%s]", objects.get(nextFetchIndex - 1));
- }
- final FetchedFile cached = cacheIfPossible(fetchedFile);
- return new OpenObject<>(cached);
- }
- catch (Exception e) {
- throw new RuntimeException(e);
- }
- } else {
- final T object = objects.get(nextFetchIndex);
- LOG.info("Reading [%d]th object[%s]", nextFetchIndex, object);
- nextFetchIndex++;
- return generateOpenObject(object);
- }
- }
-
- private FetchedFile cacheIfPossible(FetchedFile fetchedFile)
- {
- if (cacheManager.cacheable()) {
- final FetchedFile cachedFile = cacheManager.cache(fetchedFile);
- // If the fetchedFile is cached, make a room for fetching more data immediately.
- // This is because cache space and fetch space are separated.
- fetchedBytes.addAndGet(-fetchedFile.length());
- return cachedFile;
- } else {
- return fetchedFile;
- }
- }
-
- private static Closeable getFileCloser(
- final File file,
- final AtomicLong fetchedBytes
- )
- {
- return () -> {
- final long fileSize = file.length();
- file.delete();
- fetchedBytes.addAndGet(-fileSize);
- };
- }
-}
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/FileFetcher.java b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/FileFetcher.java
deleted file mode 100644
index 7d869dacf719..000000000000
--- a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/FileFetcher.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.data.input.impl.prefetch;
-
-import com.google.common.base.Predicate;
-import org.apache.druid.data.input.impl.RetryingInputStream;
-import org.apache.druid.java.util.common.FileUtils;
-import org.apache.druid.java.util.common.StringUtils;
-
-import javax.annotation.Nullable;
-import java.io.Closeable;
-import java.io.File;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-
-/**
- * A file fetcher used by {@link PrefetchableTextFilesFirehoseFactory}.
- * See the javadoc of {@link PrefetchableTextFilesFirehoseFactory} for more details.
- */
-public class FileFetcher extends Fetcher
-{
- private static final int BUFFER_SIZE = 1024 * 4;
- private final ObjectOpenFunction openObjectFunction;
- private final Predicate retryCondition;
- private final byte[] buffer;
-
- FileFetcher(
- CacheManager cacheManager,
- List objects,
- ExecutorService fetchExecutor,
- @Nullable File temporaryDirectory,
- FetchConfig fetchConfig,
- ObjectOpenFunction openObjectFunction,
- Predicate retryCondition
- )
- {
-
- super(
- cacheManager,
- objects,
- fetchExecutor,
- temporaryDirectory,
- fetchConfig
- );
-
- this.openObjectFunction = openObjectFunction;
- this.retryCondition = retryCondition;
- this.buffer = new byte[BUFFER_SIZE];
- }
-
- /**
- * Downloads an object. It retries downloading {@link FetchConfig#maxFetchRetry}
- * times and throws an exception.
- *
- * @param object an object to be downloaded
- * @param outFile a file which the object data is stored
- *
- * @return number of downloaded bytes
- */
- @Override
- protected long download(T object, File outFile) throws IOException
- {
- return FileUtils.copyLarge(
- object,
- openObjectFunction,
- outFile,
- buffer,
- retryCondition,
- getFetchConfig().getMaxFetchRetry() + 1,
- StringUtils.format("Failed to download object[%s]", object)
- );
- }
-
- /**
- * Generates an instance of {@link OpenObject} for which the underlying stream may be re-opened and retried
- * based on the exception and retry condition.
- */
- @Override
- protected OpenObject generateOpenObject(T object) throws IOException
- {
- return new OpenObject<>(
- object,
- new RetryingInputStream<>(object, openObjectFunction, retryCondition, getFetchConfig().getMaxFetchRetry()),
- getNoopCloser()
- );
- }
-
- private static Closeable getNoopCloser()
- {
- return () -> {
- };
- }
-}
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/OpenObject.java b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/OpenObject.java
deleted file mode 100644
index 07b3125099c4..000000000000
--- a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/OpenObject.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.data.input.impl.prefetch;
-
-import org.apache.commons.io.FileUtils;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * A class containing meta information about an opened object. This class is used to put related objects together. It
- * contains an original object, an objectStream from the object, and a resourceCloser which knows how to release
- * associated resources on closing.
- *
- * {@link PrefetchableTextFilesFirehoseFactory.ResourceCloseableLineIterator} consumes the objectStream and closes
- * it with the resourceCloser.
- */
-public class OpenObject
-{
- // Original object
- private final T object;
- // Input stream from the object
- private final InputStream objectStream;
- // Closer which is called when the file is not needed anymore. Usually this deletes the file except for cached files.
- private final Closeable resourceCloser;
-
- public OpenObject(FetchedFile fetchedFile) throws IOException
- {
- this(fetchedFile.getObject(), FileUtils.openInputStream(fetchedFile.getFile()), fetchedFile.getResourceCloser());
- }
-
- public OpenObject(T object, InputStream objectStream, Closeable resourceCloser)
- {
- this.object = object;
- this.objectStream = objectStream;
- this.resourceCloser = resourceCloser;
- }
-
- public T getObject()
- {
- return object;
- }
-
- public InputStream getObjectStream()
- {
- return objectStream;
- }
-
- public Closeable getResourceCloser()
- {
- return resourceCloser;
- }
-}
diff --git a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactory.java b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactory.java
deleted file mode 100644
index 50eefae73665..000000000000
--- a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactory.java
+++ /dev/null
@@ -1,318 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.data.input.impl.prefetch;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
-import com.google.common.collect.ImmutableList;
-import org.apache.commons.io.LineIterator;
-import org.apache.druid.data.input.Firehose;
-import org.apache.druid.data.input.impl.AbstractTextFilesFirehoseFactory;
-import org.apache.druid.data.input.impl.FileIteratingFirehose;
-import org.apache.druid.data.input.impl.RetryingInputStream;
-import org.apache.druid.data.input.impl.StringInputRowParser;
-import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.concurrent.Execs;
-import org.apache.druid.java.util.common.logger.Logger;
-
-import javax.annotation.Nullable;
-import java.io.Closeable;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.Reader;
-import java.nio.charset.StandardCharsets;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-
-/**
- * PrefetchableTextFilesFirehoseFactory is an abstract firehose factory for reading text files. The firehose returned
- * by this class provides three key functionalities.
- *
- *
- * - Caching: for the first call of {@link #connect(StringInputRowParser, File)}, it caches objects in a local disk
- * up to maxCacheCapacityBytes. These caches are NOT deleted until the process terminates, and thus can be used for
- * future reads.
- *
- * - Fetching: when it reads all cached data, it fetches remaining objects into a local disk and reads data from
- * them. For the performance reason, prefetch technique is used, that is, when the size of remaining fetched data is
- * smaller than {@link FetchConfig#prefetchTriggerBytes}, a background prefetch thread automatically starts to fetch remaining
- * objects.
- *
- * - Retry: if an exception occurs while downloading an object, it retries again up to {@link FetchConfig#maxFetchRetry}.
- *
- *
- * This implementation can be useful when the cost for reading input objects is large as reading from AWS S3 because
- * batch tasks like IndexTask or HadoopIndexTask can read the whole data twice for determining partition specs and
- * generating segments if the intervals of GranularitySpec is not specified.
- *
- * Prefetching can be turned on/off by setting maxFetchCapacityBytes. Depending on prefetching is enabled or
- * disabled, the behavior of the firehose is different like below.
- *
- *
- * 1. If prefetch is enabled, this firehose can fetch input objects in background.
- *
- * 2. When next() is called, it first checks that there are already fetched files in local storage.
- *
- * 2.1 If exists, it simply chooses a fetched file and returns a {@link LineIterator} reading that file.
- *
- * 2.2 If there is no fetched files in local storage but some objects are still remained to be read, the firehose
- * fetches one of input objects in background immediately. If an IOException occurs while downloading the object,
- * it retries up to the maximum retry count. Finally, the firehose returns a {@link LineIterator} only when the
- * download operation is successfully finished.
- *
- * 3. If prefetch is disabled, the firehose returns a {@link LineIterator} which directly reads the stream opened by
- * {@link #openObjectStream}. If there is an IOException, it will throw it and the read will fail.
- */
-public abstract class PrefetchableTextFilesFirehoseFactory
- extends AbstractTextFilesFirehoseFactory
-{
- private static final Logger LOG = new Logger(PrefetchableTextFilesFirehoseFactory.class);
-
- private static final CacheManager DISABLED_CACHE_MANAGER = new CacheManager(0);
- private static final FetchConfig DISABLED_PREFETCH_CONFIG = new FetchConfig(0L, 0L, 0L, 0L, 0);
-
- private final CacheManager cacheManager;
- private final FetchConfig fetchConfig;
-
- private List objects;
-
- public PrefetchableTextFilesFirehoseFactory(
- @Nullable Long maxCacheCapacityBytes,
- @Nullable Long maxFetchCapacityBytes,
- @Nullable Long prefetchTriggerBytes,
- @Nullable Long fetchTimeout,
- @Nullable Integer maxFetchRetry
- )
- {
- this.fetchConfig = new FetchConfig(
- maxCacheCapacityBytes,
- maxFetchCapacityBytes,
- prefetchTriggerBytes,
- fetchTimeout,
- maxFetchRetry
- );
- this.cacheManager = new CacheManager<>(
- fetchConfig.getMaxCacheCapacityBytes()
- );
- }
-
- @JsonProperty
- public long getMaxCacheCapacityBytes()
- {
- return cacheManager.getMaxCacheCapacityBytes();
- }
-
- @JsonProperty
- public long getMaxFetchCapacityBytes()
- {
- return fetchConfig.getMaxFetchCapacityBytes();
- }
-
- @JsonProperty
- public long getPrefetchTriggerBytes()
- {
- return fetchConfig.getPrefetchTriggerBytes();
- }
-
- @JsonProperty
- public long getFetchTimeout()
- {
- return fetchConfig.getFetchTimeout();
- }
-
- @JsonProperty
- public int getMaxFetchRetry()
- {
- return fetchConfig.getMaxFetchRetry();
- }
-
- @VisibleForTesting
- CacheManager getCacheManager()
- {
- return cacheManager;
- }
-
- @Override
- public Firehose connect(StringInputRowParser firehoseParser, @Nullable File temporaryDirectory) throws IOException
- {
- return connectInternal(firehoseParser, temporaryDirectory, this.fetchConfig, this.cacheManager);
- }
-
- @Override
- public Firehose connectForSampler(StringInputRowParser parser, @Nullable File temporaryDirectory) throws IOException
- {
- return connectInternal(parser, temporaryDirectory, DISABLED_PREFETCH_CONFIG, DISABLED_CACHE_MANAGER);
- }
-
- private Firehose connectInternal(
- StringInputRowParser firehoseParser,
- @Nullable File temporaryDirectory,
- FetchConfig fetchConfig,
- CacheManager cacheManager
- ) throws IOException
- {
- if (objects == null) {
- objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "objects"));
- }
-
- if (cacheManager.isEnabled() || fetchConfig.getMaxFetchCapacityBytes() > 0) {
- Preconditions.checkNotNull(temporaryDirectory, "temporaryDirectory");
- Preconditions.checkArgument(
- temporaryDirectory.exists(),
- "temporaryDirectory[%s] does not exist",
- temporaryDirectory
- );
- Preconditions.checkArgument(
- temporaryDirectory.isDirectory(),
- "temporaryDirectory[%s] is not a directory",
- temporaryDirectory
- );
- }
-
- LOG.info("Create a new firehose for [%d] objects", objects.size());
-
- // fetchExecutor is responsible for background data fetching
- final ExecutorService fetchExecutor = Execs.singleThreaded("firehose_fetch_%d");
- final FileFetcher fetcher = new FileFetcher(
- cacheManager,
- objects,
- fetchExecutor,
- temporaryDirectory,
- fetchConfig,
- new ObjectOpenFunction()
- {
- @Override
- public InputStream open(T object) throws IOException
- {
- return openObjectStream(object);
- }
-
- @Override
- public InputStream open(T object, long start) throws IOException
- {
- return openObjectStream(object, start);
- }
- },
- getRetryCondition()
- );
-
- return new FileIteratingFirehose(
- new Iterator()
- {
- @Override
- public boolean hasNext()
- {
- return fetcher.hasNext();
- }
-
- @Override
- public LineIterator next()
- {
- if (!hasNext()) {
- throw new NoSuchElementException();
- }
-
- final OpenObject openObject = fetcher.next();
- try {
- return new ResourceCloseableLineIterator(
- new InputStreamReader(
- wrapObjectStream(openObject.getObject(), openObject.getObjectStream()),
- StandardCharsets.UTF_8
- ),
- openObject.getResourceCloser()
- );
- }
- catch (IOException e) {
- try {
- openObject.getResourceCloser().close();
- }
- catch (Throwable t) {
- e.addSuppressed(t);
- }
- throw new RuntimeException(e);
- }
- }
- },
- firehoseParser,
- () -> {
- fetchExecutor.shutdownNow();
- try {
- Preconditions.checkState(fetchExecutor.awaitTermination(
- fetchConfig.getFetchTimeout(),
- TimeUnit.MILLISECONDS
- ));
- }
- catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new ISE("Failed to shutdown fetch executor during close");
- }
- }
- );
- }
-
- /**
- * Returns a predicate describing retry conditions. {@link Fetcher} and {@link RetryingInputStream} will retry on the
- * errors satisfying this condition.
- */
- protected abstract Predicate getRetryCondition();
-
- /**
- * Open an input stream from the given object. If the object is compressed, this method should return a byte stream
- * as it is compressed. The object compression should be handled in {@link #wrapObjectStream(Object, InputStream)}.
- *
- * @param object an object to be read
- * @param start start offset
- *
- * @return an input stream for the object
- */
- protected abstract InputStream openObjectStream(T object, long start) throws IOException;
-
- /**
- * This class calls the {@link Closeable#close()} method of the resourceCloser when it is closed.
- */
- static class ResourceCloseableLineIterator extends LineIterator
- {
- private final Closeable resourceCloser;
-
- ResourceCloseableLineIterator(Reader reader, Closeable resourceCloser) throws IllegalArgumentException
- {
- super(reader);
- this.resourceCloser = resourceCloser;
- }
-
- @Override
- public void close()
- {
- try (Closeable ignore = this.resourceCloser) {
- super.close();
- }
- catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
- }
-}
diff --git a/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java b/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java
deleted file mode 100644
index 3def28293272..000000000000
--- a/core/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.data.input;
-
-import com.google.common.collect.Iterables;
-import org.apache.druid.data.input.impl.CSVParseSpec;
-import org.apache.druid.data.input.impl.DimensionsSpec;
-import org.apache.druid.data.input.impl.StringInputRowParser;
-import org.apache.druid.data.input.impl.TimestampSpec;
-import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.parsers.CloseableIterator;
-import org.apache.druid.java.util.common.parsers.ParseException;
-import org.apache.druid.testing.InitializedNullHandlingTest;
-import org.junit.Assert;
-import org.junit.Test;
-
-import javax.annotation.Nullable;
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.stream.Stream;
-
-public class FirehoseFactoryToInputSourceAdaptorTest extends InitializedNullHandlingTest
-{
- @Test
- public void testUnimplementedInputFormat() throws IOException
- {
- final List lines = new ArrayList<>();
- for (int i = 0; i < 10; i++) {
- lines.add(StringUtils.format("%d,name_%d,%d", 20190101 + i, i, i + 100));
- }
- final TestFirehoseFactory firehoseFactory = new TestFirehoseFactory(lines);
- final StringInputRowParser inputRowParser = new StringInputRowParser(
- new CSVParseSpec(
- new TimestampSpec(null, "yyyyMMdd", null),
- new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "name", "score"))),
- ",",
- Arrays.asList("timestamp", "name", "score"),
- false,
- 0
- ),
- StringUtils.UTF8_STRING
- );
- final FirehoseFactoryToInputSourceAdaptor inputSourceAdaptor = new FirehoseFactoryToInputSourceAdaptor(
- firehoseFactory,
- inputRowParser
- );
- final InputSourceReader reader = inputSourceAdaptor.reader(
- new InputRowSchema(
- inputRowParser.getParseSpec().getTimestampSpec(),
- inputRowParser.getParseSpec().getDimensionsSpec(),
- ColumnsFilter.all()
- ),
- null,
- null
- );
- final List result = new ArrayList<>();
- try (CloseableIterator iterator = reader.read()) {
- while (iterator.hasNext()) {
- result.add(iterator.next());
- }
- }
- Assert.assertEquals(10, result.size());
- for (int i = 0; i < 10; i++) {
- Assert.assertEquals(DateTimes.of(StringUtils.format("2019-01-%02d", 1 + i)), result.get(i).getTimestamp());
- Assert.assertEquals(
- StringUtils.format("name_%d", i),
- Iterables.getOnlyElement(result.get(i).getDimension("name"))
- );
- Assert.assertEquals(
- StringUtils.format("%d", i + 100),
- Iterables.getOnlyElement(result.get(i).getDimension("score"))
- );
- }
- }
-
- private static class TestFirehoseFactory implements FiniteFirehoseFactory
- {
- private final List lines;
-
- private TestFirehoseFactory(List lines)
- {
- this.lines = lines;
- }
-
- @Override
- public Firehose connect(StringInputRowParser parser, @Nullable File temporaryDirectory) throws ParseException
- {
- return new Firehose()
- {
- final Iterator iterator = lines.iterator();
-
- @Override
- public boolean hasMore()
- {
- return iterator.hasNext();
- }
-
- @Override
- public InputRow nextRow()
- {
- return parser.parse(iterator.next());
- }
-
- @Override
- public void close()
- {
- // do nothing
- }
- };
- }
-
- @Override
- public boolean isSplittable()
- {
- return false;
- }
-
- @Override
- public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec)
- {
- return null;
- }
-
- @Override
- public int getNumSplits(@Nullable SplitHintSpec splitHintSpec)
- {
- return 0;
- }
-
- @Override
- public FiniteFirehoseFactory withSplit(InputSplit split)
- {
- return null;
- }
- }
-}
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java b/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java
deleted file mode 100644
index 2669a968d045..000000000000
--- a/core/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.data.input.impl;
-
-import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.InputSplit;
-import org.apache.druid.data.input.SplitHintSpec;
-
-import javax.annotation.Nullable;
-import java.util.stream.Stream;
-
-public class NoopFirehoseFactory implements FiniteFirehoseFactory
-{
- @Override
- public String toString()
- {
- return "NoopFirehoseFactory{}";
- }
-
- @Override
- public Stream getSplits(@Nullable SplitHintSpec splitHintSpec)
- {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public int getNumSplits(@Nullable SplitHintSpec splitHintSpec)
- {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public FiniteFirehoseFactory withSplit(InputSplit split)
- {
- throw new UnsupportedOperationException();
- }
-}
diff --git a/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java b/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java
deleted file mode 100644
index dd9c384e91a5..000000000000
--- a/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java
+++ /dev/null
@@ -1,640 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.data.input.impl.prefetch;
-
-import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
-import com.google.common.io.CountingOutputStream;
-import io.netty.util.SuppressForbidden;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.filefilter.TrueFileFilter;
-import org.apache.druid.common.config.NullHandling;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.Firehose;
-import org.apache.druid.data.input.InputSplit;
-import org.apache.druid.data.input.Row;
-import org.apache.druid.data.input.impl.CSVParseSpec;
-import org.apache.druid.data.input.impl.DimensionsSpec;
-import org.apache.druid.data.input.impl.StringInputRowParser;
-import org.apache.druid.data.input.impl.TimestampSpec;
-import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.RetryUtils;
-import org.apache.druid.java.util.common.StringUtils;
-import org.hamcrest.CoreMatchers;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
-import java.net.SocketException;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeoutException;
-
-public class PrefetchableTextFilesFirehoseFactoryTest
-{
- private static long FILE_SIZE = -1;
-
- private static final StringInputRowParser PARSER = new StringInputRowParser(
- new CSVParseSpec(
- new TimestampSpec(
- "timestamp",
- "auto",
- null
- ),
- new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b"))),
- ",",
- Arrays.asList("timestamp", "a", "b"),
- false,
- 0
- ),
- StandardCharsets.UTF_8.name()
- );
-
- @ClassRule
- public static TemporaryFolder tempDir = new TemporaryFolder();
- private static File TEST_DIR;
-
- @Rule
- public ExpectedException expectedException = ExpectedException.none();
-
- @BeforeClass
- public static void setup() throws IOException
- {
- NullHandling.initializeForTests();
- TEST_DIR = tempDir.newFolder();
- for (int i = 0; i < 100; i++) {
- try (
- CountingOutputStream cos = new CountingOutputStream(
- Files.newOutputStream(new File(TEST_DIR, "test_" + i).toPath())
- );
- Writer writer = new BufferedWriter(new OutputStreamWriter(cos, StandardCharsets.UTF_8))
- ) {
- for (int j = 0; j < 100; j++) {
- final String a = StringUtils.format("%d,%03d,%03d\n", (20171220 + i), i, j);
- writer.write(a);
- }
- writer.flush();
- // Every file size must be same
- if (FILE_SIZE == -1) {
- FILE_SIZE = cos.getCount();
- } else {
- Assert.assertEquals(FILE_SIZE, cos.getCount());
- }
- }
- }
- }
-
- private static void assertResult(List rows)
- {
- Assert.assertEquals(10000, rows.size());
- rows.sort((r1, r2) -> {
- int c = r1.getTimestamp().compareTo(r2.getTimestamp());
- if (c != 0) {
- return c;
- }
- c = Integer.valueOf(r1.getDimension("a").get(0)).compareTo(Integer.valueOf(r2.getDimension("a").get(0)));
- if (c != 0) {
- return c;
- }
-
- return Integer.valueOf(r1.getDimension("b").get(0)).compareTo(Integer.valueOf(r2.getDimension("b").get(0)));
- });
-
- for (int i = 0; i < 100; i++) {
- for (int j = 0; j < 100; j++) {
- final Row row = rows.get(i * 100 + j);
- Assert.assertEquals(DateTimes.utc(20171220 + i), row.getTimestamp());
- Assert.assertEquals(i, Integer.valueOf(row.getDimension("a").get(0)).intValue());
- Assert.assertEquals(j, Integer.valueOf(row.getDimension("b").get(0)).intValue());
- }
- }
- }
-
- private static void assertNumRemainingCacheFiles(File firehoseTmpDir, int expectedNumFiles)
- {
- final String[] files = firehoseTmpDir.list();
- Assert.assertNotNull(files);
- Assert.assertEquals(expectedNumFiles, files.length);
- }
-
- @SuppressForbidden(reason = "Files#createTempDirectory")
- private static File createFirehoseTmpDir(String dirPrefix) throws IOException
- {
- return Files.createTempDirectory(tempDir.getRoot().toPath(), dirPrefix).toFile();
- }
-
- @Test
- public void testWithoutCacheAndFetch() throws IOException
- {
- final TestPrefetchableTextFilesFirehoseFactory factory =
- TestPrefetchableTextFilesFirehoseFactory.with(TEST_DIR, 0, 0);
-
- final List rows = new ArrayList<>();
- final File firehoseTmpDir = createFirehoseTmpDir("testWithoutCacheAndFetch");
- try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) {
- while (firehose.hasMore()) {
- rows.add(firehose.nextRow());
- }
- }
-
- Assert.assertEquals(0, factory.getCacheManager().getTotalCachedBytes());
- assertResult(rows);
- assertNumRemainingCacheFiles(firehoseTmpDir, 0);
- }
-
- @Test
- public void testWithoutCacheAndFetchAgainstConnectionReset() throws IOException
- {
- final TestPrefetchableTextFilesFirehoseFactory factory =
- TestPrefetchableTextFilesFirehoseFactory.withConnectionResets(TEST_DIR, 0, 0, 2);
-
- final List rows = new ArrayList<>();
- final File firehoseTmpDir = createFirehoseTmpDir("testWithoutCacheAndFetch");
- try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) {
- while (firehose.hasMore()) {
- rows.add(firehose.nextRow());
- }
- }
-
- Assert.assertEquals(0, factory.getCacheManager().getTotalCachedBytes());
- assertResult(rows);
- assertNumRemainingCacheFiles(firehoseTmpDir, 0);
- }
-
- @Test
- public void testWithoutCache() throws IOException
- {
- final TestPrefetchableTextFilesFirehoseFactory factory =
- TestPrefetchableTextFilesFirehoseFactory.with(TEST_DIR, 0, 2048);
-
- final List rows = new ArrayList<>();
- final File firehoseTmpDir = createFirehoseTmpDir("testWithoutCache");
- try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) {
- while (firehose.hasMore()) {
- rows.add(firehose.nextRow());
- }
- }
-
- Assert.assertEquals(0, factory.getCacheManager().getTotalCachedBytes());
- assertResult(rows);
- assertNumRemainingCacheFiles(firehoseTmpDir, 0);
- }
-
- @Test
- public void testWithZeroFetchCapacity() throws IOException
- {
- final TestPrefetchableTextFilesFirehoseFactory factory =
- TestPrefetchableTextFilesFirehoseFactory.with(TEST_DIR, 2048, 0);
-
- final List rows = new ArrayList<>();
- final File firehoseTmpDir = createFirehoseTmpDir("testWithZeroFetchCapacity");
- try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) {
- while (firehose.hasMore()) {
- rows.add(firehose.nextRow());
- }
- }
-
- assertResult(rows);
- assertNumRemainingCacheFiles(firehoseTmpDir, 2);
- }
-
- @Test
- public void testWithCacheAndFetch() throws IOException
- {
- final TestPrefetchableTextFilesFirehoseFactory factory =
- TestPrefetchableTextFilesFirehoseFactory.of(TEST_DIR);
-
- final List rows = new ArrayList<>();
- final File firehoseTmpDir = createFirehoseTmpDir("testWithCacheAndFetch");
- try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) {
- while (firehose.hasMore()) {
- rows.add(firehose.nextRow());
- }
- }
-
- assertResult(rows);
- assertNumRemainingCacheFiles(firehoseTmpDir, 2);
- }
-
- @Test
- public void testWithLargeCacheAndSmallFetch() throws IOException
- {
- final TestPrefetchableTextFilesFirehoseFactory factory =
- TestPrefetchableTextFilesFirehoseFactory.with(TEST_DIR, 2048, 1024);
-
- final List rows = new ArrayList<>();
- final File firehoseTmpDir = createFirehoseTmpDir("testWithLargeCacheAndSmallFetch");
- try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) {
- while (firehose.hasMore()) {
- rows.add(firehose.nextRow());
- }
- }
-
- assertResult(rows);
- assertNumRemainingCacheFiles(firehoseTmpDir, 2);
- }
-
- @Test
- public void testWithSmallCacheAndLargeFetch() throws IOException
- {
- final TestPrefetchableTextFilesFirehoseFactory factory =
- TestPrefetchableTextFilesFirehoseFactory.with(TEST_DIR, 1024, 2048);
-
- final List rows = new ArrayList<>();
- final File firehoseTmpDir = createFirehoseTmpDir("testWithSmallCacheAndLargeFetch");
- try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) {
- while (firehose.hasMore()) {
- rows.add(firehose.nextRow());
- }
- }
-
- assertResult(rows);
- assertNumRemainingCacheFiles(firehoseTmpDir, 1);
- }
-
- @Test
- public void testRetry() throws IOException
- {
- final TestPrefetchableTextFilesFirehoseFactory factory =
- TestPrefetchableTextFilesFirehoseFactory.withOpenExceptions(TEST_DIR, 1);
-
- final List rows = new ArrayList<>();
- final File firehoseTmpDir = createFirehoseTmpDir("testRetry");
- try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) {
- while (firehose.hasMore()) {
- rows.add(firehose.nextRow());
- }
- }
-
- assertResult(rows);
- assertNumRemainingCacheFiles(firehoseTmpDir, 2);
- }
-
- @Test
- public void testMaxRetry() throws IOException
- {
- expectedException.expect(RuntimeException.class);
- expectedException.expectCause(CoreMatchers.instanceOf(ExecutionException.class));
- expectedException.expectMessage("Exception for retry test");
-
- final TestPrefetchableTextFilesFirehoseFactory factory =
- TestPrefetchableTextFilesFirehoseFactory.withOpenExceptions(TEST_DIR, 5);
-
- try (Firehose firehose = factory.connect(PARSER, createFirehoseTmpDir("testMaxRetry"))) {
- while (firehose.hasMore()) {
- firehose.nextRow();
- }
- }
- }
-
- @Test
- public void testTimeout() throws IOException
- {
- expectedException.expect(RuntimeException.class);
- expectedException.expectCause(CoreMatchers.instanceOf(TimeoutException.class));
-
- final TestPrefetchableTextFilesFirehoseFactory factory =
- TestPrefetchableTextFilesFirehoseFactory.withSleepMillis(TEST_DIR, 1000);
-
- try (Firehose firehose = factory.connect(PARSER, createFirehoseTmpDir("testTimeout"))) {
- while (firehose.hasMore()) {
- firehose.nextRow();
- }
- }
- }
-
- @Test
- public void testReconnectWithCacheAndPrefetch() throws IOException
- {
- final TestPrefetchableTextFilesFirehoseFactory factory =
- TestPrefetchableTextFilesFirehoseFactory.of(TEST_DIR);
- final File firehoseTmpDir = createFirehoseTmpDir("testReconnectWithCacheAndPrefetch");
-
- for (int i = 0; i < 5; i++) {
- final List rows = new ArrayList<>();
- try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) {
- if (i > 0) {
- Assert.assertEquals(FILE_SIZE * 2, factory.getCacheManager().getTotalCachedBytes());
- }
-
- while (firehose.hasMore()) {
- rows.add(firehose.nextRow());
- }
- }
- assertResult(rows);
- assertNumRemainingCacheFiles(firehoseTmpDir, 2);
- }
- }
-
- @Test
- public void testReconnectWithCache() throws IOException
- {
- final TestPrefetchableTextFilesFirehoseFactory factory =
- TestPrefetchableTextFilesFirehoseFactory.with(TEST_DIR, 2048, 0);
- final File firehoseTmpDir = createFirehoseTmpDir("testReconnectWithCache");
-
- for (int i = 0; i < 5; i++) {
- final List rows = new ArrayList<>();
- try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) {
- if (i > 0) {
- Assert.assertEquals(FILE_SIZE * 2, factory.getCacheManager().getTotalCachedBytes());
- }
-
- while (firehose.hasMore()) {
- rows.add(firehose.nextRow());
- }
- }
- assertResult(rows);
- assertNumRemainingCacheFiles(firehoseTmpDir, 2);
- }
- }
-
- static class TestPrefetchableTextFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory
- {
- private final long sleepMillis;
- private final File baseDir;
- private int numOpenExceptions;
- private int maxConnectionResets;
-
- static TestPrefetchableTextFilesFirehoseFactory with(File baseDir, long cacheCapacity, long fetchCapacity)
- {
- return new TestPrefetchableTextFilesFirehoseFactory(
- baseDir,
- 1024,
- cacheCapacity,
- fetchCapacity,
- 60_000, // fetch timeout
- 3,
- 0,
- 0,
- 0
- );
- }
-
- static TestPrefetchableTextFilesFirehoseFactory of(File baseDir)
- {
- return new TestPrefetchableTextFilesFirehoseFactory(
- baseDir,
- 1024,
- 2048,
- 2048,
- 3,
- 0,
- 0,
- 0
- );
- }
-
- static TestPrefetchableTextFilesFirehoseFactory withOpenExceptions(File baseDir, int count)
- {
- return new TestPrefetchableTextFilesFirehoseFactory(
- baseDir,
- 1024,
- 2048,
- 2048,
- 3,
- count,
- 0,
- 0
- );
- }
-
- static TestPrefetchableTextFilesFirehoseFactory withConnectionResets(
- File baseDir,
- long cacheCapacity,
- long fetchCapacity,
- int numConnectionResets
- )
- {
- return new TestPrefetchableTextFilesFirehoseFactory(
- baseDir,
- fetchCapacity / 2,
- cacheCapacity,
- fetchCapacity,
- 3,
- 0,
- numConnectionResets,
- 0
- );
- }
-
- static TestPrefetchableTextFilesFirehoseFactory withSleepMillis(File baseDir, long ms)
- {
- return new TestPrefetchableTextFilesFirehoseFactory(
- baseDir,
- 1024,
- 2048,
- 2048,
- 100,
- 3,
- 0,
- 0,
- ms
- );
- }
-
- private static long computeTimeout(int maxRetry)
- {
- // See RetryUtils.nextRetrySleepMillis()
- final double maxFuzzyMultiplier = 2.;
- return (long) Math.min(
- RetryUtils.MAX_SLEEP_MILLIS,
- RetryUtils.BASE_SLEEP_MILLIS * Math.pow(2, maxRetry - 1) * maxFuzzyMultiplier
- );
- }
-
- TestPrefetchableTextFilesFirehoseFactory(
- File baseDir,
- long prefetchTriggerThreshold,
- long maxCacheCapacityBytes,
- long maxFetchCapacityBytes,
- int maxRetry,
- int numOpenExceptions,
- int numConnectionResets,
- long sleepMillis
- )
- {
- this(
- baseDir,
- prefetchTriggerThreshold,
- maxCacheCapacityBytes,
- maxFetchCapacityBytes,
- computeTimeout(maxRetry),
- maxRetry,
- numOpenExceptions,
- numConnectionResets,
- sleepMillis
- );
- }
-
- TestPrefetchableTextFilesFirehoseFactory(
- File baseDir,
- long prefetchTriggerThreshold,
- long maxCacheCapacityBytes,
- long maxFetchCapacityBytes,
- long fetchTimeout,
- int maxRetry,
- int numOpenExceptions,
- int maxConnectionResets,
- long sleepMillis
- )
- {
- super(
- maxCacheCapacityBytes,
- maxFetchCapacityBytes,
- prefetchTriggerThreshold,
- fetchTimeout,
- maxRetry
- );
- this.numOpenExceptions = numOpenExceptions;
- this.maxConnectionResets = maxConnectionResets;
- this.sleepMillis = sleepMillis;
- this.baseDir = baseDir;
- }
-
- @Override
- protected Collection initObjects()
- {
- return FileUtils.listFiles(
- Preconditions.checkNotNull(baseDir).getAbsoluteFile(),
- TrueFileFilter.INSTANCE,
- TrueFileFilter.INSTANCE
- );
- }
-
- @Override
- protected InputStream openObjectStream(File object) throws IOException
- {
- if (numOpenExceptions > 0) {
- numOpenExceptions--;
- throw new IOException("Exception for retry test");
- }
- if (sleepMillis > 0) {
- try {
- Thread.sleep(sleepMillis);
- }
- catch (InterruptedException e) {
- throw new RuntimeException(e);
- }
- }
- return maxConnectionResets > 0 ?
- new TestInputStream(FileUtils.openInputStream(object), maxConnectionResets) :
- FileUtils.openInputStream(object);
- }
-
- @Override
- protected InputStream wrapObjectStream(File object, InputStream stream)
- {
- return stream;
- }
-
- @Override
- protected Predicate getRetryCondition()
- {
- return e -> e instanceof IOException;
- }
-
- @Override
- protected InputStream openObjectStream(File object, long start) throws IOException
- {
- if (numOpenExceptions > 0) {
- numOpenExceptions--;
- throw new IOException("Exception for retry test");
- }
- if (sleepMillis > 0) {
- try {
- Thread.sleep(sleepMillis);
- }
- catch (InterruptedException e) {
- throw new RuntimeException(e);
- }
- }
-
- final InputStream in = FileUtils.openInputStream(object);
- in.skip(start);
-
- return maxConnectionResets > 0 ? new TestInputStream(in, maxConnectionResets) : in;
- }
-
- private int readCount;
- private int numConnectionResets;
-
- @Override
- public FiniteFirehoseFactory withSplit(InputSplit split)
- {
- throw new UnsupportedOperationException();
- }
-
- private class TestInputStream extends InputStream
- {
- private static final int NUM_READ_COUNTS_BEFORE_ERROR = 10;
- private final InputStream delegate;
- private final int maxConnectionResets;
-
- TestInputStream(
- InputStream delegate,
- int maxConnectionResets
- )
- {
- this.delegate = delegate;
- this.maxConnectionResets = maxConnectionResets;
- }
-
- @Override
- public int read() throws IOException
- {
- if (readCount++ % NUM_READ_COUNTS_BEFORE_ERROR == 0) {
- if (numConnectionResets++ < maxConnectionResets) {
- // Simulate connection reset
- throw new SocketException("Test Connection reset");
- }
- }
- return delegate.read();
- }
-
- @Override
- public int read(byte[] b, int off, int len) throws IOException
- {
- if (readCount++ % NUM_READ_COUNTS_BEFORE_ERROR == 0) {
- if (numConnectionResets++ < maxConnectionResets) {
- // Simulate connection reset
- throw new SocketException("Test Connection reset");
- }
- }
- return delegate.read(b, off, len);
- }
- }
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/OssFirehoseDruidModule.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/OssFirehoseDruidModule.java
deleted file mode 100644
index 864717657fe8..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/OssFirehoseDruidModule.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.aliyun;
-
-import com.fasterxml.jackson.databind.Module;
-import com.fasterxml.jackson.databind.jsontype.NamedType;
-import com.fasterxml.jackson.databind.module.SimpleModule;
-import com.google.common.collect.ImmutableList;
-import com.google.inject.Binder;
-import org.apache.druid.initialization.DruidModule;
-
-import java.util.List;
-
-public class OssFirehoseDruidModule implements DruidModule
-{
- @Override
- public List extends Module> getJacksonModules()
- {
- return ImmutableList.of(
- new SimpleModule().registerSubtypes(new NamedType(StaticOssFirehoseFactory.class, "static-aliyun-oss"))
- );
- }
-
- @Override
- public void configure(Binder binder)
- {
-
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/StaticOssFirehoseFactory.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/StaticOssFirehoseFactory.java
deleted file mode 100644
index d71198826dc7..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/StaticOssFirehoseFactory.java
+++ /dev/null
@@ -1,243 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.OSSException;
-import com.aliyun.oss.model.GetObjectRequest;
-import com.aliyun.oss.model.OSSObject;
-import com.aliyun.oss.model.OSSObjectSummary;
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.InputSplit;
-import org.apache.druid.data.input.impl.StringInputRowParser;
-import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
-import org.apache.druid.java.util.common.IAE;
-import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.storage.aliyun.OssStorageDruidModule;
-import org.apache.druid.storage.aliyun.OssUtils;
-import org.apache.druid.utils.CompressionUtils;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
-/**
- * Builds firehoses that read from a predefined list of aliyun OSS objects and then dry up.
- */
-public class StaticOssFirehoseFactory extends PrefetchableTextFilesFirehoseFactory
-{
- private static final Logger log = new Logger(StaticOssFirehoseFactory.class);
-
- private final OSS client;
- private final List uris;
- private final List prefixes;
-
- @JsonCreator
- public StaticOssFirehoseFactory(
- @JacksonInject OSS client,
- @JsonProperty("uris") List uris,
- @JsonProperty("prefixes") List prefixes,
- @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
- @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
- @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
- @JsonProperty("fetchTimeout") Long fetchTimeout,
- @JsonProperty("maxFetchRetry") Integer maxFetchRetry
- )
- {
- super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
- this.client = Preconditions.checkNotNull(client, "client");
- this.uris = uris == null ? new ArrayList<>() : uris;
- this.prefixes = prefixes == null ? new ArrayList<>() : prefixes;
-
- if (!this.uris.isEmpty() && !this.prefixes.isEmpty()) {
- throw new IAE("uris and prefixes cannot be used together");
- }
-
- if (this.uris.isEmpty() && this.prefixes.isEmpty()) {
- throw new IAE("uris or prefixes must be specified");
- }
-
- for (final URI inputURI : this.uris) {
- Preconditions.checkArgument(OssStorageDruidModule.SCHEME.equals(inputURI.getScheme()),
- "input uri scheme == %s (%s)",
- OssStorageDruidModule.SCHEME,
- inputURI);
- }
-
- for (final URI inputURI : this.prefixes) {
- Preconditions.checkArgument(OssStorageDruidModule.SCHEME.equals(inputURI.getScheme()),
- "input uri scheme == %s (%s)",
- OssStorageDruidModule.SCHEME,
- inputURI);
- }
- }
-
- @JsonProperty
- public List getUris()
- {
- return uris;
- }
-
- @JsonProperty("prefixes")
- public List getPrefixes()
- {
- return prefixes;
- }
-
- @Override
- protected Collection initObjects()
- {
- if (!uris.isEmpty()) {
- return uris;
- } else {
- final List objects = new ArrayList<>();
- for (final URI prefix : prefixes) {
- final Iterator objectSummaryIterator = OssUtils.objectSummaryIterator(
- client,
- Collections.singletonList(prefix),
- OssUtils.MAX_LISTING_LENGTH
- );
-
- objectSummaryIterator.forEachRemaining(objects::add);
- }
- return objects.stream().map(OssUtils::summaryToUri).collect(Collectors.toList());
- }
- }
-
- @Override
- protected InputStream openObjectStream(URI object) throws IOException
- {
- try {
- // Get data of the given object and open an input stream
- final String bucket = object.getAuthority();
- final String key = OssUtils.extractKey(object);
-
- final OSSObject ossObject = client.getObject(bucket, key);
- if (ossObject == null) {
- throw new ISE("Failed to get an Aliyun OSS object for bucket[%s] and key[%s]", bucket, key);
- }
- return ossObject.getObjectContent();
- }
- catch (OSSException e) {
- throw new IOException(e);
- }
- }
-
- @Override
- protected InputStream openObjectStream(URI object, long start) throws IOException
- {
- final String bucket = object.getAuthority();
- final String key = OssUtils.extractKey(object);
-
- final GetObjectRequest request = new GetObjectRequest(bucket, key);
- try {
- final OSSObject ossObject = client.getObject(request);
- if (ossObject == null) {
- throw new ISE(
- "Failed to get an Aliyun OSS object for bucket[%s], key[%s], and start[%d]",
- bucket,
- key,
- start
- );
- }
- InputStream is = ossObject.getObjectContent();
- is.skip(start);
- return is;
- }
- catch (OSSException e) {
- throw new IOException(e);
- }
- }
-
- @Override
- protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException
- {
- return CompressionUtils.decompress(stream, OssUtils.extractKey(object));
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- StaticOssFirehoseFactory that = (StaticOssFirehoseFactory) o;
-
- return Objects.equals(uris, that.uris) &&
- Objects.equals(prefixes, that.prefixes) &&
- getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() &&
- getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() &&
- getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() &&
- getFetchTimeout() == that.getFetchTimeout() &&
- getMaxFetchRetry() == that.getMaxFetchRetry();
- }
-
- @Override
- public int hashCode()
- {
- return Objects.hash(
- uris,
- prefixes,
- getMaxCacheCapacityBytes(),
- getMaxFetchCapacityBytes(),
- getPrefetchTriggerBytes(),
- getFetchTimeout(),
- getMaxFetchRetry()
- );
- }
-
- @Override
- protected Predicate getRetryCondition()
- {
- return OssUtils.RETRYABLE;
- }
-
- @Override
- public FiniteFirehoseFactory withSplit(InputSplit split)
- {
- return new StaticOssFirehoseFactory(
- client,
- Collections.singletonList(split.get()),
- null,
- getMaxCacheCapacityBytes(),
- getMaxFetchCapacityBytes(),
- getPrefetchTriggerBytes(),
- getFetchTimeout(),
- getMaxFetchRetry()
- );
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/aliyun-oss-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
index 3d434e7c9021..8613c8380131 100644
--- a/extensions-contrib/aliyun-oss-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
+++ b/extensions-contrib/aliyun-oss-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
@@ -14,5 +14,4 @@
# limitations under the License.
org.apache.druid.storage.aliyun.OssStorageDruidModule
-org.apache.druid.firehose.aliyun.OssFirehoseDruidModule
org.apache.druid.data.input.aliyun.OssInputSourceDruidModule
\ No newline at end of file
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesBlob.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesBlob.java
deleted file mode 100644
index edc0bbb65c06..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesBlob.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.cloudfiles;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import javax.validation.constraints.NotNull;
-import java.util.Objects;
-
-public class CloudFilesBlob
-{
- @JsonProperty
- @NotNull
- private String container;
-
- @JsonProperty
- @NotNull
- private String path;
-
- @JsonProperty
- @NotNull
- private String region;
-
- @JsonCreator
- public CloudFilesBlob(
- @JsonProperty("container") String container,
- @JsonProperty("path") String path,
- @JsonProperty("region") String region
- )
- {
- this.container = container;
- this.path = path;
- this.region = region;
- }
-
- public String getContainer()
- {
- return container;
- }
-
- public String getPath()
- {
- return path;
- }
-
- public String getRegion()
- {
- return region;
- }
-
- @Override
- public String toString()
- {
- return "CloudFilesBlob{"
- + "container=" + container
- + ",path=" + path
- + ",region=" + region
- + "}";
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
-
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- final CloudFilesBlob that = (CloudFilesBlob) o;
- return Objects.equals(container, that.container) &&
- Objects.equals(path, that.path) &&
- Objects.equals(region, that.region);
- }
-
- @Override
- public int hashCode()
- {
- return Objects.hash(container, path, region);
- }
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesFirehoseDruidModule.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesFirehoseDruidModule.java
deleted file mode 100644
index 3531e7584e45..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesFirehoseDruidModule.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.cloudfiles;
-
-import com.fasterxml.jackson.databind.Module;
-import com.fasterxml.jackson.databind.jsontype.NamedType;
-import com.fasterxml.jackson.databind.module.SimpleModule;
-import com.google.common.collect.ImmutableList;
-import com.google.inject.Binder;
-import org.apache.druid.initialization.DruidModule;
-
-import java.util.List;
-
-public class CloudFilesFirehoseDruidModule implements DruidModule
-{
-
- @Override
- public List extends Module> getJacksonModules()
- {
- return ImmutableList.of(
- new SimpleModule().registerSubtypes(
- new NamedType(StaticCloudFilesFirehoseFactory.class, "staticcloudfiles")));
- }
-
- @Override
- public void configure(Binder arg0)
- {
-
- }
-
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java
deleted file mode 100644
index f0de9f7e98de..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.cloudfiles;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Predicate;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.InputSplit;
-import org.apache.druid.data.input.impl.StringInputRowParser;
-import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.storage.cloudfiles.CloudFilesByteSource;
-import org.apache.druid.storage.cloudfiles.CloudFilesObjectApiProxy;
-import org.apache.druid.storage.cloudfiles.CloudFilesUtils;
-import org.apache.druid.utils.CompressionUtils;
-import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
-
-public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory
-{
- private static final Logger log = new Logger(StaticCloudFilesFirehoseFactory.class);
-
- private final CloudFilesApi cloudFilesApi;
- private final List blobs;
-
- @JsonCreator
- public StaticCloudFilesFirehoseFactory(
- @JacksonInject CloudFilesApi cloudFilesApi,
- @JsonProperty("blobs") List blobs,
- @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
- @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
- @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
- @JsonProperty("fetchTimeout") Long fetchTimeout,
- @JsonProperty("maxFetchRetry") Integer maxFetchRetry
- )
- {
- super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
- this.cloudFilesApi = cloudFilesApi;
- this.blobs = blobs;
- }
-
- @JsonProperty
- public List getBlobs()
- {
- return blobs;
- }
-
- @Override
- protected Collection initObjects()
- {
- return blobs;
- }
-
- @Override
- protected InputStream openObjectStream(CloudFilesBlob object) throws IOException
- {
- return openObjectStream(object, 0);
- }
-
- @Override
- protected InputStream openObjectStream(CloudFilesBlob object, long start) throws IOException
- {
- return createCloudFilesByteSource(object).openStream(start);
- }
-
- private CloudFilesByteSource createCloudFilesByteSource(CloudFilesBlob object)
- {
- final String region = object.getRegion();
- final String container = object.getContainer();
- final String path = object.getPath();
-
- log.info("Retrieving file from region[%s], container[%s] and path [%s]",
- region, container, path
- );
- CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy(cloudFilesApi, region, container);
- return new CloudFilesByteSource(objectApi, path);
- }
-
- @Override
- protected InputStream wrapObjectStream(CloudFilesBlob object, InputStream stream) throws IOException
- {
- return CompressionUtils.decompress(stream, object.getPath());
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (o == this) {
- return true;
- }
-
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- final StaticCloudFilesFirehoseFactory that = (StaticCloudFilesFirehoseFactory) o;
- return Objects.equals(blobs, that.blobs) &&
- getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() &&
- getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() &&
- getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() &&
- getFetchTimeout() == that.getFetchTimeout() &&
- getMaxFetchRetry() == that.getMaxFetchRetry();
- }
-
- @Override
- public int hashCode()
- {
- return Objects.hash(
- blobs,
- getMaxCacheCapacityBytes(),
- getMaxFetchCapacityBytes(),
- getPrefetchTriggerBytes(),
- getFetchTimeout(),
- getMaxFetchRetry()
- );
- }
-
- @Override
- protected Predicate getRetryCondition()
- {
- return CloudFilesUtils.CLOUDFILESRETRY;
- }
-
- @Override
- public FiniteFirehoseFactory withSplit(InputSplit split)
- {
- return new StaticCloudFilesFirehoseFactory(
- cloudFilesApi,
- Collections.singletonList(split.get()),
- getMaxCacheCapacityBytes(),
- getMaxFetchCapacityBytes(),
- getPrefetchTriggerBytes(),
- getFetchTimeout(),
- getMaxFetchRetry()
- );
- }
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/cloudfiles-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
index 81a1411f89b6..367f66497ba9 100644
--- a/extensions-contrib/cloudfiles-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
+++ b/extensions-contrib/cloudfiles-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
@@ -16,4 +16,3 @@
# under the License.
org.apache.druid.storage.cloudfiles.CloudFilesStorageDruidModule
-org.apache.druid.firehose.cloudfiles.CloudFilesFirehoseDruidModule
diff --git a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactoryTest.java b/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactoryTest.java
deleted file mode 100644
index 48f3ca694fa3..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactoryTest.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.cloudfiles;
-
-import com.fasterxml.jackson.databind.Module;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.module.SimpleModule;
-import com.fasterxml.jackson.module.guice.ObjectMapperModule;
-import com.google.common.collect.ImmutableList;
-import com.google.inject.Binder;
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.Provides;
-import org.apache.druid.initialization.DruidModule;
-import org.apache.druid.jackson.DefaultObjectMapper;
-import org.easymock.EasyMock;
-import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.List;
-
-public class StaticCloudFilesFirehoseFactoryTest
-{
- private static final CloudFilesApi API = EasyMock.niceMock(CloudFilesApi.class);
-
- @Test
- public void testSerde() throws IOException
- {
- final ObjectMapper mapper = createObjectMapper(new TestModule());
-
- final List blobs = ImmutableList.of(
- new CloudFilesBlob("container", "foo", "bar"),
- new CloudFilesBlob("container", "foo", "bar2")
- );
-
- final StaticCloudFilesFirehoseFactory factory = new StaticCloudFilesFirehoseFactory(
- API,
- blobs,
- 2048L,
- 1024L,
- 512L,
- 100L,
- 5
- );
-
- final StaticCloudFilesFirehoseFactory outputFact = mapper.readValue(
- mapper.writeValueAsString(factory),
- StaticCloudFilesFirehoseFactory.class
- );
-
- Assert.assertEquals(factory, outputFact);
- }
-
- private static ObjectMapper createObjectMapper(DruidModule baseModule)
- {
- final ObjectMapper baseMapper = new DefaultObjectMapper();
- baseModule.getJacksonModules().forEach(baseMapper::registerModule);
-
- final Injector injector = Guice.createInjector(
- new ObjectMapperModule(),
- baseModule
- );
- return injector.getInstance(ObjectMapper.class);
- }
-
- private static class TestModule implements DruidModule
- {
- @Override
- public List extends Module> getJacksonModules()
- {
- return ImmutableList.of(new SimpleModule());
- }
-
- @Override
- public void configure(Binder binder)
- {
-
- }
-
- @Provides
- public CloudFilesApi getRestS3Service()
- {
- return API;
- }
- }
-}
diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/AzureBlob.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/AzureBlob.java
deleted file mode 100644
index ef8278eff913..000000000000
--- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/AzureBlob.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.azure;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import javax.validation.constraints.NotNull;
-import java.util.Objects;
-
-/**
- * Represents an Azure based blob. Used with {@link StaticAzureBlobStoreFirehoseFactory}.
- *
- * @deprecated as of version 0.18.0 because support for firehose has been discontinued. Please use
- * {@link org.apache.druid.data.input.azure.AzureEntity} with {@link org.apache.druid.data.input.azure.AzureInputSource}
- * instead.
- */
-@Deprecated
-public class AzureBlob
-{
- @JsonProperty
- @NotNull
- private String container;
-
- @JsonProperty
- @NotNull
- private String path;
-
- @JsonCreator
- public AzureBlob(@JsonProperty("container") String container, @JsonProperty("path") String path)
- {
- this.container = container;
- this.path = path;
- }
-
- public String getContainer()
- {
- return container;
- }
-
- public String getPath()
- {
- return path;
- }
-
- @Override
- public String toString()
- {
- return "AzureBlob{"
- + "container=" + container
- + ",path=" + path
- + "}";
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (o == this) {
- return true;
- }
-
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- final AzureBlob that = (AzureBlob) o;
- return Objects.equals(container, that.container) &&
- Objects.equals(path, that.path);
- }
-
- @Override
- public int hashCode()
- {
- return Objects.hash(container, path);
- }
-}
diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java
deleted file mode 100644
index 449873523f2d..000000000000
--- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java
+++ /dev/null
@@ -1,167 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.azure;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.InputSplit;
-import org.apache.druid.data.input.impl.StringInputRowParser;
-import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.storage.azure.AzureByteSource;
-import org.apache.druid.storage.azure.AzureStorage;
-import org.apache.druid.storage.azure.AzureUtils;
-import org.apache.druid.utils.CompressionUtils;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
-
-/**
- * This class is heavily inspired by the StaticS3FirehoseFactory class in the org.apache.druid.firehose.s3 package
- *
- * @deprecated as of version 0.18.0 because support for firehose has been discontinued. Please use
- * {@link org.apache.druid.data.input.azure.AzureInputSource} instead.
- */
-@Deprecated
-public class StaticAzureBlobStoreFirehoseFactory extends PrefetchableTextFilesFirehoseFactory
-{
- private final AzureStorage azureStorage;
- private final List blobs;
-
- @JsonCreator
- public StaticAzureBlobStoreFirehoseFactory(
- @JacksonInject AzureStorage azureStorage,
- @JsonProperty("blobs") List blobs,
- @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
- @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
- @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
- @JsonProperty("fetchTimeout") Long fetchTimeout,
- @JsonProperty("maxFetchRetry") Integer maxFetchRetry
- )
- {
- super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
- this.blobs = blobs;
- this.azureStorage = azureStorage;
- }
-
- @JsonProperty
- public List getBlobs()
- {
- return blobs;
- }
-
- @Override
- protected Collection initObjects()
- {
- return blobs;
- }
-
- @Override
- protected InputStream openObjectStream(AzureBlob object) throws IOException
- {
- return makeByteSource(azureStorage, object).openStream();
- }
-
- @Override
- protected InputStream openObjectStream(AzureBlob object, long start) throws IOException
- {
- // BlobInputStream.skip() moves the next read offset instead of skipping first 'start' bytes.
- final InputStream in = openObjectStream(object);
- final long skip = in.skip(start);
- Preconditions.checkState(skip == start, "start offset was [%s] but [%s] bytes were skipped", start, skip);
- return in;
- }
-
- @Override
- protected InputStream wrapObjectStream(AzureBlob object, InputStream stream) throws IOException
- {
- return CompressionUtils.decompress(stream, object.getPath());
- }
-
- private static AzureByteSource makeByteSource(AzureStorage azureStorage, AzureBlob object)
- {
- final String container = object.getContainer();
- final String path = StringUtils.maybeRemoveLeadingSlash(object.getPath());
-
- return new AzureByteSource(azureStorage, container, path);
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
-
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- final StaticAzureBlobStoreFirehoseFactory that = (StaticAzureBlobStoreFirehoseFactory) o;
-
- return Objects.equals(blobs, that.blobs) &&
- getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() &&
- getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() &&
- getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() &&
- getFetchTimeout() == that.getFetchTimeout() &&
- getMaxFetchRetry() == that.getMaxFetchRetry();
- }
-
- @Override
- public int hashCode()
- {
- return Objects.hash(
- blobs,
- getMaxCacheCapacityBytes(),
- getMaxFetchCapacityBytes(),
- getPrefetchTriggerBytes(),
- getFetchTimeout(),
- getMaxFetchRetry()
- );
- }
-
- @Override
- protected Predicate getRetryCondition()
- {
- return AzureUtils.AZURE_RETRY;
- }
-
- @Override
- public FiniteFirehoseFactory withSplit(InputSplit split)
- {
- return new StaticAzureBlobStoreFirehoseFactory(
- azureStorage,
- Collections.singletonList(split.get()),
- getMaxCacheCapacityBytes(),
- getMaxFetchCapacityBytes(),
- getPrefetchTriggerBytes(),
- getFetchTimeout(),
- getMaxFetchRetry()
- );
- }
-}
diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorageDruidModule.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorageDruidModule.java
index e870aa08eca5..c7c4655a3d21 100644
--- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorageDruidModule.java
+++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorageDruidModule.java
@@ -33,7 +33,6 @@
import com.microsoft.azure.storage.blob.CloudBlobClient;
import org.apache.druid.data.input.azure.AzureEntityFactory;
import org.apache.druid.data.input.azure.AzureInputSource;
-import org.apache.druid.firehose.azure.StaticAzureBlobStoreFirehoseFactory;
import org.apache.druid.guice.Binders;
import org.apache.druid.guice.JsonConfigProvider;
import org.apache.druid.guice.LazySingleton;
@@ -84,7 +83,6 @@ public void setupModule(SetupContext context)
}
},
new SimpleModule().registerSubtypes(
- new NamedType(StaticAzureBlobStoreFirehoseFactory.class, "static-azure-blobstore"),
new NamedType(AzureInputSource.class, SCHEME)
)
);
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactoryTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactoryTest.java
deleted file mode 100644
index 52751e211718..000000000000
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactoryTest.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.azure;
-
-import com.fasterxml.jackson.databind.Module;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.module.SimpleModule;
-import com.fasterxml.jackson.module.guice.ObjectMapperModule;
-import com.google.common.collect.ImmutableList;
-import com.google.inject.Binder;
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.Provides;
-import org.apache.druid.initialization.DruidModule;
-import org.apache.druid.jackson.DefaultObjectMapper;
-import org.apache.druid.storage.azure.AzureStorage;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.List;
-
-public class StaticAzureBlobStoreFirehoseFactoryTest
-{
- private static final AzureStorage STORAGE = new AzureStorage(null);
-
- @Test
- public void testSerde() throws IOException
- {
- final ObjectMapper mapper = createObjectMapper(new TestModule());
-
- final List blobs = ImmutableList.of(
- new AzureBlob("foo", "bar"),
- new AzureBlob("foo", "bar2")
- );
-
- final StaticAzureBlobStoreFirehoseFactory factory = new StaticAzureBlobStoreFirehoseFactory(
- STORAGE,
- blobs,
- 2048L,
- 1024L,
- 512L,
- 100L,
- 5
- );
-
- final StaticAzureBlobStoreFirehoseFactory outputFact = mapper.readValue(
- mapper.writeValueAsString(factory),
- StaticAzureBlobStoreFirehoseFactory.class
- );
-
- Assert.assertEquals(factory, outputFact);
- }
-
- private static ObjectMapper createObjectMapper(DruidModule baseModule)
- {
- final ObjectMapper baseMapper = new DefaultObjectMapper();
- baseModule.getJacksonModules().forEach(baseMapper::registerModule);
-
- final Injector injector = Guice.createInjector(
- new ObjectMapperModule(),
- baseModule
- );
- return injector.getInstance(ObjectMapper.class);
- }
-
- private static class TestModule implements DruidModule
- {
- @Override
- public List extends Module> getJacksonModules()
- {
- return ImmutableList.of(new SimpleModule());
- }
-
- @Override
- public void configure(Binder binder)
- {
-
- }
-
- @Provides
- public AzureStorage getRestS3Service()
- {
- return STORAGE;
- }
- }
-}
diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/GoogleBlob.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/GoogleBlob.java
deleted file mode 100644
index 0e51b62080cf..000000000000
--- a/extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/GoogleBlob.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.google;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.util.Objects;
-
-public class GoogleBlob
-{
- private final String bucket;
- private final String path;
-
- @JsonCreator
- public GoogleBlob(@JsonProperty("bucket") String bucket, @JsonProperty("path") String path)
- {
- this.bucket = bucket;
- this.path = path;
- }
-
- @JsonProperty
- public String getBucket()
- {
- return bucket;
- }
-
- @JsonProperty
- public String getPath()
- {
- return path;
- }
-
- @Override
- public String toString()
- {
- return "GoogleBlob {"
- + "bucket=" + bucket
- + ",path=" + path
- + "}";
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
-
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- final GoogleBlob that = (GoogleBlob) o;
- return Objects.equals(bucket, that.bucket) &&
- Objects.equals(path, that.path);
- }
-
- @Override
- public int hashCode()
- {
- return Objects.hash(bucket, path);
- }
-}
diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java
deleted file mode 100644
index 22437d780e13..000000000000
--- a/extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.google;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Predicate;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.InputSplit;
-import org.apache.druid.data.input.impl.StringInputRowParser;
-import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.storage.google.GoogleByteSource;
-import org.apache.druid.storage.google.GoogleStorage;
-import org.apache.druid.storage.google.GoogleUtils;
-import org.apache.druid.utils.CompressionUtils;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
-
-public class StaticGoogleBlobStoreFirehoseFactory extends PrefetchableTextFilesFirehoseFactory
-{
- private final GoogleStorage storage;
- private final List blobs;
-
- @JsonCreator
- public StaticGoogleBlobStoreFirehoseFactory(
- @JacksonInject GoogleStorage storage,
- @JsonProperty("blobs") List blobs,
- @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
- @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
- @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
- @JsonProperty("fetchTimeout") Long fetchTimeout,
- @JsonProperty("maxFetchRetry") Integer maxFetchRetry
- )
- {
- super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
- this.storage = storage;
- this.blobs = blobs;
- }
-
- @JsonProperty
- public List getBlobs()
- {
- return blobs;
- }
-
- @Override
- protected Collection initObjects()
- {
- return blobs;
- }
-
- @Override
- protected InputStream openObjectStream(GoogleBlob object) throws IOException
- {
- return openObjectStream(object, 0);
- }
-
- @Override
- protected InputStream openObjectStream(GoogleBlob object, long start) throws IOException
- {
- return createGoogleByteSource(object).openStream(start);
- }
-
- private GoogleByteSource createGoogleByteSource(GoogleBlob object)
- {
- final String bucket = object.getBucket();
- final String path = StringUtils.maybeRemoveLeadingSlash(object.getPath());
-
- return new GoogleByteSource(storage, bucket, path);
- }
-
- @Override
- protected InputStream wrapObjectStream(GoogleBlob object, InputStream stream) throws IOException
- {
- return CompressionUtils.decompress(stream, object.getPath());
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- final StaticGoogleBlobStoreFirehoseFactory that = (StaticGoogleBlobStoreFirehoseFactory) o;
-
- return Objects.equals(blobs, that.blobs) &&
- getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() &&
- getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() &&
- getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() &&
- getFetchTimeout() == that.getFetchTimeout() &&
- getMaxFetchRetry() == that.getMaxFetchRetry();
- }
-
- @Override
- public int hashCode()
- {
- return Objects.hash(
- blobs,
- getMaxCacheCapacityBytes(),
- getMaxFetchCapacityBytes(),
- getPrefetchTriggerBytes(),
- getFetchTimeout(),
- getMaxFetchRetry()
- );
- }
-
- @Override
- protected Predicate getRetryCondition()
- {
- return GoogleUtils::isRetryable;
- }
-
- @Override
- public FiniteFirehoseFactory withSplit(InputSplit split)
- {
- return new StaticGoogleBlobStoreFirehoseFactory(
- storage,
- Collections.singletonList(split.get()),
- getMaxCacheCapacityBytes(),
- getMaxFetchCapacityBytes(),
- getPrefetchTriggerBytes(),
- getFetchTimeout(),
- getMaxFetchRetry()
- );
- }
-}
-
diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorageDruidModule.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorageDruidModule.java
index 838b46f2287f..8aa77b0298f5 100644
--- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorageDruidModule.java
+++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorageDruidModule.java
@@ -34,7 +34,6 @@
import com.google.inject.multibindings.MapBinder;
import org.apache.druid.data.SearchableVersionedDataFinder;
import org.apache.druid.data.input.google.GoogleCloudStorageInputSource;
-import org.apache.druid.firehose.google.StaticGoogleBlobStoreFirehoseFactory;
import org.apache.druid.guice.Binders;
import org.apache.druid.guice.JsonConfigProvider;
import org.apache.druid.guice.LazySingleton;
@@ -77,7 +76,6 @@ public void setupModule(SetupContext context)
}
},
new SimpleModule().registerSubtypes(
- new NamedType(StaticGoogleBlobStoreFirehoseFactory.class, "static-google-blobstore"),
new NamedType(GoogleCloudStorageInputSource.class, SCHEME)
)
);
diff --git a/extensions-core/google-extensions/src/test/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactoryTest.java b/extensions-core/google-extensions/src/test/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactoryTest.java
deleted file mode 100644
index c9996b53ccbc..000000000000
--- a/extensions-core/google-extensions/src/test/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactoryTest.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.google;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.ImmutableList;
-import org.apache.druid.data.input.google.GoogleCloudStorageInputSourceTest;
-import org.apache.druid.storage.google.GoogleStorage;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.List;
-
-public class StaticGoogleBlobStoreFirehoseFactoryTest
-{
- private static final GoogleStorage STORAGE = new GoogleStorage(null);
-
- @Test
- public void testSerde() throws IOException
- {
- final ObjectMapper mapper = GoogleCloudStorageInputSourceTest.createGoogleObjectMapper();
-
- final List blobs = ImmutableList.of(
- new GoogleBlob("foo", "bar"),
- new GoogleBlob("foo", "bar2")
- );
-
- final StaticGoogleBlobStoreFirehoseFactory factory = new StaticGoogleBlobStoreFirehoseFactory(
- STORAGE,
- blobs,
- 2048L,
- 1024L,
- 512L,
- 100L,
- 5
- );
-
- final StaticGoogleBlobStoreFirehoseFactory outputFact = mapper.readValue(
- mapper.writeValueAsString(factory),
- StaticGoogleBlobStoreFirehoseFactory.class
- );
-
- Assert.assertEquals(factory, outputFact);
- }
-}
diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactory.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactory.java
deleted file mode 100644
index f7fac9f3b479..000000000000
--- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactory.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.hdfs;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Predicate;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.InputSplit;
-import org.apache.druid.data.input.impl.StringInputRowParser;
-import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
-import org.apache.druid.guice.Hdfs;
-import org.apache.druid.inputsource.hdfs.HdfsInputSource;
-import org.apache.druid.inputsource.hdfs.HdfsInputSourceConfig;
-import org.apache.druid.storage.hdfs.HdfsDataSegmentPuller;
-import org.apache.druid.utils.CompressionUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.Path;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Collection;
-import java.util.List;
-
-public class HdfsFirehoseFactory extends PrefetchableTextFilesFirehoseFactory
-{
- private final List inputPaths;
- private final Configuration conf;
- private final HdfsInputSourceConfig inputSourceConfig;
-
- @JsonCreator
- public HdfsFirehoseFactory(
- @JsonProperty("paths") Object inputPaths,
- @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
- @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
- @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
- @JsonProperty("fetchTimeout") Long fetchTimeout,
- @JsonProperty("maxFetchRetry") Integer maxFetchRetry,
- @JacksonInject @Hdfs Configuration conf,
- @JacksonInject HdfsInputSourceConfig inputSourceConfig
- )
- {
- super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
- this.inputPaths = HdfsInputSource.coerceInputPathsToList(inputPaths, "paths");
- this.conf = conf;
- this.inputSourceConfig = inputSourceConfig;
- this.inputPaths.forEach(p -> HdfsInputSource.verifyProtocol(conf, inputSourceConfig, p));
- }
-
- @JsonProperty("paths")
- public List getInputPaths()
- {
- return inputPaths;
- }
-
- @Override
- protected Collection initObjects() throws IOException
- {
- return HdfsInputSource.getPaths(inputPaths, conf);
- }
-
- @Override
- protected InputStream openObjectStream(Path path) throws IOException
- {
- return path.getFileSystem(conf).open(path);
- }
-
- @Override
- protected InputStream openObjectStream(Path path, long start) throws IOException
- {
- final FSDataInputStream in = path.getFileSystem(conf).open(path);
- in.seek(start);
- return in;
- }
-
- @Override
- protected InputStream wrapObjectStream(Path path, InputStream stream) throws IOException
- {
- return CompressionUtils.decompress(stream, path.getName());
- }
-
- @Override
- protected Predicate getRetryCondition()
- {
- return HdfsDataSegmentPuller.RETRY_PREDICATE;
- }
-
- @Override
- public boolean isSplittable()
- {
- return true;
- }
-
- @Override
- public FiniteFirehoseFactory withSplit(InputSplit split)
- {
- return new HdfsFirehoseFactory(
- split.get().toString(),
- getMaxCacheCapacityBytes(),
- getMaxFetchCapacityBytes(),
- getPrefetchTriggerBytes(),
- getFetchTimeout(),
- getMaxFetchRetry(),
- conf,
- inputSourceConfig
- );
- }
-}
diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsStorageDruidModule.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsStorageDruidModule.java
index 3ca8e23535e1..68c7650960b8 100644
--- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsStorageDruidModule.java
+++ b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsStorageDruidModule.java
@@ -26,7 +26,6 @@
import com.google.inject.Inject;
import com.google.inject.multibindings.MapBinder;
import org.apache.druid.data.SearchableVersionedDataFinder;
-import org.apache.druid.firehose.hdfs.HdfsFirehoseFactory;
import org.apache.druid.guice.Binders;
import org.apache.druid.guice.Hdfs;
import org.apache.druid.guice.JsonConfigProvider;
@@ -66,7 +65,6 @@ public List extends Module> getJacksonModules()
return Collections.singletonList(
new SimpleModule().registerSubtypes(
new NamedType(HdfsLoadSpec.class, HdfsStorageDruidModule.SCHEME),
- new NamedType(HdfsFirehoseFactory.class, HdfsStorageDruidModule.SCHEME),
new NamedType(HdfsInputSource.class, HdfsStorageDruidModule.SCHEME)
)
);
diff --git a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactoryTest.java b/extensions-core/hdfs-storage/src/test/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactoryTest.java
deleted file mode 100644
index e96a773c0cef..000000000000
--- a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactoryTest.java
+++ /dev/null
@@ -1,218 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.hdfs;
-
-import com.fasterxml.jackson.databind.InjectableValues.Std;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.ImmutableSet;
-import org.apache.druid.data.input.FirehoseFactory;
-import org.apache.druid.inputsource.hdfs.HdfsInputSourceConfig;
-import org.apache.druid.storage.hdfs.HdfsStorageDruidModule;
-import org.apache.hadoop.conf.Configuration;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.io.IOException;
-import java.util.Collections;
-
-public class HdfsFirehoseFactoryTest
-{
- private static final HdfsInputSourceConfig DEFAULT_INPUT_SOURCE_CONFIG = new HdfsInputSourceConfig(null);
- private static final Configuration DEFAULT_CONFIGURATION = new Configuration();
-
- @BeforeClass
- public static void setup()
- {
- DEFAULT_CONFIGURATION.set("fs.default.name", "hdfs://localhost:7020");
- }
-
- @Rule
- public ExpectedException expectedException = ExpectedException.none();
-
- @Test
- public void testArrayPaths() throws IOException
- {
- final HdfsFirehoseFactory firehoseFactory = new HdfsFirehoseFactory(
- Collections.singletonList("/foo/bar"),
- null,
- null,
- null,
- null,
- null,
- DEFAULT_CONFIGURATION,
- DEFAULT_INPUT_SOURCE_CONFIG
- );
-
- final ObjectMapper mapper = createMapper();
-
- final HdfsFirehoseFactory firehoseFactory2 = (HdfsFirehoseFactory)
- mapper.readValue(mapper.writeValueAsString(firehoseFactory), FirehoseFactory.class);
-
- Assert.assertEquals(
- firehoseFactory.getInputPaths(),
- firehoseFactory2.getInputPaths()
- );
- }
-
- @Test
- public void testStringPaths() throws IOException
- {
- final HdfsFirehoseFactory firehoseFactory = new HdfsFirehoseFactory(
- "/foo/bar",
- null,
- null,
- null,
- null,
- null,
- DEFAULT_CONFIGURATION,
- DEFAULT_INPUT_SOURCE_CONFIG
- );
- final ObjectMapper mapper = createMapper();
-
- final HdfsFirehoseFactory firehoseFactory2 = (HdfsFirehoseFactory)
- mapper.readValue(mapper.writeValueAsString(firehoseFactory), FirehoseFactory.class);
-
- Assert.assertEquals(
- firehoseFactory.getInputPaths(),
- firehoseFactory2.getInputPaths()
- );
- }
-
- @Test
- public void testConstructorAllowsOnlyDefaultProtocol()
- {
- new HdfsFirehoseFactory(
- "hdfs://localhost:7020/foo/bar",
- null,
- null,
- null,
- null,
- null,
- DEFAULT_CONFIGURATION,
- DEFAULT_INPUT_SOURCE_CONFIG
- );
-
- expectedException.expect(IllegalArgumentException.class);
- expectedException.expectMessage("Only [hdfs] protocols are allowed");
- new HdfsFirehoseFactory(
- "file:/foo/bar",
- null,
- null,
- null,
- null,
- null,
- DEFAULT_CONFIGURATION,
- DEFAULT_INPUT_SOURCE_CONFIG
- );
- }
-
- @Test
- public void testConstructorAllowsOnlyCustomProtocol()
- {
- final Configuration conf = new Configuration();
- conf.set("fs.ftp.impl", "org.apache.hadoop.fs.ftp.FTPFileSystem");
- new HdfsFirehoseFactory(
- "ftp://localhost:21/foo/bar",
- null,
- null,
- null,
- null,
- null,
- DEFAULT_CONFIGURATION,
- new HdfsInputSourceConfig(ImmutableSet.of("ftp"))
- );
-
- expectedException.expect(IllegalArgumentException.class);
- expectedException.expectMessage("Only [druid] protocols are allowed");
- new HdfsFirehoseFactory(
- "hdfs://localhost:7020/foo/bar",
- null,
- null,
- null,
- null,
- null,
- DEFAULT_CONFIGURATION,
- new HdfsInputSourceConfig(ImmutableSet.of("druid"))
- );
- }
-
- @Test
- public void testConstructorWithDefaultHdfs()
- {
- new HdfsFirehoseFactory(
- "/foo/bar*",
- null,
- null,
- null,
- null,
- null,
- DEFAULT_CONFIGURATION,
- DEFAULT_INPUT_SOURCE_CONFIG
- );
-
- new HdfsFirehoseFactory(
- "foo/bar*",
- null,
- null,
- null,
- null,
- null,
- DEFAULT_CONFIGURATION,
- DEFAULT_INPUT_SOURCE_CONFIG
- );
-
- new HdfsFirehoseFactory(
- "hdfs:///foo/bar*",
- null,
- null,
- null,
- null,
- null,
- DEFAULT_CONFIGURATION,
- DEFAULT_INPUT_SOURCE_CONFIG
- );
-
- new HdfsFirehoseFactory(
- "hdfs://localhost:10020/foo/bar*", // different hdfs
- null,
- null,
- null,
- null,
- null,
- DEFAULT_CONFIGURATION,
- DEFAULT_INPUT_SOURCE_CONFIG
- );
- }
-
- private static ObjectMapper createMapper()
- {
- final ObjectMapper mapper = new ObjectMapper();
- new HdfsStorageDruidModule().getJacksonModules().forEach(mapper::registerModule);
- mapper.setInjectableValues(
- new Std()
- .addValue(Configuration.class, DEFAULT_CONFIGURATION)
- .addValue(HdfsInputSourceConfig.class, DEFAULT_INPUT_SOURCE_CONFIG)
- );
- return mapper;
- }
-}
diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/S3FirehoseDruidModule.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/S3FirehoseDruidModule.java
deleted file mode 100644
index 6cb5ac5c1d84..000000000000
--- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/S3FirehoseDruidModule.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.s3;
-
-import com.fasterxml.jackson.databind.Module;
-import com.fasterxml.jackson.databind.jsontype.NamedType;
-import com.fasterxml.jackson.databind.module.SimpleModule;
-import com.google.common.collect.ImmutableList;
-import com.google.inject.Binder;
-import org.apache.druid.initialization.DruidModule;
-
-import java.util.List;
-
-/**
- */
-public class S3FirehoseDruidModule implements DruidModule
-{
- @Override
- public List extends Module> getJacksonModules()
- {
- return ImmutableList.of(
- new SimpleModule().registerSubtypes(new NamedType(StaticS3FirehoseFactory.class, "static-s3"))
- );
- }
-
- @Override
- public void configure(Binder binder)
- {
-
- }
-}
diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactory.java
deleted file mode 100644
index 0bf93313dc6a..000000000000
--- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactory.java
+++ /dev/null
@@ -1,236 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.s3;
-
-import com.amazonaws.services.s3.model.AmazonS3Exception;
-import com.amazonaws.services.s3.model.GetObjectRequest;
-import com.amazonaws.services.s3.model.S3Object;
-import com.amazonaws.services.s3.model.S3ObjectSummary;
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.InputSplit;
-import org.apache.druid.data.input.impl.StringInputRowParser;
-import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
-import org.apache.druid.java.util.common.IAE;
-import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.storage.s3.S3Utils;
-import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3;
-import org.apache.druid.utils.CompressionUtils;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
-/**
- * Builds firehoses that read from a predefined list of S3 objects and then dry up.
- */
-public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory
-{
- private static final Logger log = new Logger(StaticS3FirehoseFactory.class);
- private static final int MAX_LISTING_LENGTH = 1024;
-
- private final ServerSideEncryptingAmazonS3 s3Client;
- private final List uris;
- private final List prefixes;
-
- @JsonCreator
- public StaticS3FirehoseFactory(
- @JacksonInject ServerSideEncryptingAmazonS3 s3Client,
- @JsonProperty("uris") List uris,
- @JsonProperty("prefixes") List prefixes,
- @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
- @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
- @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
- @JsonProperty("fetchTimeout") Long fetchTimeout,
- @JsonProperty("maxFetchRetry") Integer maxFetchRetry
- )
- {
- super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
- this.s3Client = Preconditions.checkNotNull(s3Client, "s3Client");
- this.uris = uris == null ? new ArrayList<>() : uris;
- this.prefixes = prefixes == null ? new ArrayList<>() : prefixes;
-
- if (!this.uris.isEmpty() && !this.prefixes.isEmpty()) {
- throw new IAE("uris and prefixes cannot be used together");
- }
-
- if (this.uris.isEmpty() && this.prefixes.isEmpty()) {
- throw new IAE("uris or prefixes must be specified");
- }
-
- for (final URI inputURI : this.uris) {
- Preconditions.checkArgument("s3".equals(inputURI.getScheme()), "input uri scheme == s3 (%s)", inputURI);
- }
-
- for (final URI inputURI : this.prefixes) {
- Preconditions.checkArgument("s3".equals(inputURI.getScheme()), "input uri scheme == s3 (%s)", inputURI);
- }
- }
-
- @JsonProperty
- public List getUris()
- {
- return uris;
- }
-
- @JsonProperty("prefixes")
- public List getPrefixes()
- {
- return prefixes;
- }
-
- @Override
- protected Collection initObjects()
- {
- if (!uris.isEmpty()) {
- return uris;
- } else {
- final List objects = new ArrayList<>();
- for (final URI prefix : prefixes) {
- final Iterator objectSummaryIterator = S3Utils.objectSummaryIterator(
- s3Client,
- Collections.singletonList(prefix),
- MAX_LISTING_LENGTH
- );
-
- objectSummaryIterator.forEachRemaining(objects::add);
- }
- return objects.stream().map(S3Utils::summaryToUri).collect(Collectors.toList());
- }
- }
-
- @Override
- protected InputStream openObjectStream(URI object) throws IOException
- {
- try {
- // Get data of the given object and open an input stream
- final String bucket = object.getAuthority();
- final String key = S3Utils.extractS3Key(object);
-
- final S3Object s3Object = s3Client.getObject(bucket, key);
- if (s3Object == null) {
- throw new ISE("Failed to get an s3 object for bucket[%s] and key[%s]", bucket, key);
- }
- return s3Object.getObjectContent();
- }
- catch (AmazonS3Exception e) {
- throw new IOException(e);
- }
- }
-
- @Override
- protected InputStream openObjectStream(URI object, long start) throws IOException
- {
- final String bucket = object.getAuthority();
- final String key = S3Utils.extractS3Key(object);
-
- final GetObjectRequest request = new GetObjectRequest(bucket, key);
- request.setRange(start);
- try {
- final S3Object s3Object = s3Client.getObject(request);
- if (s3Object == null) {
- throw new ISE(
- "Failed to get an s3 object for bucket[%s], key[%s], and start[%d]",
- bucket,
- key,
- start
- );
- }
- return s3Object.getObjectContent();
- }
- catch (AmazonS3Exception e) {
- throw new IOException(e);
- }
- }
-
- @Override
- protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException
- {
- return CompressionUtils.decompress(stream, S3Utils.extractS3Key(object));
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- StaticS3FirehoseFactory that = (StaticS3FirehoseFactory) o;
-
- return Objects.equals(uris, that.uris) &&
- Objects.equals(prefixes, that.prefixes) &&
- getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() &&
- getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() &&
- getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() &&
- getFetchTimeout() == that.getFetchTimeout() &&
- getMaxFetchRetry() == that.getMaxFetchRetry();
- }
-
- @Override
- public int hashCode()
- {
- return Objects.hash(
- uris,
- prefixes,
- getMaxCacheCapacityBytes(),
- getMaxFetchCapacityBytes(),
- getPrefetchTriggerBytes(),
- getFetchTimeout(),
- getMaxFetchRetry()
- );
- }
-
- @Override
- protected Predicate getRetryCondition()
- {
- return S3Utils.S3RETRY;
- }
-
- @Override
- public FiniteFirehoseFactory withSplit(InputSplit split)
- {
- return new StaticS3FirehoseFactory(
- s3Client,
- Collections.singletonList(split.get()),
- null,
- getMaxCacheCapacityBytes(),
- getMaxFetchCapacityBytes(),
- getPrefetchTriggerBytes(),
- getFetchTimeout(),
- getMaxFetchRetry()
- );
- }
-}
diff --git a/extensions-core/s3-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/s3-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
index 51968bbb97bf..4861e22cb3d5 100644
--- a/extensions-core/s3-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
+++ b/extensions-core/s3-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
@@ -14,5 +14,4 @@
# limitations under the License.
org.apache.druid.storage.s3.S3StorageDruidModule
-org.apache.druid.firehose.s3.S3FirehoseDruidModule
org.apache.druid.data.input.s3.S3InputSourceDruidModule
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactoryTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactoryTest.java
deleted file mode 100644
index c809bf37040d..000000000000
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactoryTest.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.firehose.s3;
-
-import com.amazonaws.services.s3.AmazonS3Client;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.impl.StringInputRowParser;
-import org.apache.druid.data.input.s3.S3InputSourceTest;
-import org.apache.druid.storage.s3.NoopServerSideEncryption;
-import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3;
-import org.easymock.EasyMock;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.List;
-import java.util.stream.Collectors;
-
-/**
- */
-public class StaticS3FirehoseFactoryTest
-{
- private static final AmazonS3Client S3_CLIENT = EasyMock.createNiceMock(AmazonS3Client.class);
- private static final ServerSideEncryptingAmazonS3 SERVICE = new ServerSideEncryptingAmazonS3(
- S3_CLIENT,
- new NoopServerSideEncryption()
- );
-
- @Test
- public void testSerde() throws Exception
- {
- final ObjectMapper mapper = S3InputSourceTest.createS3ObjectMapper();
-
- final List uris = Arrays.asList(
- new URI("s3://foo/bar/file.gz"),
- new URI("s3://bar/foo/file2.gz")
- );
-
- final StaticS3FirehoseFactory factory = new StaticS3FirehoseFactory(
- SERVICE,
- uris,
- null,
- 2048L,
- 1024L,
- 512L,
- 100L,
- 5
- );
-
- final StaticS3FirehoseFactory outputFact = mapper.readValue(
- mapper.writeValueAsString(factory),
- StaticS3FirehoseFactory.class
- );
-
- Assert.assertEquals(factory, outputFact);
- }
-
- @Test
- public void testWithSplit() throws IOException
- {
- final List uris = Arrays.asList(
- URI.create("s3://foo/bar/file.gz"),
- URI.create("s3://bar/foo/file2.gz")
- );
- uris.sort(Comparator.comparing(URI::toString));
-
- final StaticS3FirehoseFactory factory = new StaticS3FirehoseFactory(
- SERVICE,
- uris,
- null,
- 2048L,
- 1024L,
- 512L,
- 100L,
- 5
- );
- final List> subFactories = factory
- .getSplits(null)
- .map(factory::withSplit)
- .sorted(Comparator.comparing(eachFactory -> {
- final StaticS3FirehoseFactory staticS3FirehoseFactory = (StaticS3FirehoseFactory) eachFactory;
- return staticS3FirehoseFactory.getUris().toString();
- }))
- .collect(Collectors.toList());
-
- Assert.assertEquals(uris.size(), subFactories.size());
- for (int i = 0; i < uris.size(); i++) {
- final StaticS3FirehoseFactory staticS3FirehoseFactory = (StaticS3FirehoseFactory) subFactories.get(i);
- final List subFactoryUris = staticS3FirehoseFactory.getUris();
- Assert.assertEquals(1, subFactoryUris.size());
- Assert.assertEquals(uris.get(i), subFactoryUris.get(0));
- }
- }
-}
diff --git a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java
index 83d9825d14f4..8e4952210997 100644
--- a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java
+++ b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java
@@ -24,7 +24,6 @@
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
-import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory;
@@ -38,8 +37,7 @@ public List extends Module> getJacksonModules()
return ImmutableList.of(
new SimpleModule("IndexingServiceFirehoseModule")
.registerSubtypes(
- new NamedType(EventReceiverFirehoseFactory.class, "receiver"),
- new NamedType(IngestSegmentFirehoseFactory.class, "ingestSegment")
+ new NamedType(EventReceiverFirehoseFactory.class, "receiver")
)
);
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/ReingestionTimelineUtils.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/ReingestionTimelineUtils.java
deleted file mode 100644
index 8714fa6933ac..000000000000
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/ReingestionTimelineUtils.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.indexing.common;
-
-import com.google.common.collect.BiMap;
-import com.google.common.collect.HashBiMap;
-import com.google.common.collect.Lists;
-import org.apache.druid.data.input.impl.DimensionsSpec;
-import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.TimelineObjectHolder;
-import org.apache.druid.timeline.partition.PartitionChunk;
-
-import javax.annotation.Nullable;
-import javax.validation.constraints.NotNull;
-import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-/**
- * @deprecated only used by {@link org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory}
- */
-@Deprecated
-public class ReingestionTimelineUtils
-{
- /**
- * @param timelineSegments A list of timeline objects, such as that returned by VersionedIntervalTimeline.lookup().
- * @param excludeDimensions Dimensions to be excluded
- * @return A list of all the unique dimension column names present in the segments within timelineSegments
- */
- public static List getUniqueDimensions(
- List> timelineSegments,
- @Nullable Set excludeDimensions
- )
- {
- final BiMap uniqueDims = HashBiMap.create();
-
- // Here, we try to retain the order of dimensions as they were specified since the order of dimensions may be
- // optimized for performance.
- // Dimensions are extracted from the recent segments to olders because recent segments are likely to be queried more
- // frequently, and thus the performance should be optimized for recent ones rather than old ones.
-
- // timelineSegments are sorted in order of interval
- int index = 0;
- for (TimelineObjectHolder timelineHolder : Lists.reverse(timelineSegments)) {
- for (PartitionChunk chunk : timelineHolder.getObject()) {
- for (String dimension : chunk.getObject().getDimensions()) {
- if (!uniqueDims.containsKey(dimension) &&
- (excludeDimensions == null || !excludeDimensions.contains(dimension))) {
- uniqueDims.put(dimension, index++);
- }
- }
- }
- }
-
- final BiMap orderedDims = uniqueDims.inverse();
- return IntStream.range(0, orderedDims.size())
- .mapToObj(orderedDims::get)
- .collect(Collectors.toList());
- }
-
- /**
- * @param timelineSegments A list of timeline objects, such as that returned by VersionedIntervalTimeline.lookup().
- * @return A list of all the unique metric column names present in the segments within timelineSegments
- */
- public static List getUniqueMetrics(List> timelineSegments)
- {
- final BiMap uniqueMetrics = HashBiMap.create();
-
- // Here, we try to retain the order of metrics as they were specified. Metrics are extracted from the recent
- // segments to olders.
-
- // timelineSegments are sorted in order of interval
- int[] index = {0};
- for (TimelineObjectHolder timelineHolder : Lists.reverse(timelineSegments)) {
- for (PartitionChunk chunk : timelineHolder.getObject()) {
- for (String metric : chunk.getObject().getMetrics()) {
- uniqueMetrics.computeIfAbsent(
- metric,
- k -> {
- return index[0]++;
- }
- );
- }
- }
- }
-
- final BiMap orderedMetrics = uniqueMetrics.inverse();
- return IntStream.range(0, orderedMetrics.size())
- .mapToObj(orderedMetrics::get)
- .collect(Collectors.toList());
- }
-
- /**
- * Utility function to get dimensions that should be ingested. The preferred order is
- * - Explicit dimensions if they are provided.
- * - Custom dimensions are provided in the inputSpec.
- * - Calculate dimensions from the timeline but exclude any dimension exclusions.
- *
- * @param explicitDimensions sent as part of the re-ingestion InputSource.
- * @param dimensionsSpec from the provided ingestion spec.
- * @param timeLineSegments for the datasource that is being read.
- * @return
- */
- public static List getDimensionsToReingest(
- @Nullable List explicitDimensions,
- @NotNull DimensionsSpec dimensionsSpec,
- @NotNull List> timeLineSegments)
- {
- final List dims;
- if (explicitDimensions != null) {
- dims = explicitDimensions;
- } else if (dimensionsSpec.hasCustomDimensions()) {
- dims = dimensionsSpec.getDimensionNames();
- } else {
- dims = ReingestionTimelineUtils.getUniqueDimensions(
- timeLineSegments,
- dimensionsSpec.getDimensionExclusions()
- );
- }
- return dims;
- }
-}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java
index 8fcadc301d22..076298cca07f 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java
@@ -25,7 +25,6 @@
import com.google.common.collect.ImmutableList;
import com.google.errorprone.annotations.concurrent.GuardedBy;
import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec;
-import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputSource;
@@ -45,8 +44,6 @@
import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig;
import org.apache.druid.indexing.common.task.batch.MaxAllowedLocksExceededException;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
-import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory;
-import org.apache.druid.indexing.firehose.WindowedSegmentId;
import org.apache.druid.indexing.input.InputRowSchemas;
import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.java.util.common.IAE;
@@ -599,13 +596,9 @@ static Granularity findGranularityFromSegments(List segments)
}
/**
- * If the given firehoseFactory is {@link IngestSegmentFirehoseFactory}, then it finds the segments to lock
- * from the firehoseFactory. This is because those segments will be read by this task no matter what segments would be
- * filtered by intervalsToRead, so they need to be locked.
*
- * However, firehoseFactory is not IngestSegmentFirehoseFactory, it means this task will overwrite some segments
- * with data read from some input source outside of Druid. As a result, only the segments falling in intervalsToRead
- * should be locked.
+ * This task will overwrite some segments with data read from some input source outside of Druid.
+ * As a result, only the segments falling in intervalsToRead should be locked.
*
* The order of segments within the returned list is unspecified, but each segment is guaranteed to appear in the list
* only once.
@@ -613,48 +606,14 @@ static Granularity findGranularityFromSegments(List segments)
protected static List findInputSegments(
String dataSource,
TaskActionClient actionClient,
- List intervalsToRead,
- FirehoseFactory firehoseFactory
+ List intervalsToRead
) throws IOException
{
- if (firehoseFactory instanceof IngestSegmentFirehoseFactory) {
- // intervalsToRead is ignored here.
- final List inputSegments = ((IngestSegmentFirehoseFactory) firehoseFactory).getSegments();
- if (inputSegments == null) {
- final Interval inputInterval = Preconditions.checkNotNull(
- ((IngestSegmentFirehoseFactory) firehoseFactory).getInterval(),
- "input interval"
- );
-
- return ImmutableList.copyOf(
- actionClient.submit(
- new RetrieveUsedSegmentsAction(dataSource, inputInterval, null, Segments.ONLY_VISIBLE)
- )
- );
- } else {
- final List inputSegmentIds =
- inputSegments.stream().map(WindowedSegmentId::getSegmentId).collect(Collectors.toList());
- final Collection dataSegmentsInIntervals = actionClient.submit(
- new RetrieveUsedSegmentsAction(
- dataSource,
- null,
- inputSegments.stream()
- .flatMap(windowedSegmentId -> windowedSegmentId.getIntervals().stream())
- .collect(Collectors.toSet()),
- Segments.ONLY_VISIBLE
- )
- );
- return dataSegmentsInIntervals.stream()
- .filter(segment -> inputSegmentIds.contains(segment.getId().toString()))
- .collect(Collectors.toList());
- }
- } else {
- return ImmutableList.copyOf(
- actionClient.submit(
- new RetrieveUsedSegmentsAction(dataSource, null, intervalsToRead, Segments.ONLY_VISIBLE)
- )
- );
- }
+ return ImmutableList.copyOf(
+ actionClient.submit(
+ new RetrieveUsedSegmentsAction(dataSource, null, intervalsToRead, Segments.ONLY_VISIBLE)
+ )
+ );
}
/**
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
index 288107d0b944..6af2a394fb13 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
@@ -32,14 +32,11 @@
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing;
import com.google.common.util.concurrent.ListenableFuture;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.FirehoseFactory;
-import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.Rows;
-import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.hll.HyperLogLogCollector;
import org.apache.druid.indexer.Checks;
import org.apache.druid.indexer.IngestionState;
@@ -269,8 +266,7 @@ public List findSegmentsToLock(TaskActionClient taskActionClient, L
return findInputSegments(
getDataSource(),
taskActionClient,
- intervals,
- ingestionSchema.ioConfig.firehoseFactory
+ intervals
);
}
@@ -486,9 +482,7 @@ public TaskStatus runTask(final TaskToolbox toolbox)
.inputIntervals()
.isEmpty();
- final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
- ingestionSchema.getDataSchema().getParser()
- );
+ final InputSource inputSource = ingestionSchema.getIOConfig().getInputSource();
final File tmpDir = toolbox.getIndexingTmpDir();
@@ -1196,16 +1190,9 @@ public InputFormat getInputFormat()
return inputFormat;
}
- public InputSource getNonNullInputSource(@Nullable InputRowParser inputRowParser)
+ public InputSource getNonNullInputSource()
{
- if (inputSource == null) {
- return new FirehoseFactoryToInputSourceAdaptor(
- (FiniteFirehoseFactory) firehoseFactory,
- inputRowParser
- );
- } else {
- return inputSource;
- }
+ return Preconditions.checkNotNull(inputSource, "inputSource");
}
public InputFormat getNonNullInputFormat()
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 3f41dba24c16..bfdad0115562 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
@@ -20,7 +20,6 @@
package org.apache.druid.indexing.common.task.batch.parallel;
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;
@@ -59,9 +58,7 @@ abstract class InputSourceSplitParallelIndexTaskRunner newTaskSpec(InputSplit split)
{
final FirehoseFactory firehoseFactory;
final InputSource inputSource;
- if (baseInputSource instanceof FirehoseFactoryToInputSourceAdaptor) {
- firehoseFactory = ((FirehoseFactoryToInputSourceAdaptor) baseInputSource).getFirehoseFactory().withSplit(split);
- inputSource = null;
- } else {
- firehoseFactory = null;
- inputSource = baseInputSource.withSplit(split);
- }
+ firehoseFactory = null;
+ inputSource = baseInputSource.withSplit(split);
final ParallelIndexIngestionSpec subTaskIngestionSpec = new ParallelIndexIngestionSpec(
ingestionSchema.getDataSchema(),
new ParallelIndexIOConfig(
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java
index 5d057cded060..60a164560d55 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java
@@ -22,7 +22,6 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
-import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
import org.apache.druid.indexer.Checks;
import org.apache.druid.indexer.Property;
import org.apache.druid.java.util.common.IAE;
@@ -45,9 +44,7 @@ public ParallelIndexIngestionSpec(
super(dataSchema, ioConfig, tuningConfig);
if (dataSchema.getParserMap() != null && ioConfig.getInputSource() != null) {
- if (!(ioConfig.getInputSource() instanceof FirehoseFactoryToInputSourceAdaptor)) {
- throw new IAE("Cannot use parser and inputSource together. Try using inputFormat instead of parser.");
- }
+ throw new IAE("Cannot use parser and inputSource together. Try using inputFormat instead of parser.");
}
if (ioConfig.getInputSource() != null && ioConfig.getInputSource().needsFormat()) {
Checks.checkOneNotNullOrEmpty(
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
index f8381ffafe8f..950f6aaf6951 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
@@ -30,7 +30,6 @@
import org.apache.datasketches.hll.Union;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.common.guava.FutureUtils;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.indexer.IngestionState;
@@ -121,9 +120,9 @@
import java.util.stream.Collectors;
/**
- * ParallelIndexSupervisorTask is capable of running multiple subTasks for parallel indexing. This is
- * applicable if the input {@link FiniteFirehoseFactory} is splittable. While this task is running, it can submit
- * multiple child tasks to overlords. This task succeeds only when all its child tasks succeed; otherwise it fails.
+ * ParallelIndexSupervisorTask is capable of running multiple subTasks for parallel indexing.
+ * While this task is running, it can submit multiple child tasks to overlords. This task
+ * succeeds only when all its child tasks succeed; otherwise it fails.
*
* @see ParallelIndexTaskRunner
*/
@@ -241,9 +240,7 @@ public ParallelIndexSupervisorTask(
checkPartitionsSpecForForceGuaranteedRollup(ingestionSchema.getTuningConfig().getGivenOrDefaultPartitionsSpec());
}
- this.baseInputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
- ingestionSchema.getDataSchema().getParser()
- );
+ this.baseInputSource = ingestionSchema.getIOConfig().getNonNullInputSource();
this.missingIntervalsInOverwriteMode = (getIngestionMode()
!= IngestionMode.APPEND)
&& ingestionSchema.getDataSchema()
@@ -426,8 +423,7 @@ public List findSegmentsToLock(TaskActionClient taskActionClient, L
return findInputSegments(
getDataSource(),
taskActionClient,
- intervals,
- ingestionSchema.getIOConfig().getFirehoseFactory()
+ intervals
);
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java
index 0b4def915ac0..250df1afc3f2 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java
@@ -156,9 +156,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception
HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) tuningConfig.getPartitionsSpec();
Preconditions.checkNotNull(partitionsSpec, "partitionsSpec required in tuningConfig");
- InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
- ingestionSchema.getDataSchema().getParser()
- );
+ InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource();
InputFormat inputFormat = inputSource.needsFormat()
? ParallelIndexSupervisorTask.getInputFormat(ingestionSchema)
: null;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java
index e491e0244250..3764ec3884b4 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java
@@ -203,9 +203,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception
);
boolean isAssumeGrouped = partitionsSpec.isAssumeGrouped();
- InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
- ingestionSchema.getDataSchema().getParser()
- );
+ InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource();
InputFormat inputFormat = inputSource.needsFormat()
? ParallelIndexSupervisorTask.getInputFormat(ingestionSchema)
: null;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java
index 03a66d1fd972..212f0c1ab17a 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java
@@ -109,9 +109,7 @@ abstract class PartialSegmentGenerateTask e
@Override
public final TaskStatus runTask(TaskToolbox toolbox) throws Exception
{
- final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
- ingestionSchema.getDataSchema().getParser()
- );
+ final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource();
final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build(
supervisorTaskId,
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 1d51d6fa9cab..7f781e466edd 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
@@ -22,7 +22,6 @@
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.lang3.mutable.MutableObject;
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;
@@ -118,9 +117,7 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner
context
);
this.ingestionSchema = ingestionSchema;
- this.baseInputSource = (SplittableInputSource) ingestionSchema.getIOConfig().getNonNullInputSource(
- ingestionSchema.getDataSchema().getParser()
- );
+ this.baseInputSource = (SplittableInputSource) ingestionSchema.getIOConfig().getNonNullInputSource();
}
@VisibleForTesting
@@ -171,13 +168,9 @@ SubTaskSpec newTaskSpec(InputSplit split)
{
final FirehoseFactory firehoseFactory;
final InputSource inputSource;
- if (baseInputSource instanceof FirehoseFactoryToInputSourceAdaptor) {
- firehoseFactory = ((FirehoseFactoryToInputSourceAdaptor) baseInputSource).getFirehoseFactory().withSplit(split);
- inputSource = null;
- } else {
- firehoseFactory = null;
- inputSource = baseInputSource.withSplit(split);
- }
+ firehoseFactory = null;
+ inputSource = baseInputSource.withSplit(split);
+
final Map subtaskContext = new HashMap<>(getContext());
return new SinglePhaseSubTaskSpec(
getBaseSubtaskSpecName() + "_" + getAndIncrementNextSpecId(),
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java
index db325f8778dc..b1cb7eab86c9 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java
@@ -247,9 +247,7 @@ public TaskStatus runTask(final TaskToolbox toolbox)
ingestionSchema.getTuningConfig().getMaxSavedParseExceptions()
);
- final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
- ingestionSchema.getDataSchema().getParser()
- );
+ final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource();
final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build(
supervisorTaskId,
@@ -307,8 +305,7 @@ public List findSegmentsToLock(TaskActionClient taskActionClient, L
return findInputSegments(
getDataSource(),
taskActionClient,
- intervals,
- ingestionSchema.getIOConfig().getFirehoseFactory()
+ intervals
);
}
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
deleted file mode 100644
index f20a0ddd1a1b..000000000000
--- a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java
+++ /dev/null
@@ -1,330 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.indexing.firehose;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.druid.client.coordinator.CoordinatorClient;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.Firehose;
-import org.apache.druid.data.input.InputSplit;
-import org.apache.druid.data.input.SegmentsSplitHintSpec;
-import org.apache.druid.data.input.SplitHintSpec;
-import org.apache.druid.data.input.impl.InputRowParser;
-import org.apache.druid.indexing.common.ReingestionTimelineUtils;
-import org.apache.druid.indexing.common.RetryPolicyFactory;
-import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
-import org.apache.druid.indexing.input.DruidInputSource;
-import org.apache.druid.java.util.common.HumanReadableBytes;
-import org.apache.druid.java.util.common.IAE;
-import org.apache.druid.java.util.common.parsers.ParseException;
-import org.apache.druid.java.util.emitter.EmittingLogger;
-import org.apache.druid.query.filter.DimFilter;
-import org.apache.druid.segment.IndexIO;
-import org.apache.druid.segment.QueryableIndexStorageAdapter;
-import org.apache.druid.segment.loading.SegmentCacheManager;
-import org.apache.druid.segment.loading.SegmentLoadingException;
-import org.apache.druid.segment.realtime.firehose.IngestSegmentFirehose;
-import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter;
-import org.apache.druid.segment.transform.TransformSpec;
-import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.TimelineObjectHolder;
-import org.apache.druid.timeline.partition.PartitionChunk;
-import org.joda.time.Interval;
-
-import javax.annotation.Nullable;
-import java.io.File;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Stream;
-
-/**
- * @deprecated use {@link DruidInputSource} instead
- */
-@Deprecated
-public class IngestSegmentFirehoseFactory implements FiniteFirehoseFactory>
-{
- private static final EmittingLogger log = new EmittingLogger(IngestSegmentFirehoseFactory.class);
- private final String dataSource;
- // Exactly one of interval and segmentIds should be non-null. Typically 'interval' is specified directly
- // by the user creating this firehose and 'segmentIds' is used for sub-tasks if it is split for parallel
- // batch ingestion.
- @Nullable
- private final Interval interval;
- @Nullable
- private final List segmentIds;
- private final DimFilter dimFilter;
- private final List dimensions;
- private final List metrics;
- @Nullable
- private final Long maxInputSegmentBytesPerTask;
- private final IndexIO indexIO;
- private final CoordinatorClient coordinatorClient;
- private final SegmentCacheManagerFactory segmentCacheManagerFactory;
- private final RetryPolicyFactory retryPolicyFactory;
-
- private List>> splits;
-
- @JsonCreator
- public IngestSegmentFirehoseFactory(
- @JsonProperty("dataSource") final String dataSource,
- @JsonProperty("interval") @Nullable Interval interval,
- // Specifying "segments" is intended only for when this FirehoseFactory has split itself,
- // not for direct end user use.
- @JsonProperty("segments") @Nullable List segmentIds,
- @JsonProperty("filter") DimFilter dimFilter,
- @JsonProperty("dimensions") List dimensions,
- @JsonProperty("metrics") List metrics,
- @JsonProperty("maxInputSegmentBytesPerTask") @Deprecated @Nullable Long maxInputSegmentBytesPerTask,
- @JacksonInject IndexIO indexIO,
- @JacksonInject CoordinatorClient coordinatorClient,
- @JacksonInject SegmentCacheManagerFactory segmentCacheManagerFactory,
- @JacksonInject RetryPolicyFactory retryPolicyFactory
- )
- {
- Preconditions.checkNotNull(dataSource, "dataSource");
- if ((interval == null && segmentIds == null) || (interval != null && segmentIds != null)) {
- throw new IAE("Specify exactly one of 'interval' and 'segments'");
- }
- this.dataSource = dataSource;
- this.interval = interval;
- this.segmentIds = segmentIds;
- this.dimFilter = dimFilter;
- this.dimensions = dimensions;
- this.metrics = metrics;
- this.maxInputSegmentBytesPerTask = maxInputSegmentBytesPerTask;
- this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO");
- this.coordinatorClient = Preconditions.checkNotNull(coordinatorClient, "null CoordinatorClient");
- this.segmentCacheManagerFactory = Preconditions.checkNotNull(segmentCacheManagerFactory, "null segmentCacheManagerFactory");
- this.retryPolicyFactory = Preconditions.checkNotNull(retryPolicyFactory, "null RetryPolicyFactory");
- }
-
- @Override
- public FiniteFirehoseFactory> withSplit(InputSplit> split)
- {
- return new IngestSegmentFirehoseFactory(
- dataSource,
- null,
- split.get(),
- dimFilter,
- dimensions,
- metrics,
- maxInputSegmentBytesPerTask,
- indexIO,
- coordinatorClient,
- segmentCacheManagerFactory,
- retryPolicyFactory
- );
- }
-
- @JsonProperty
- public String getDataSource()
- {
- return dataSource;
- }
-
- @JsonProperty
- @Nullable
- public Interval getInterval()
- {
- return interval;
- }
-
- @JsonProperty
- @Nullable
- public List getSegments()
- {
- return segmentIds;
- }
-
- @JsonProperty("filter")
- public DimFilter getDimensionsFilter()
- {
- return dimFilter;
- }
-
- @JsonProperty
- public List getDimensions()
- {
- return dimensions;
- }
-
- @JsonProperty
- public List getMetrics()
- {
- return metrics;
- }
-
- @Nullable
- @JsonProperty
- public Long getMaxInputSegmentBytesPerTask()
- {
- return maxInputSegmentBytesPerTask;
- }
-
- @Override
- public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) throws ParseException
- {
- log.debug(
- "Connecting firehose: dataSource[%s], interval[%s], segmentIds[%s]",
- dataSource,
- interval,
- segmentIds
- );
-
- final List> timeLineSegments = getTimeline();
-
- // Download all segments locally.
- // Note: this requires enough local storage space to fit all of the segments, even though
- // IngestSegmentFirehose iterates over the segments in series. We may want to change this
- // to download files lazily, perhaps sharing code with PrefetchableTextFilesFirehoseFactory.
- final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(temporaryDirectory);
- Map segmentFileMap = Maps.newLinkedHashMap();
- for (TimelineObjectHolder holder : timeLineSegments) {
- for (PartitionChunk chunk : holder.getObject()) {
- final DataSegment segment = chunk.getObject();
-
- segmentFileMap.computeIfAbsent(segment, k -> {
- try {
- return segmentCacheManager.getSegmentFiles(segment);
- }
- catch (SegmentLoadingException e) {
- throw new RuntimeException(e);
- }
- });
- }
- }
-
- final List dims = ReingestionTimelineUtils.getDimensionsToReingest(
- dimensions,
- inputRowParser.getParseSpec().getDimensionsSpec(),
- timeLineSegments
- );
- final List metricsList = metrics == null
- ? ReingestionTimelineUtils.getUniqueMetrics(timeLineSegments)
- : metrics;
-
- final List adapters = Lists.newArrayList(
- Iterables.concat(
- Iterables.transform(
- timeLineSegments,
- new Function, Iterable>() {
- @Override
- public Iterable apply(final TimelineObjectHolder holder)
- {
- return
- Iterables.transform(
- holder.getObject(),
- new Function, WindowedStorageAdapter>() {
- @Override
- public WindowedStorageAdapter apply(final PartitionChunk input)
- {
- final DataSegment segment = input.getObject();
- try {
- return new WindowedStorageAdapter(
- new QueryableIndexStorageAdapter(
- indexIO.loadIndex(
- Preconditions.checkNotNull(
- segmentFileMap.get(segment),
- "File for segment %s", segment.getId()
- )
- )
- ),
- holder.getInterval()
- );
- }
- catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
- }
- );
- }
- }
- )
- )
- );
-
- final TransformSpec transformSpec = TransformSpec.fromInputRowParser(inputRowParser);
- return new IngestSegmentFirehose(adapters, transformSpec, dims, metricsList, dimFilter);
- }
-
- private List> getTimeline()
- {
- if (interval == null) {
- return DruidInputSource.getTimelineForSegmentIds(coordinatorClient, dataSource, segmentIds);
- } else {
- return DruidInputSource.getTimelineForInterval(coordinatorClient, retryPolicyFactory, dataSource, interval);
- }
- }
-
- private void initializeSplitsIfNeeded(@Nullable SplitHintSpec splitHintSpec)
- {
- if (splits != null) {
- return;
- }
-
- splits = Lists.newArrayList(
- DruidInputSource.createSplits(
- coordinatorClient,
- retryPolicyFactory,
- dataSource,
- interval,
- splitHintSpec == null
- ? new SegmentsSplitHintSpec(
- maxInputSegmentBytesPerTask == null
- ? null
- : new HumanReadableBytes(maxInputSegmentBytesPerTask),
- null
- )
- : splitHintSpec
- )
- );
- }
-
- @Override
- public boolean isSplittable()
- {
- // Specifying 'segments' to this factory instead of 'interval' is intended primarily for internal use by
- // parallel batch injection: we don't need to support splitting a list of segments.
- return interval != null;
- }
-
- @Override
- public Stream>> getSplits(@Nullable SplitHintSpec splitHintSpec)
- {
- initializeSplitsIfNeeded(splitHintSpec);
- return splits.stream();
- }
-
- @Override
- public int getNumSplits(@Nullable SplitHintSpec splitHintSpec)
- {
- initializeSplitsIfNeeded(splitHintSpec);
- return splits.size();
- }
-}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java
index 4140e014ef7a..a0b84e1e7571 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java
@@ -25,13 +25,9 @@
import com.google.common.base.Preconditions;
import org.apache.druid.client.indexing.SamplerResponse;
import org.apache.druid.client.indexing.SamplerSpec;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.FirehoseFactory;
-import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.indexing.common.task.IndexTask;
-import org.apache.druid.java.util.common.IAE;
import org.apache.druid.segment.indexing.DataSchema;
import javax.annotation.Nullable;
@@ -61,28 +57,17 @@ public IndexTaskSamplerSpec(
Preconditions.checkNotNull(ingestionSpec.getIOConfig(), "[spec.ioConfig] is required");
- if (ingestionSpec.getIOConfig().getInputSource() != null) {
- this.inputSource = ingestionSpec.getIOConfig().getInputSource();
- if (ingestionSpec.getIOConfig().getInputSource().needsFormat()) {
- this.inputFormat = Preconditions.checkNotNull(
- ingestionSpec.getIOConfig().getInputFormat(),
- "[spec.ioConfig.inputFormat] is required"
- );
- } else {
- this.inputFormat = null;
- }
- } else {
- final FirehoseFactory firehoseFactory = Preconditions.checkNotNull(
- ingestionSpec.getIOConfig().getFirehoseFactory(),
- "[spec.ioConfig.firehose] is required"
- );
- if (!(firehoseFactory instanceof FiniteFirehoseFactory)) {
- throw new IAE("firehose should be an instance of FiniteFirehoseFactory");
- }
- this.inputSource = new FirehoseFactoryToInputSourceAdaptor(
- (FiniteFirehoseFactory) firehoseFactory,
- ingestionSpec.getDataSchema().getParser()
+ this.inputSource = Preconditions.checkNotNull(
+ ingestionSpec.getIOConfig().getInputSource(),
+ "[spec.ioConfig.inputSource] is required"
+ );
+
+ if (inputSource.needsFormat()) {
+ this.inputFormat = Preconditions.checkNotNull(
+ ingestionSpec.getIOConfig().getInputFormat(),
+ "[spec.ioConfig.inputFormat] is required"
);
+ } else {
this.inputFormat = null;
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java
index 3be7a4feb191..424511c8e8e0 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java
@@ -23,20 +23,9 @@
import com.google.common.base.Throwables;
import org.apache.druid.client.indexing.SamplerResponse;
import org.apache.druid.client.indexing.SamplerSpec;
-import org.apache.druid.data.input.ByteBufferInputRowParser;
-import org.apache.druid.data.input.FiniteFirehoseFactory;
-import org.apache.druid.data.input.Firehose;
-import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
-import org.apache.druid.data.input.InputEntity;
import org.apache.druid.data.input.InputFormat;
-import org.apache.druid.data.input.InputRow;
-import org.apache.druid.data.input.InputRowListPlusRawValues;
import org.apache.druid.data.input.InputSource;
-import org.apache.druid.data.input.InputSplit;
-import org.apache.druid.data.input.SplitHintSpec;
import org.apache.druid.data.input.impl.ByteEntity;
-import org.apache.druid.data.input.impl.InputRowParser;
-import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.indexing.overlord.sampler.InputSourceSampler;
import org.apache.druid.indexing.overlord.sampler.SamplerConfig;
import org.apache.druid.indexing.overlord.sampler.SamplerException;
@@ -44,17 +33,9 @@
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig;
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec;
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig;
-import org.apache.druid.java.util.common.parsers.CloseableIterator;
-import org.apache.druid.java.util.common.parsers.ParseException;
import org.apache.druid.segment.indexing.DataSchema;
import javax.annotation.Nullable;
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Stream;
public abstract class SeekableStreamSamplerSpec implements SamplerSpec
{
@@ -87,139 +68,29 @@ public SamplerResponse sample()
{
final InputSource inputSource;
final InputFormat inputFormat;
- if (dataSchema.getParser() != null) {
- inputSource = new FirehoseFactoryToInputSourceAdaptor(
- new SeekableStreamSamplerFirehoseFactory(),
- dataSchema.getParser()
- );
- inputFormat = null;
- } else {
- RecordSupplier recordSupplier;
- try {
- recordSupplier = createRecordSupplier();
- }
- catch (Exception e) {
- throw new SamplerException(e, "Unable to create RecordSupplier: %s", Throwables.getRootCause(e).getMessage());
- }
+ RecordSupplier recordSupplier;
- inputSource = new RecordSupplierInputSource<>(
- ioConfig.getStream(),
- recordSupplier,
- ioConfig.isUseEarliestSequenceNumber()
- );
- inputFormat = Preconditions.checkNotNull(
- ioConfig.getInputFormat(),
- "[spec.ioConfig.inputFormat] is required"
- );
+ try {
+ recordSupplier = createRecordSupplier();
}
-
- return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig);
- }
-
- protected abstract RecordSupplier createRecordSupplier();
-
- private class SeekableStreamSamplerFirehoseFactory implements FiniteFirehoseFactory
- {
- @Override
- public Firehose connect(ByteBufferInputRowParser parser, @Nullable File temporaryDirectory)
- {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Firehose connectForSampler(ByteBufferInputRowParser parser, @Nullable File temporaryDirectory)
- {
- return new SeekableStreamSamplerFirehose(parser);
- }
-
- @Override
- public boolean isSplittable()
- {
- return false;
- }
-
- @Override
- public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec)
- {
- throw new UnsupportedOperationException();
+ catch (Exception e) {
+ throw new SamplerException(e, "Unable to create RecordSupplier: %s", Throwables.getRootCause(e).getMessage());
}
- @Override
- public int getNumSplits(@Nullable SplitHintSpec splitHintSpec)
- {
- throw new UnsupportedOperationException();
- }
+ inputSource = new RecordSupplierInputSource<>(
+ ioConfig.getStream(),
+ recordSupplier,
+ ioConfig.isUseEarliestSequenceNumber()
+ );
+ inputFormat = Preconditions.checkNotNull(
+ ioConfig.getInputFormat(),
+ "[spec.ioConfig.inputFormat] is required"
+ );
- @Override
- public FiniteFirehoseFactory withSplit(InputSplit split)
- {
- throw new UnsupportedOperationException();
- }
+ return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig);
}
- private class SeekableStreamSamplerFirehose implements Firehose
- {
- private final InputRowParser parser;
- private final CloseableIterator entityIterator;
-
- protected SeekableStreamSamplerFirehose(InputRowParser parser)
- {
- this.parser = parser;
- if (parser instanceof StringInputRowParser) {
- ((StringInputRowParser) parser).startFileFromBeginning();
- }
-
- RecordSupplierInputSource inputSource = new RecordSupplierInputSource<>(
- ioConfig.getStream(),
- createRecordSupplier(),
- ioConfig.isUseEarliestSequenceNumber()
- );
- this.entityIterator = inputSource.createEntityIterator();
- }
-
- @Override
- public boolean hasMore()
- {
- return entityIterator.hasNext();
- }
-
- @Override
- public InputRow nextRow()
- {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public InputRowListPlusRawValues nextRowWithRaw()
- {
- final ByteBuffer bb = ((ByteEntity) entityIterator.next()).getBuffer();
-
- final Map rawColumns;
- try {
- if (parser instanceof StringInputRowParser) {
- rawColumns = ((StringInputRowParser) parser).buildStringKeyMap(bb);
- } else {
- rawColumns = null;
- }
- }
- catch (ParseException e) {
- return InputRowListPlusRawValues.of(null, e);
- }
-
- try {
- final List rows = parser.parseBatch(bb);
- return InputRowListPlusRawValues.of(rows.isEmpty() ? null : rows, rawColumns);
- }
- catch (ParseException e) {
- return InputRowListPlusRawValues.of(rawColumns, e);
- }
- }
+ protected abstract RecordSupplier createRecordSupplier();
- @Override
- public void close() throws IOException
- {
- entityIterator.close();
- }
- }
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestRealtimeTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java
similarity index 62%
rename from indexing-service/src/test/java/org/apache/druid/indexing/common/TestRealtimeTask.java
rename to indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java
index e8975e39ff43..72b0a610156f 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestRealtimeTask.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java
@@ -24,26 +24,27 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.data.input.impl.JsonInputFormat;
+import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.common.task.RealtimeIndexTask;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.indexing.DataSchema;
-import org.apache.druid.segment.indexing.RealtimeIOConfig;
-import org.apache.druid.segment.realtime.FireDepartment;
-import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import java.io.File;
/**
*/
-@JsonTypeName("test_realtime")
-public class TestRealtimeTask extends RealtimeIndexTask
+@JsonTypeName("test_index")
+public class TestIndexTask extends IndexTask
{
private final TaskStatus status;
@JsonCreator
- public TestRealtimeTask(
+ public TestIndexTask(
@JsonProperty("id") String id,
@JsonProperty("resource") TaskResource taskResource,
@JsonProperty("dataSource") String dataSource,
@@ -54,13 +55,42 @@ public TestRealtimeTask(
super(
id,
taskResource,
- new FireDepartment(
+ new IndexIngestionSpec(
new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, null, mapper),
- new RealtimeIOConfig(
- new LocalFirehoseFactory(new File("lol"), "rofl", null),
- (schema, config, metrics) -> null
+ new IndexTask.IndexIOConfig(
+ null,
+ new LocalInputSource(new File("lol"), "rofl"),
+ new JsonInputFormat(null, null, null),
+ false,
+ false
),
- null
+
+ new IndexTask.IndexTuningConfig(
+ null,
+ null,
+ null,
+ 10,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ new DynamicPartitionsSpec(10000, null),
+ new IndexSpec(),
+ null,
+ 3,
+ false,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null
+ )
),
null
);
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java
index b0ddf61c5602..7ad65b1780c3 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java
@@ -32,7 +32,6 @@
import org.apache.druid.data.input.impl.CSVParseSpec;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.ParseSpec;
-import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
@@ -47,7 +46,6 @@
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.task.CompactionTask.Builder;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
-import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory;
import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.ISE;
@@ -70,7 +68,6 @@
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.VirtualColumns;
-import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.join.NoopJoinableFactory;
import org.apache.druid.segment.loading.LocalDataSegmentPuller;
@@ -1488,96 +1485,6 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio
Assert.assertEquals(TaskState.FAILED, compactionResult.lhs.getStatusCode());
}
- /**
- * Run a regular index task that's equivalent to the compaction task in {@link #testRunWithDynamicPartitioning()},
- * using {@link IngestSegmentFirehoseFactory}.
- *
- * This is not entirely CompactionTask related, but it's similar conceptually and it requires
- * similar setup to what this test suite already has.
- *
- * It could be moved to a separate test class if needed.
- */
- @Test
- public void testRunRegularIndexTaskWithIngestSegmentFirehose() throws Exception
- {
- runIndexTask();
-
- IndexTask indexTask = new IndexTask(
- null,
- null,
- new IndexTask.IndexIngestionSpec(
- new DataSchema(
- "test",
- getObjectMapper().convertValue(
- new StringInputRowParser(
- DEFAULT_PARSE_SPEC,
- null
- ),
- Map.class
- ),
- new AggregatorFactory[]{
- new LongSumAggregatorFactory("val", "val")
- },
- new UniformGranularitySpec(
- Granularities.HOUR,
- Granularities.MINUTE,
- null
- ),
- null,
- getObjectMapper()
- ),
- new IndexTask.IndexIOConfig(
- new IngestSegmentFirehoseFactory(
- DATA_SOURCE,
- Intervals.of("2014-01-01/2014-01-02"),
- null,
- null,
- null,
- null,
- null,
- getIndexIO(),
- coordinatorClient,
- segmentCacheManagerFactory,
- RETRY_POLICY_FACTORY
- ),
- false,
- false
- ),
- IndexTaskTest.createTuningConfig(5000000, null, null, Long.MAX_VALUE, null, false, true)
- ),
- null
- );
-
- // This is a regular index so we need to explicitly add this context to store the CompactionState
- indexTask.addToContext(Tasks.STORE_COMPACTION_STATE_KEY, true);
-
- final Pair> resultPair = runTask(indexTask);
-
- Assert.assertTrue(resultPair.lhs.isSuccess());
-
- final List segments = resultPair.rhs;
- Assert.assertEquals(3, segments.size());
-
- for (int i = 0; i < 3; i++) {
- Assert.assertEquals(
- Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1),
- segments.get(i).getInterval()
- );
- Assert.assertEquals(
- getDefaultCompactionState(Granularities.HOUR, Granularities.MINUTE, ImmutableList.of()),
- segments.get(i).getLastCompactionState()
- );
- if (lockGranularity == LockGranularity.SEGMENT) {
- Assert.assertEquals(
- new NumberedOverwriteShardSpec(32768, 0, 2, (short) 1, (short) 1),
- segments.get(i).getShardSpec()
- );
- } else {
- Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec());
- }
- }
- }
-
private Pair> runIndexTask() throws Exception
{
return runIndexTask(null, null, false);
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java
index 6af2e194c1f3..7555c6b1e378 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java
@@ -20,11 +20,8 @@
package org.apache.druid.indexing.common.task;
import com.google.common.collect.ImmutableMap;
-import org.apache.druid.data.input.impl.DimensionsSpec;
-import org.apache.druid.data.input.impl.NoopFirehoseFactory;
import org.apache.druid.data.input.impl.NoopInputFormat;
import org.apache.druid.data.input.impl.NoopInputSource;
-import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig;
import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec;
import org.apache.druid.java.util.common.granularity.Granularities;
@@ -91,56 +88,4 @@ public void testParserAndInputSource()
null
);
}
-
- @Test
- public void testFirehoseAndInputSource()
- {
- expectedException.expect(IllegalArgumentException.class);
- expectedException.expectMessage(
- "At most one of [Property{name='firehose', value=NoopFirehoseFactory{}}, Property{name='inputSource'"
- );
- final IndexIngestionSpec spec = new IndexIngestionSpec(
- new DataSchema(
- "dataSource",
- new TimestampSpec(null, null, null),
- DimensionsSpec.EMPTY,
- new AggregatorFactory[0],
- new ArbitraryGranularitySpec(Granularities.NONE, null),
- null
- ),
- new IndexIOConfig(
- new NoopFirehoseFactory(),
- new NoopInputSource(),
- null,
- null,
- null
- ),
- null
- );
- }
-
- @Test
- public void testFirehoseAndInputFormat()
- {
- expectedException.expect(IllegalArgumentException.class);
- expectedException.expectMessage("Cannot use firehose and inputFormat together.");
- final IndexIngestionSpec spec = new IndexIngestionSpec(
- new DataSchema(
- "dataSource",
- new TimestampSpec(null, null, null),
- DimensionsSpec.EMPTY,
- new AggregatorFactory[0],
- new ArbitraryGranularitySpec(Granularities.NONE, null),
- null
- ),
- new IndexIOConfig(
- new NoopFirehoseFactory(),
- null,
- new NoopInputFormat(),
- null,
- null
- ),
- null
- );
- }
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java
index babc3ff1fd2c..49bc6399dbd4 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
@@ -38,7 +38,6 @@
import org.apache.druid.data.input.impl.LongDimensionSchema;
import org.apache.druid.data.input.impl.ParseSpec;
import org.apache.druid.data.input.impl.StringDimensionSchema;
-import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
@@ -87,7 +86,6 @@
import org.apache.druid.segment.loading.SegmentLocalCacheManager;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
-import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter;
import org.apache.druid.segment.realtime.plumber.NoopSegmentHandoffNotifierFactory;
import org.apache.druid.segment.transform.ExpressionTransform;
@@ -958,7 +956,9 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception
ingestionSpec = createIngestionSpec(
jsonMapper,
tmpDir,
- new CSVParseSpec(timestampSpec, DimensionsSpec.EMPTY, null, columns, true, 0),
+ timestampSpec,
+ DimensionsSpec.EMPTY,
+ new CsvInputFormat(columns, null, null, true, 0),
null,
null,
tuningConfig,
@@ -969,9 +969,7 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception
ingestionSpec = createIngestionSpec(
jsonMapper,
tmpDir,
- timestampSpec,
- DimensionsSpec.EMPTY,
- new CsvInputFormat(columns, null, null, true, 0),
+ new CSVParseSpec(timestampSpec, DimensionsSpec.EMPTY, null, columns, true, 0),
null,
null,
tuningConfig,
@@ -1486,12 +1484,6 @@ public void testReportParseException() throws Exception
false,
false
);
- expectedMessages = ImmutableList.of(
- StringUtils.format(
- "Timestamp[unparseable] is unparseable! Event: {time=unparseable, d=a, val=1} (Path: %s, Record: 1, Line: 2)",
- tmpFile.toURI()
- )
- );
} else {
indexIngestionSpec = createIngestionSpec(
jsonMapper,
@@ -1503,11 +1495,14 @@ public void testReportParseException() throws Exception
false,
false
);
- expectedMessages = ImmutableList.of(
- "Timestamp[unparseable] is unparseable! Event: {time=unparseable, d=a, val=1}"
- );
}
+ expectedMessages = ImmutableList.of(
+ StringUtils.format(
+ "Timestamp[unparseable] is unparseable! Event: {time=unparseable, d=a, val=1} (Path: %s, Record: 1, Line: 2)",
+ tmpFile.toURI()
+ )
+ );
IndexTask indexTask = new IndexTask(
null,
null,
@@ -1661,36 +1656,24 @@ public void testMultipleParseExceptionsSuccess() throws Exception
.get(RowIngestionMeters.BUILD_SEGMENTS);
List expectedMessages;
- if (useInputFormatApi) {
- expectedMessages = Arrays.asList(
- StringUtils.format("Unable to parse row [this is not JSON] (Path: %s, Record: 6, Line: 9)", tmpFile.toURI()),
- StringUtils.format(
- "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 6, Line: 8)",
- tmpFile.toURI()
- ),
- StringUtils.format(
- "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}] (Path: %s, Record: 5, Line: 6)",
- tmpFile.toURI()
- ),
- "Unable to parse value[notnumber] for field[val]",
- "could not convert value [notnumber] to float",
- "could not convert value [notnumber] to long",
- StringUtils.format(
- "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 1)",
- tmpFile.toURI()
- )
- );
- } else {
- expectedMessages = Arrays.asList(
- "Unable to parse row [this is not JSON]",
- "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}",
- "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}]",
- "Unable to parse value[notnumber] for field[val]",
- "could not convert value [notnumber] to float",
- "could not convert value [notnumber] to long",
- "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}"
- );
- }
+ expectedMessages = Arrays.asList(
+ StringUtils.format("Unable to parse row [this is not JSON] (Path: %s, Record: 6, Line: 9)", tmpFile.toURI()),
+ StringUtils.format(
+ "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 6, Line: 8)",
+ tmpFile.toURI()
+ ),
+ StringUtils.format(
+ "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}] (Path: %s, Record: 5, Line: 6)",
+ tmpFile.toURI()
+ ),
+ "Unable to parse value[notnumber] for field[val]",
+ "could not convert value [notnumber] to float",
+ "could not convert value [notnumber] to long",
+ StringUtils.format(
+ "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 1)",
+ tmpFile.toURI()
+ )
+ );
List actualMessages = parseExceptionReports.stream().map((r) -> {
return ((List) r.get("details")).get(0);
@@ -1715,30 +1698,21 @@ public void testMultipleParseExceptionsSuccess() throws Exception
.getUnparseableEvents()
.get(RowIngestionMeters.DETERMINE_PARTITIONS);
- if (useInputFormatApi) {
- expectedMessages = Arrays.asList(
- StringUtils.format("Unable to parse row [this is not JSON] (Path: %s, Record: 6, Line: 9)", tmpFile.toURI()),
- StringUtils.format(
- "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 6, Line: 8)",
- tmpFile.toURI()
- ),
- StringUtils.format(
- "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}] (Path: %s, Record: 5, Line: 6)",
- tmpFile.toURI()
- ),
- StringUtils.format(
- "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 1)",
- tmpFile.toURI()
- )
- );
- } else {
- expectedMessages = Arrays.asList(
- "Unable to parse row [this is not JSON]",
- "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}",
- "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}]",
- "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}"
- );
- }
+ expectedMessages = Arrays.asList(
+ StringUtils.format("Unable to parse row [this is not JSON] (Path: %s, Record: 6, Line: 9)", tmpFile.toURI()),
+ StringUtils.format(
+ "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 6, Line: 8)",
+ tmpFile.toURI()
+ ),
+ StringUtils.format(
+ "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}] (Path: %s, Record: 5, Line: 6)",
+ tmpFile.toURI()
+ ),
+ StringUtils.format(
+ "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 1)",
+ tmpFile.toURI()
+ )
+ );
actualMessages = parseExceptionReports.stream().map((r) -> {
return ((List) r.get("details")).get(0);
@@ -1827,20 +1801,6 @@ public void testMultipleParseExceptionsFailure() throws Exception
false,
false
);
- expectedMessages = Arrays.asList(
- StringUtils.format(
- "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 3, Line: 6)",
- tmpFile.toURI()
- ),
- StringUtils.format(
- "Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 2, Line: 4)",
- tmpFile.toURI()
- ),
- StringUtils.format(
- "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 2)",
- tmpFile.toURI()
- )
- );
} else {
ingestionSpec = createIngestionSpec(
jsonMapper,
@@ -1852,13 +1812,22 @@ public void testMultipleParseExceptionsFailure() throws Exception
false,
false
);
- expectedMessages = Arrays.asList(
- "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}",
- "Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1}",
- "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}"
- );
}
+ expectedMessages = Arrays.asList(
+ StringUtils.format(
+ "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 3, Line: 6)",
+ tmpFile.toURI()
+ ),
+ StringUtils.format(
+ "Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 2, Line: 4)",
+ tmpFile.toURI()
+ ),
+ StringUtils.format(
+ "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 2)",
+ tmpFile.toURI()
+ )
+ );
IndexTask indexTask = new IndexTask(
null,
null,
@@ -1885,7 +1854,7 @@ public void testMultipleParseExceptionsFailure() throws Exception
RowIngestionMeters.PROCESSED_WITH_ERROR, 0,
RowIngestionMeters.PROCESSED, 1,
RowIngestionMeters.UNPARSEABLE, 3,
- RowIngestionMeters.THROWN_AWAY, useInputFormatApi ? 1 : 2
+ RowIngestionMeters.THROWN_AWAY, 1
)
);
@@ -1980,11 +1949,6 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc
false,
false
);
- expectedMessages = Arrays.asList(
- StringUtils.format("Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 3, Line: 6)", tmpFile.toURI()),
- StringUtils.format("Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 2, Line: 4)", tmpFile.toURI()),
- StringUtils.format("Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 2)", tmpFile.toURI())
- );
} else {
ingestionSpec = createIngestionSpec(
jsonMapper,
@@ -1996,13 +1960,13 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc
false,
false
);
- expectedMessages = Arrays.asList(
- "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}",
- "Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1}",
- "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}"
- );
}
+ expectedMessages = Arrays.asList(
+ StringUtils.format("Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 3, Line: 6)", tmpFile.toURI()),
+ StringUtils.format("Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 2, Line: 4)", tmpFile.toURI()),
+ StringUtils.format("Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 2)", tmpFile.toURI())
+ );
IndexTask indexTask = new IndexTask(
null,
null,
@@ -2022,7 +1986,7 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc
RowIngestionMeters.PROCESSED_WITH_ERROR, 0,
RowIngestionMeters.PROCESSED, 1,
RowIngestionMeters.UNPARSEABLE, 3,
- RowIngestionMeters.THROWN_AWAY, useInputFormatApi ? 1 : 2
+ RowIngestionMeters.THROWN_AWAY, 1
),
RowIngestionMeters.BUILD_SEGMENTS,
ImmutableMap.of(
@@ -2172,12 +2136,6 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception
false,
false
);
- expectedMessages = ImmutableList.of(
- StringUtils.format(
- "Timestamp[null] is unparseable! Event: {column_1=2014-01-01T00:00:10Z, column_2=a, column_3=1} (Path: %s, Record: 1, Line: 2)",
- tmpFile.toURI()
- )
- );
} else {
ingestionSpec = createIngestionSpec(
jsonMapper,
@@ -2189,11 +2147,14 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception
false,
false
);
- expectedMessages = ImmutableList.of(
- "Timestamp[null] is unparseable! Event: {column_1=2014-01-01T00:00:10Z, column_2=a, column_3=1}"
- );
}
+ expectedMessages = ImmutableList.of(
+ StringUtils.format(
+ "Timestamp[null] is unparseable! Event: {column_1=2014-01-01T00:00:10Z, column_2=a, column_3=1} (Path: %s, Record: 1, Line: 2)",
+ tmpFile.toURI()
+ )
+ );
IndexTask indexTask = new IndexTask(
null,
null,
@@ -2954,16 +2915,12 @@ private static IndexIngestionSpec createIngestionSpec(
tuningConfig
);
} else {
+ parseSpec = parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC;
return new IndexIngestionSpec(
new DataSchema(
DATASOURCE,
- objectMapper.convertValue(
- new StringInputRowParser(
- parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC,
- null
- ),
- Map.class
- ),
+ parseSpec.getTimestampSpec(),
+ parseSpec.getDimensionsSpec(),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
@@ -2973,14 +2930,13 @@ private static IndexIngestionSpec createIngestionSpec(
Collections.singletonList(Intervals.of("2014/2015"))
),
transformSpec,
+ null,
objectMapper
),
new IndexIOConfig(
- new LocalFirehoseFactory(
- baseDir,
- "druid*",
- null
- ),
+ null,
+ new LocalInputSource(baseDir, "druid*"),
+ createInputFormatFromParseSpec(parseSpec),
appendToExisting,
dropExisting
),
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
index c6d10f2b87e3..11196d86876b 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
@@ -23,6 +23,18 @@
import com.google.common.base.Optional;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.druid.data.input.FirehoseFactory;
+import org.apache.druid.data.input.InputFormat;
+import org.apache.druid.data.input.impl.CSVParseSpec;
+import org.apache.druid.data.input.impl.CsvInputFormat;
+import org.apache.druid.data.input.impl.DelimitedInputFormat;
+import org.apache.druid.data.input.impl.DelimitedParseSpec;
+import org.apache.druid.data.input.impl.InputRowParser;
+import org.apache.druid.data.input.impl.JSONParseSpec;
+import org.apache.druid.data.input.impl.JsonInputFormat;
+import org.apache.druid.data.input.impl.ParseSpec;
+import org.apache.druid.data.input.impl.RegexInputFormat;
+import org.apache.druid.data.input.impl.RegexParseSpec;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter;
@@ -46,6 +58,7 @@
import org.apache.druid.indexing.overlord.autoscaling.ScalingStats;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.metadata.EntryExistsException;
@@ -220,6 +233,50 @@ public IndexMergerV9Factory getIndexMergerV9Factory()
return testUtils.getIndexMergerV9Factory();
}
+ /**
+ * Converts ParseSpec to InputFormat for indexing tests. To be used until {@link FirehoseFactory}
+ * & {@link InputRowParser} is deprecated and removed.
+ *
+ * @param parseSpec
+ * @return
+ */
+ public static InputFormat createInputFormatFromParseSpec(ParseSpec parseSpec)
+ {
+ if (parseSpec instanceof JSONParseSpec) {
+ JSONParseSpec jsonParseSpec = (JSONParseSpec) parseSpec;
+ return new JsonInputFormat(jsonParseSpec.getFlattenSpec(), jsonParseSpec.getFeatureSpec(), jsonParseSpec.getKeepNullColumns());
+ } else if (parseSpec instanceof CSVParseSpec) {
+ CSVParseSpec csvParseSpec = (CSVParseSpec) parseSpec;
+ boolean getColumnsFromHeader = csvParseSpec.isHasHeaderRow() && csvParseSpec.getSkipHeaderRows() == 0;
+ return new CsvInputFormat(
+ csvParseSpec.getColumns(),
+ csvParseSpec.getListDelimiter(),
+ getColumnsFromHeader ? null : true,
+ getColumnsFromHeader ? true : null,
+ csvParseSpec.getSkipHeaderRows()
+ );
+ } else if (parseSpec instanceof DelimitedParseSpec) {
+ DelimitedParseSpec delimitedParseSpec = (DelimitedParseSpec) parseSpec;
+ boolean getColumnsFromHeader = delimitedParseSpec.isHasHeaderRow() && delimitedParseSpec.getSkipHeaderRows() == 0;
+ return new DelimitedInputFormat(
+ delimitedParseSpec.getColumns(),
+ delimitedParseSpec.getListDelimiter(),
+ delimitedParseSpec.getDelimiter(),
+ getColumnsFromHeader ? null : true,
+ getColumnsFromHeader ? true : null,
+ delimitedParseSpec.getSkipHeaderRows()
+ );
+ } else if (parseSpec instanceof RegexParseSpec) {
+ RegexParseSpec regexParseSpec = (RegexParseSpec) parseSpec;
+ return new RegexInputFormat(
+ regexParseSpec.getPattern(),
+ regexParseSpec.getListDelimiter(),
+ regexParseSpec.getColumns());
+ } else {
+ throw new RE(StringUtils.format("Unsupported ParseSpec format %s", parseSpec.toString()));
+ }
+ }
+
public class TestLocalTaskActionClientFactory implements TaskActionClientFactory
{
@Override
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java
index 499ba22bb19f..7509d55ddbd2 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java
@@ -24,7 +24,10 @@
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
+import org.apache.druid.data.input.Firehose;
+import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.data.input.impl.NoopInputFormat;
import org.apache.druid.data.input.impl.TimestampSpec;
@@ -49,7 +52,6 @@
import org.apache.druid.segment.indexing.RealtimeTuningConfig;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.realtime.FireDepartment;
-import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.hamcrest.CoreMatchers;
@@ -402,7 +404,7 @@ public void testRealtimeIndexTaskSerde() throws Exception
jsonMapper
),
new RealtimeIOConfig(
- new LocalFirehoseFactory(new File("lol"), "rofl", null),
+ new MockFirehoseFactory(),
(schema, config, metrics) -> null
),
@@ -432,6 +434,9 @@ public void testRealtimeIndexTaskSerde() throws Exception
null
);
+ jsonMapper.registerSubtypes(
+ new NamedType(MockFirehoseFactory.class, "mockFirehoseFactory")
+ );
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
@@ -587,4 +592,13 @@ public void testHadoopIndexTaskSerde() throws Exception
Assert.assertEquals("blah", task.getClasspathPrefix());
Assert.assertEquals("blah", task2.getClasspathPrefix());
}
+
+ private static class MockFirehoseFactory implements FirehoseFactory
+ {
+ @Override
+ public Firehose connect(InputRowParser parser, File temporaryDirectory)
+ {
+ return 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 38f70408232d..9c6733d2543e 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
@@ -25,7 +25,6 @@
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.data.input.impl.ParseSpec;
-import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
@@ -59,7 +58,6 @@
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.loading.SegmentLocalCacheLoader;
import org.apache.druid.segment.loading.TombstoneLoadSpec;
-import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
import org.junit.Assert;
@@ -243,22 +241,20 @@ protected ParallelIndexSupervisorTask createTask(
} else {
Preconditions.checkArgument(inputFormat == null);
ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
- new LocalFirehoseFactory(inputDirectory, filter, null),
+ null,
+ new LocalInputSource(inputDirectory, filter),
+ createInputFormatFromParseSpec(parseSpec),
appendToExisting,
dropExisting
);
- //noinspection unchecked
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
- "dataSource",
- getObjectMapper().convertValue(
- new StringInputRowParser(parseSpec, null),
- Map.class
- ),
+ DATASOURCE,
+ parseSpec.getTimestampSpec(),
+ parseSpec.getDimensionsSpec(),
DEFAULT_METRICS_SPEC,
granularitySpec,
- null,
- getObjectMapper()
+ null
),
ioConfig,
tuningConfig
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
index ecb00917a9b3..94fb3baba092 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
@@ -888,8 +888,10 @@ protected void compareTaskReports(
Map expectedPayload = (Map) expectedReports.get("payload");
Map actualPayload = (Map) actualReports.get("payload");
Assert.assertEquals(expectedPayload.get("ingestionState"), actualPayload.get("ingestionState"));
- Assert.assertEquals(expectedPayload.get("rowStats"), actualPayload.get("rowStats"));
- Assert.assertEquals(expectedPayload.get("ingestionState"), actualPayload.get("ingestionState"));
+
+ Map expectedTotals = (Map) expectedPayload.get("totals");
+ Map actualTotals = (Map) actualReports.get("totals");
+ Assert.assertEquals(expectedTotals, actualTotals);
List expectedParseExceptionReports =
(List) ((Map)
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java
index 536844f226ae..7dc464093ee2 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java
@@ -47,7 +47,6 @@
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
-import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
@@ -267,8 +266,11 @@ private ParallelIndexSupervisorTask createTestTask(
} else {
Preconditions.checkArgument(inputFormat == null);
ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
- new LocalFirehoseFactory(inputDir, filter, null),
- appendToExisting
+ null,
+ new LocalInputSource(inputDir, filter),
+ createInputFormatFromParseSpec(parseSpec),
+ appendToExisting,
+ null
);
//noinspection unchecked
ingestionSpec = new ParallelIndexIngestionSpec(
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java
index 7d8d681c4ab3..6c83c225fa9f 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java
@@ -20,7 +20,6 @@
package org.apache.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.jsontype.NamedType;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.DimensionsSpec;
@@ -38,7 +37,6 @@
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
-import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.hamcrest.CoreMatchers;
import org.joda.time.Interval;
import org.junit.Assert;
@@ -67,9 +65,6 @@ private static ObjectMapper createObjectMapper()
{
TestUtils testUtils = new TestUtils();
ObjectMapper objectMapper = testUtils.getTestObjectMapper();
- objectMapper.registerSubtypes(
- new NamedType(LocalFirehoseFactory.class, "local")
- );
return objectMapper;
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java
index 031dd68d4cd0..0998258d3a18 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java
@@ -27,7 +27,6 @@
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.LocalInputSource;
-import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskToolbox;
@@ -47,7 +46,6 @@
import org.apache.druid.segment.incremental.RowIngestionMetersTotals;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
-import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.Partitions;
import org.apache.druid.timeline.VersionedIntervalTimeline;
@@ -946,27 +944,22 @@ private ParallelIndexSupervisorTask newTask(
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
- getObjectMapper().convertValue(
- new StringInputRowParser(
- DEFAULT_PARSE_SPEC,
- null
- ),
- Map.class
- ),
- new AggregatorFactory[]{
- new LongSumAggregatorFactory("val", "val")
- },
+ DEFAULT_TIMESTAMP_SPEC,
+ DEFAULT_DIMENSIONS_SPEC,
+ DEFAULT_METRICS_SPEC,
new UniformGranularitySpec(
segmentGranularity,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
),
- null,
- getObjectMapper()
+ null
),
new ParallelIndexIOConfig(
- new LocalFirehoseFactory(inputDir, inputSourceFilter, null),
- appendToExisting
+ null,
+ new LocalInputSource(inputDir, inputSourceFilter),
+ createInputFormatFromParseSpec(DEFAULT_PARSE_SPEC),
+ appendToExisting,
+ null
),
tuningConfig
);
@@ -984,10 +977,10 @@ private ParallelIndexSupervisorTask newTask(
private String getErrorMessageForUnparseableTimestamp()
{
- return useInputFormatApi ? StringUtils.format(
+ return StringUtils.format(
"Timestamp[2017unparseable] is unparseable! Event: {ts=2017unparseable} (Path: %s, Record: 5, Line: 5)",
new File(inputDir, "test_0").toURI()
- ) : "Timestamp[2017unparseable] is unparseable! Event: {ts=2017unparseable}";
+ );
}
private static class SettableSplittableLocalInputSource extends LocalInputSource
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
deleted file mode 100644
index 591a3adb84cd..000000000000
--- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
+++ /dev/null
@@ -1,615 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.indexing.firehose;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair;
-import com.fasterxml.jackson.databind.module.SimpleModule;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.inject.Binder;
-import com.google.inject.Module;
-import org.apache.druid.client.coordinator.CoordinatorClient;
-import org.apache.druid.data.input.Firehose;
-import org.apache.druid.data.input.FirehoseFactory;
-import org.apache.druid.data.input.InputRow;
-import org.apache.druid.data.input.impl.DimensionsSpec;
-import org.apache.druid.data.input.impl.InputRowParser;
-import org.apache.druid.data.input.impl.JSONParseSpec;
-import org.apache.druid.data.input.impl.MapInputRowParser;
-import org.apache.druid.data.input.impl.TimeAndDimsParseSpec;
-import org.apache.druid.data.input.impl.TimestampSpec;
-import org.apache.druid.guice.GuiceAnnotationIntrospector;
-import org.apache.druid.guice.GuiceInjectableValues;
-import org.apache.druid.guice.GuiceInjectors;
-import org.apache.druid.indexing.common.ReingestionTimelineUtils;
-import org.apache.druid.indexing.common.RetryPolicyConfig;
-import org.apache.druid.indexing.common.RetryPolicyFactory;
-import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
-import org.apache.druid.indexing.common.TestUtils;
-import org.apache.druid.indexing.common.config.TaskStorageConfig;
-import org.apache.druid.indexing.common.task.NoopTask;
-import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage;
-import org.apache.druid.indexing.overlord.Segments;
-import org.apache.druid.indexing.overlord.TaskLockbox;
-import org.apache.druid.indexing.overlord.TaskStorage;
-import org.apache.druid.java.util.common.FileUtils;
-import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.java.util.common.JodaUtils;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.java.util.emitter.service.ServiceEmitter;
-import org.apache.druid.math.expr.ExprMacroTable;
-import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
-import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
-import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
-import org.apache.druid.query.expression.TestExprMacroTable;
-import org.apache.druid.query.filter.SelectorDimFilter;
-import org.apache.druid.segment.IndexIO;
-import org.apache.druid.segment.IndexMergerV9;
-import org.apache.druid.segment.IndexSpec;
-import org.apache.druid.segment.TestHelper;
-import org.apache.druid.segment.column.ColumnHolder;
-import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory;
-import org.apache.druid.segment.incremental.IncrementalIndex;
-import org.apache.druid.segment.incremental.IncrementalIndexSchema;
-import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
-import org.apache.druid.segment.loading.LocalDataSegmentPuller;
-import org.apache.druid.segment.loading.LocalLoadSpec;
-import org.apache.druid.segment.realtime.firehose.CombiningFirehoseFactory;
-import org.apache.druid.segment.transform.ExpressionTransform;
-import org.apache.druid.segment.transform.TransformSpec;
-import org.apache.druid.server.metrics.NoopServiceEmitter;
-import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.TimelineObjectHolder;
-import org.apache.druid.timeline.partition.NumberedPartitionChunk;
-import org.apache.druid.timeline.partition.NumberedShardSpec;
-import org.apache.druid.timeline.partition.PartitionChunk;
-import org.apache.druid.timeline.partition.PartitionHolder;
-import org.easymock.EasyMock;
-import org.joda.time.Interval;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-/**
- *
- */
-@RunWith(Parameterized.class)
-public class IngestSegmentFirehoseFactoryTest
-{
- private static final ObjectMapper MAPPER;
- private static final IndexMergerV9 INDEX_MERGER_V9;
- private static final IndexIO INDEX_IO;
- private static final TaskStorage TASK_STORAGE;
- private static final IndexerSQLMetadataStorageCoordinator MDC;
- private static final TaskLockbox TASK_LOCKBOX;
- private static final Task TASK;
-
- @Rule
- public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
- static {
- TestUtils testUtils = new TestUtils();
- MAPPER = setupInjectablesInObjectMapper(TestHelper.makeJsonMapper());
- INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9();
- INDEX_IO = testUtils.getTestIndexIO();
- TASK_STORAGE = new HeapMemoryTaskStorage(
- new TaskStorageConfig(null)
- {
- }
- );
- MDC = new IndexerSQLMetadataStorageCoordinator(null, null, null)
- {
- private final Set published = new HashSet<>();
-
- @Override
- public List retrieveUsedSegmentsForIntervals(
- String dataSource,
- List interval,
- Segments visibility
- )
- {
- return ImmutableList.copyOf(SEGMENT_SET);
- }
-
- @Override
- public List retrieveUnusedSegmentsForInterval(String dataSource, Interval interval)
- {
- return ImmutableList.of();
- }
-
- @Override
- public Set announceHistoricalSegments(Set segments)
- {
- Set added = new HashSet<>();
- for (final DataSegment segment : segments) {
- if (published.add(segment)) {
- added.add(segment);
- }
- }
-
- return ImmutableSet.copyOf(added);
- }
-
- @Override
- public void deleteSegments(Set segments)
- {
- // do nothing
- }
- };
- TASK_LOCKBOX = new TaskLockbox(TASK_STORAGE, MDC);
- TASK = NoopTask.create();
- TASK_LOCKBOX.add(TASK);
- }
-
- @Parameterized.Parameters(name = "{0}")
- public static Collection