From d203cac0e6c79617413d3f3a610d1ca62fc3005d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 19 Apr 2017 16:28:12 +0900 Subject: [PATCH 01/26] Add PrefetcheableTextFilesFirehoseFactory --- .../AbstractTextFilesFirehoseFactory.java | 151 +++++++ .../input/impl/FileIteratingFirehose.java | 20 + ...PrefetcheableTextFilesFirehoseFactory.java | 368 +++++++++++++++++ ...etcheableTextFilesFirehoseFactoryTest.java | 369 ++++++++++++++++++ .../io/druid/firehose/azure/AzureBlob.java | 9 + .../StaticAzureBlobStoreFirehoseFactory.java | 107 ++--- .../firehose/cloudfiles/CloudFilesBlob.java | 10 + .../StaticCloudFilesFirehoseFactory.java | 106 ++--- .../io/druid/firehose/google/GoogleBlob.java | 9 + .../StaticGoogleBlobStoreFirehoseFactory.java | 101 ++--- .../firehose/s3/StaticS3FirehoseFactory.java | 123 ++---- .../s3/StaticS3FirehoseFactoryTest.java | 6 +- .../druid/indexing/common/task/IndexTask.java | 27 +- .../indexing/common/task/IndexTaskTest.java | 2 +- .../indexing/common/task/TaskSerdeTest.java | 10 +- .../overlord/RemoteTaskRunnerTest.java | 7 + .../indexing/overlord/TaskLifecycleTest.java | 6 +- .../indexing/worker/TaskAnnouncementTest.java | 4 + .../firehose/LocalFirehoseFactory.java | 81 +--- .../firehose/LocalFirehoseFactoryTest.java | 123 ++++++ .../cli/validate/DruidJsonValidatorTest.java | 4 + 21 files changed, 1229 insertions(+), 414 deletions(-) create mode 100644 api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java create mode 100644 api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java create mode 100644 api/src/test/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactoryTest.java create mode 100644 server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java new file mode 100644 index 000000000000..54bdcceee051 --- /dev/null +++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java @@ -0,0 +1,151 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import io.druid.data.input.Firehose; +import io.druid.data.input.FirehoseFactory; +import io.druid.java.util.common.CompressionUtils; +import io.druid.java.util.common.logger.Logger; +import org.apache.commons.io.Charsets; +import org.apache.commons.io.IOUtils; +import org.apache.commons.io.LineIterator; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * This is an abstract class for firehose factory for making firehoses reading text files. + * It provides an unified {@link #connect(StringInputRowParser)} implementation for its subclasses. + * + * @param object type representing input data + */ +public abstract class AbstractTextFilesFirehoseFactory + implements FirehoseFactory +{ + private static final Logger LOG = new Logger(AbstractTextFilesFirehoseFactory.class); + + private final List objects; + + public AbstractTextFilesFirehoseFactory(Collection objects) + { + this.objects = ImmutableList.copyOf(Preconditions.checkNotNull(objects)); + } + + @Override + public Firehose connect(StringInputRowParser firehoseParser) throws IOException + { + final Iterator iterator = objects.iterator(); + return new FileIteratingFirehose( + new Iterator() + { + @Override + public boolean hasNext() + { + return iterator.hasNext(); + } + + @Override + public LineIterator next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final ObjectType object = iterator.next(); + try { + final InputStream stream = openStream(object); + return IOUtils.lineIterator( + new BufferedReader( + new InputStreamReader(wrapIfNeeded(object, stream), Charsets.UTF_8) + ) + ); + } + catch (Exception e) { + LOG.error( + e, + "Exception reading object[%s]", + object + ); + throw Throwables.propagate(e); + } + } + }, + firehoseParser + ); + } + + public List getObjects() + { + return objects; + } + + /** + * Wrap the given stream if needed, currently when the given object is compressed with gzip. + * + * @param object an object + * @param innerStream input stream for object + * + * @return wrapped input stream if the object is gzipped + * + * @throws IOException + */ + protected InputStream wrapIfNeeded(ObjectType object, InputStream innerStream) throws IOException + { + return wrapIfNeeded(innerStream, isGzipped(object)); + } + + protected static InputStream wrapIfNeeded(InputStream innerStream, boolean isGzipped) throws IOException + { + if (isGzipped) { + return CompressionUtils.gzipInputStream(innerStream); + } + + return innerStream; + } + + /** + * Open an input stream from the given object. + * The result input stream must not be wrapped even if the object is compressed with gzip. + * + * @param object an object to be read + * + * @return an input stream for the object + * + * @throws IOException + */ + protected abstract InputStream openStream(ObjectType object) throws IOException; + + /** + * Check if the object is compressed with gzip. + * + * @param object an object to be checked + * + * @return true if the object is compressed with gzip + */ + protected abstract boolean isGzipped(ObjectType object); +} diff --git a/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java b/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java index 97e33f04a894..e4d313e9ee72 100644 --- a/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java +++ b/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java @@ -25,6 +25,7 @@ import io.druid.utils.Runnables; import org.apache.commons.io.LineIterator; +import java.io.Closeable; import java.io.IOException; import java.util.Iterator; @@ -37,19 +38,34 @@ public class FileIteratingFirehose implements Firehose private LineIterator lineIterator = null; + private final Closeable closer; + public FileIteratingFirehose( Iterator lineIterators, StringInputRowParser parser ) + { + this(lineIterators, parser, null); + } + + public FileIteratingFirehose( + Iterator lineIterators, + StringInputRowParser parser, + Closeable closer + ) { this.lineIterators = lineIterators; this.parser = parser; + this.closer = closer; } @Override public boolean hasMore() { while ((lineIterator == null || !lineIterator.hasNext()) && lineIterators.hasNext()) { + if (lineIterator != null) { + lineIterator.close(); + } lineIterator = lineIterators.next(); } @@ -88,5 +104,9 @@ public void close() throws IOException if (lineIterator != null) { lineIterator.close(); } + + if (closer != null) { + closer.close(); + } } } diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java new file mode 100644 index 000000000000..d2031c82581d --- /dev/null +++ b/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java @@ -0,0 +1,368 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Throwables; +import com.google.common.io.CountingOutputStream; +import com.google.common.io.Files; +import io.druid.data.input.Firehose; +import io.druid.java.util.common.logger.Logger; +import org.apache.commons.io.Charsets; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.commons.io.LineIterator; + +import java.io.BufferedReader; +import java.io.Closeable; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.Reader; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; + +/** + * PrefetcheableTextFilesFirehoseFactory is an abstract firehose factory for reading text files. The firehose returned + * by this class provides two key functionalities. + * + *
    + *
  • Caching: for the first call of {@link #connect(StringInputRowParser)}, it caches objects in a local disk + * up to {@link #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 cached or + * fetched data is smaller than {@link #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 #maxFetchRetry}.
  • + *
+ * + * This implementation can be useful when the cost for reading input objects is large as reading from AWS S3 because + * IndexTask can read the whole data twice for determining partition specs and generating segments if the intervals of + * GranularitySpec is not specified. + */ +public abstract class PrefetcheableTextFilesFirehoseFactory + extends AbstractTextFilesFirehoseFactory +{ + private static final Logger LOG = new Logger(PrefetcheableTextFilesFirehoseFactory.class); + private static final long DEFAULT_MAX_CACHE_CAPACITY_BYTES = 1024 * 1024 * 1024; // 1GB + private static final long DEFAULT_MAX_FETCH_CAPACITY_BYTES = 1024 * 1024 * 1024; // 1GB + private static final int DEFAULT_MAX_FETCH_RETRY = 3; + private static final int DEFAULT_FETCH_TIMEOUT = 60_000; // 60 secs + + // The below two variables are roughly the max size of total cached/fetched objects, but the actual cached/fetched + // size can be larger. The reason is our current client implementations for cloud storages like s3 don't support range + // scan yet, so we must download the whole file at once. It's still possible for the size of cached/fetched data to + // not exceed these variables by estimating the after-fetch size, but it makes us consider the case when any files + // cannot be fetched due to their large size, which makes the implementation complicated. + private long maxCacheCapacityBytes; + private final long maxFetchCapacityBytes; + + private final long prefetchTriggerBytes; + private File baseDir; // Directory for cached and fetched files + + private final List cacheFiles; + private final LinkedBlockingQueue fetchFiles; + + // Number of bytes currently fetched files. + // This is updated when fetch a file is successfully fetched or a fetched file is deleted. + private final AtomicLong fetchedBytes = new AtomicLong(0); + + // timeout for fetching an object from the remote site + private final int fetchTimeout; + + // maximum retry for fetching an object from the remote site + private final int maxFetchRetry; + + private volatile int nextFetchIndex; + private volatile boolean beingFetched; + private volatile Exception fetchException; + + public PrefetcheableTextFilesFirehoseFactory( + Collection objects, + Long maxCacheCapacityBytes, + Long maxFetchCapacityBytes, + Long prefetchTriggerBytes, + Integer fetchTimeout, + Integer maxFetchRetry + ) + { + super(objects); + this.maxCacheCapacityBytes = maxCacheCapacityBytes == null + ? DEFAULT_MAX_CACHE_CAPACITY_BYTES + : maxCacheCapacityBytes; + this.maxFetchCapacityBytes = maxFetchCapacityBytes == null + ? DEFAULT_MAX_FETCH_CAPACITY_BYTES + : maxFetchCapacityBytes; + this.prefetchTriggerBytes = prefetchTriggerBytes == null + ? this.maxFetchCapacityBytes / 2 + : prefetchTriggerBytes; + this.fetchTimeout = fetchTimeout == null ? DEFAULT_FETCH_TIMEOUT : fetchTimeout; + this.maxFetchRetry = maxFetchRetry == null ? DEFAULT_MAX_FETCH_RETRY : maxFetchRetry; + + cacheFiles = new ArrayList<>(); + fetchFiles = new LinkedBlockingQueue<>(); + } + + @VisibleForTesting + List getCacheFiles() + { + return cacheFiles; + } + + /** + * Cache objects in a local disk up to {@link #maxCacheCapacityBytes}. + */ + private void cache() + { + double totalFetchedBytes = 0; + final List objects = getObjects(); + try { + for (int i = 0; i < objects.size() && totalFetchedBytes < maxCacheCapacityBytes; i++) { + final ObjectType object = objects.get(i); + LOG.info("Caching object[%s]", object); + final File outFile = File.createTempFile("cache-", null, baseDir); + totalFetchedBytes += download(object, outFile, 0); + cacheFiles.add(new FetchedFile(outFile, isGzipped(object))); + nextFetchIndex++; + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + /** + * Fetch objects to a local disk up to {@link #maxFetchCapacityBytes}. This method is not thread safe and must be + * called by a single thread. Note that even {@link #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. + */ + private void fetch() + { + if (!beingFetched) { + beingFetched = true; + final List objects = getObjects(); + try { + for (int i = nextFetchIndex; i < objects.size() && fetchedBytes.get() <= maxFetchCapacityBytes; i++) { + final ObjectType object = objects.get(i); + LOG.info("Fetching object[%s]", object); + final File outFile = File.createTempFile("fetch-", null, baseDir); + fetchedBytes.addAndGet(download(object, outFile, 0)); + fetchFiles.put(new FetchedFile(outFile, isGzipped(object))); + nextFetchIndex++; + } + } + catch (Exception e) { + fetchException = e; + throw Throwables.propagate(e); + } + finally { + beingFetched = false; + } + } + } + + /** + * Downloads an object. It retires downloading {@link #maxFetchRetry} times and throws that exception. + * + * @param object an object to be downloaded + * @param outFile a file which the object data is stored + * @param retry current retry count + * + * @return number of downloaded bytes + * + * @throws IOException + */ + private long download(ObjectType object, File outFile, int retry) throws IOException + { + try (final InputStream is = openStream(object); + final CountingOutputStream cos = new CountingOutputStream(new FileOutputStream(outFile))) { + IOUtils.copy(is, cos); + return cos.getCount(); + } + catch (IOException e) { + if (retry < maxFetchRetry) { + LOG.error(e, "Failed to download object[%s], retrying (%d of %d)", object, retry + 1, maxFetchRetry); + outFile.delete(); + return download(object, outFile, retry + 1); + } else { + LOG.error(e, "Failed to download object[%s], retries exhausted, aborting", object); + throw e; + } + } + } + + @Override + public Firehose connect(StringInputRowParser firehoseParser) throws IOException + { + if (baseDir == null) { + baseDir = Files.createTempDir(); + baseDir.deleteOnExit(); + cache(); + } else { + nextFetchIndex = cacheFiles.size(); + } + + // fetchExecutor is responsible for background data fetching + final ExecutorService fetchExecutor = Executors.newSingleThreadExecutor(); + + return new FileIteratingFirehose( + new Iterator() + { + final Iterator cacheFileIterator = cacheFiles.iterator(); + long remainingCachedBytes = cacheFiles.stream() + .mapToLong(fetchedFile -> fetchedFile.file.length()) + .sum(); + + { + fetchIfNeeded(remainingCachedBytes); + } + + @Override + public boolean hasNext() + { + return cacheFileIterator.hasNext() + || !fetchFiles.isEmpty() + || nextFetchIndex < getObjects().size(); + } + + private void fetchIfNeeded(long remainingBytes) + { + if (!beingFetched && remainingBytes <= prefetchTriggerBytes) { + fetchExecutor.submit(() -> fetch()); + } + } + + @Override + public LineIterator next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + if (fetchException != null) { + throw Throwables.propagate(fetchException); + } + + final FetchedFile fetchedFile; + final Closeable closeable; + // Check cache first + if (cacheFileIterator.hasNext()) { + fetchedFile = cacheFileIterator.next(); + remainingCachedBytes -= fetchedFile.file.length(); + fetchIfNeeded(remainingCachedBytes); + closeable = () -> { + }; + } else { + if (!fetchFiles.isEmpty()) { + // If there are already fetched files, use them + fetchedFile = fetchFiles.poll(); + fetchIfNeeded(fetchedBytes.get()); + } else { + // Otherwise, wait for fetching + try { + fetchIfNeeded(fetchedBytes.get()); + fetchedFile = fetchFiles.poll(fetchTimeout, TimeUnit.MILLISECONDS); + if (fetchedFile == null) { + throw new RuntimeException(new TimeoutException()); + } + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + } + closeable = () -> { + final long fileSize = fetchedFile.file.length(); + fetchedFile.file.delete(); + fetchedBytes.addAndGet(-fileSize); + }; + } + + try { + final InputStream stream = FileUtils.openInputStream(fetchedFile.file); + + return new ResourceCloseableLineIterator( + new BufferedReader( + new InputStreamReader(wrapIfNeeded(stream, fetchedFile.isGzipped), Charsets.UTF_8) + ), + closeable + ); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + }, + firehoseParser, + fetchExecutor::shutdown + ); + } + + static class FetchedFile + { + private final File file; + private final boolean isGzipped; + + FetchedFile(File file, boolean isGzipped) + { + this.file = file; + this.isGzipped = isGzipped; + } + } + + /** + * This class calls the {@link Closeable#close()} method of the resourceCloser when it is closed. + */ + static class ResourceCloseableLineIterator extends LineIterator + { + private final Closeable resourceCloser; + + public ResourceCloseableLineIterator(Reader reader, Closeable resourceCloser) throws IllegalArgumentException + { + super(reader); + this.resourceCloser = resourceCloser; + } + + @Override + public void close() + { + super.close(); + try { + resourceCloser.close(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + } +} diff --git a/api/src/test/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactoryTest.java new file mode 100644 index 000000000000..2a5e86713836 --- /dev/null +++ b/api/src/test/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactoryTest.java @@ -0,0 +1,369 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl; + +import com.google.common.base.Charsets; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import io.druid.data.input.Firehose; +import io.druid.data.input.Row; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.filefilter.TrueFileFilter; +import org.hamcrest.CoreMatchers; +import org.joda.time.DateTime; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.InputStream; +import java.io.Writer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeoutException; + +public class PrefetcheableTextFilesFirehoseFactoryTest +{ + private static File testDir; + + private final StringInputRowParser parser = new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec( + "timestamp", + "auto", + null + ), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b")), + Lists.newArrayList(), + Lists.newArrayList() + ), + ",", + Arrays.asList("timestamp", "a", "b") + ), + Charsets.UTF_8.name() + ); + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @BeforeClass + public static void setup() throws IOException + { + testDir = File.createTempFile(PrefetcheableTextFilesFirehoseFactoryTest.class.getSimpleName(), ""); + FileUtils.forceDelete(testDir); + FileUtils.forceMkdir(testDir); + + for (int i = 0; i < 10; i++) { + // Each file is 1390 bytes + try (final Writer writer = new BufferedWriter( + new FileWriter(new File(testDir, "test_" + i)) + )) { + for (int j = 0; j < 100; j++) { + final String a = (20171220 + i) + "," + i + "," + j + "\n"; + writer.write(a); + } + } + } + } + + @AfterClass + public static void teardown() throws IOException + { + FileUtils.forceDelete(testDir); + } + + private static void assertResult(List rows) + { + Assert.assertEquals(1000, 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 < 10; i++) { + for (int j = 0; j < 100; j++) { + final Row row = rows.get(i * 100 + j); + Assert.assertEquals(new DateTime(20171220 + i), row.getTimestamp()); + Assert.assertEquals(String.valueOf(i), row.getDimension("a").get(0)); + Assert.assertEquals(String.valueOf(j), row.getDimension("b").get(0)); + } + } + } + + @Test + public void testWithoutCache() throws IOException + { + final TestPrefetcheableTextFilesFirehoseFactory factory = + TestPrefetcheableTextFilesFirehoseFactory.withoutCache(testDir); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser)) { + Assert.assertEquals(0, factory.getCacheFiles().size()); + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + + @Test + public void testWithZeroFetchCapacity() throws IOException + { + final TestPrefetcheableTextFilesFirehoseFactory factory = + TestPrefetcheableTextFilesFirehoseFactory.withZeroFetchCapacity(testDir); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + + @Test + public void testWithCacheAndFetch() throws IOException + { + final TestPrefetcheableTextFilesFirehoseFactory factory = + TestPrefetcheableTextFilesFirehoseFactory.of(testDir); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + + @Test + public void testRetry() throws IOException + { + final TestPrefetcheableTextFilesFirehoseFactory factory = + TestPrefetcheableTextFilesFirehoseFactory.withOpenExceptions(testDir, 1); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + + @Test + public void testMaxRetry() throws IOException + { + expectedException.expect(RuntimeException.class); + expectedException.expectCause(CoreMatchers.instanceOf(IOException.class)); + expectedException.expectMessage("Exception for retry test"); + + final TestPrefetcheableTextFilesFirehoseFactory factory = + TestPrefetcheableTextFilesFirehoseFactory.withOpenExceptions(testDir, 5); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + } + + @Test + public void testTimeout() throws IOException + { + expectedException.expect(RuntimeException.class); + expectedException.expectCause(CoreMatchers.instanceOf(TimeoutException.class)); + + final TestPrefetcheableTextFilesFirehoseFactory factory = + TestPrefetcheableTextFilesFirehoseFactory.withSleepMillis(testDir, 1000); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + } + + @Test + public void testReconnect() throws IOException + { + final TestPrefetcheableTextFilesFirehoseFactory factory = + TestPrefetcheableTextFilesFirehoseFactory.of(testDir); + + for (int i = 0; i < 5; i++) { + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + assertResult(rows); + } + } + + static class TestPrefetcheableTextFilesFirehoseFactory extends PrefetcheableTextFilesFirehoseFactory + { + private long sleepMillis; + private int openExceptionCount; + + static TestPrefetcheableTextFilesFirehoseFactory withoutCache(File baseDir) + { + return new TestPrefetcheableTextFilesFirehoseFactory( + baseDir, + 1024, + 0, + 2048, + 1000, + 3, + 0, + 0 + ); + } + + static TestPrefetcheableTextFilesFirehoseFactory withZeroFetchCapacity(File baseDir) + { + return new TestPrefetcheableTextFilesFirehoseFactory( + baseDir, + 1024, + 2048, + 0, + 1000, + 3, + 0, + 0 + ); + } + + static TestPrefetcheableTextFilesFirehoseFactory of(File baseDir) + { + return new TestPrefetcheableTextFilesFirehoseFactory( + baseDir, + 1024, + 2048, + 2048, + 1000, + 3, + 0, + 0 + ); + } + + static TestPrefetcheableTextFilesFirehoseFactory withOpenExceptions(File baseDir, int count) + { + return new TestPrefetcheableTextFilesFirehoseFactory( + baseDir, + 1024, + 2048, + 2048, + 1000, + 3, + count, + 0 + ); + } + + static TestPrefetcheableTextFilesFirehoseFactory withSleepMillis(File baseDir, long ms) + { + return new TestPrefetcheableTextFilesFirehoseFactory( + baseDir, + 1024, + 2048, + 2048, + 100, + 3, + 0, + ms + ); + } + + public TestPrefetcheableTextFilesFirehoseFactory( + File baseDir, + long prefetchTriggerThreshold, + long maxCacheCapacityBytes, + long maxFetchCapacityBytes, + int timeout, + int maxRetry, + int openExceptionCount, + long sleepMillis + ) + { + super( + FileUtils.listFiles( + Preconditions.checkNotNull(baseDir).getAbsoluteFile(), + TrueFileFilter.INSTANCE, + TrueFileFilter.INSTANCE + ), + maxCacheCapacityBytes, + maxFetchCapacityBytes, + prefetchTriggerThreshold, + timeout, + maxRetry + ); + this.openExceptionCount = openExceptionCount; + this.sleepMillis = sleepMillis; + } + + @Override + protected InputStream openStream(File object) throws IOException + { + if (openExceptionCount > 0) { + openExceptionCount--; + throw new IOException("Exception for retry test"); + } + if (sleepMillis > 0) { + try { + Thread.sleep(sleepMillis); + } + catch (InterruptedException e) { + throw new IOException(e); + } + } + return FileUtils.openInputStream(object); + } + + @Override + protected boolean isGzipped(File object) + { + return object.getPath().endsWith(".gz"); + } + } +} diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/AzureBlob.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/AzureBlob.java index 2e41238fd1b1..98cd1a8ec4be 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/AzureBlob.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/AzureBlob.java @@ -48,4 +48,13 @@ public String getContainer() { public String getPath() { return path; } + + @Override + public String toString() + { + return "AzureBlob{" + + "container=" + container + + ",path=" + path + + "}"; + } } diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java index 233974cf22f8..c995ed00366b 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java @@ -22,106 +22,61 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.impl.FileIteratingFirehose; -import io.druid.data.input.impl.StringInputRowParser; -import io.druid.java.util.common.CompressionUtils; -import io.druid.java.util.common.logger.Logger; +import io.druid.data.input.impl.PrefetcheableTextFilesFirehoseFactory; import io.druid.storage.azure.AzureByteSource; import io.druid.storage.azure.AzureStorage; -import org.apache.commons.io.IOUtils; -import org.apache.commons.io.LineIterator; -import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; -import java.util.Iterator; -import java.util.LinkedList; import java.util.List; /** * This class is heavily inspired by the StaticS3FirehoseFactory class in the io.druid.firehose.s3 package */ -public class StaticAzureBlobStoreFirehoseFactory implements FirehoseFactory { - private static final Logger log = new Logger(StaticAzureBlobStoreFirehoseFactory.class); - +public class StaticAzureBlobStoreFirehoseFactory extends PrefetcheableTextFilesFirehoseFactory +{ private final AzureStorage azureStorage; - private final List blobs; @JsonCreator public StaticAzureBlobStoreFirehoseFactory( @JacksonInject("azureStorage") AzureStorage azureStorage, - @JsonProperty("blobs") AzureBlob[] blobs - ) { + @JsonProperty("blobs") List blobs, + @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, + @JsonProperty("maxCacheCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("maxCacheCapacityBytes") Long prefetchTriggerBytes, + @JsonProperty("maxCacheCapacityBytes") Integer fetchTimeout, + @JsonProperty("maxCacheCapacityBytes") Integer maxFetchRetry + ) + { + super(blobs, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); this.azureStorage = azureStorage; - this.blobs = ImmutableList.copyOf(blobs); } @JsonProperty - public List getBlobs() { - return blobs; + public List getBlobs() + { + return getObjects(); } @Override - public Firehose connect(StringInputRowParser stringInputRowParser) throws IOException { - Preconditions.checkNotNull(azureStorage, "null azureStorage"); - - final LinkedList objectQueue = Lists.newLinkedList(blobs); - - return new FileIteratingFirehose( - new Iterator() { - @Override - public boolean hasNext() { - return !objectQueue.isEmpty(); - } - - @Override - public LineIterator next() { - final AzureBlob nextURI = objectQueue.poll(); - - final String container = nextURI.getContainer(); - final String path = nextURI.getPath().startsWith("/") - ? nextURI.getPath().substring(1) - : nextURI.getPath(); - - try { - final InputStream innerInputStream = new AzureByteSource(azureStorage, container, path).openStream(); - - - final InputStream outerInputStream = path.endsWith(".gz") - ? CompressionUtils.gzipInputStream(innerInputStream) - : innerInputStream; + protected InputStream openStream(AzureBlob object) throws IOException + { + return makeByteSource(azureStorage, object).openStream(); + } - return IOUtils.lineIterator( - new BufferedReader( - new InputStreamReader(outerInputStream, Charsets.UTF_8) - ) - ); - } catch (Exception e) { - log.error(e, - "Exception opening container[%s] blob[%s]", - container, - path - ); + @Override + protected boolean isGzipped(AzureBlob object) + { + return object.getPath().endsWith(".gz"); + } - throw Throwables.propagate(e); - } - } + private static AzureByteSource makeByteSource(AzureStorage azureStorage, AzureBlob object) + { + final String container = object.getContainer(); + final String path = object.getPath().startsWith("/") + ? object.getPath().substring(1) + : object.getPath(); - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }, - stringInputRowParser - ); + return new AzureByteSource(azureStorage, container, path); } } diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/CloudFilesBlob.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/CloudFilesBlob.java index fc25ba9b6767..fb013ed55a74 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/CloudFilesBlob.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/CloudFilesBlob.java @@ -62,4 +62,14 @@ public String getRegion() { return region; } + + @Override + public String toString() + { + return "CloudFilesBlob{" + + "container=" + container + + ",path=" + path + + ",region=" + region + + "}"; + } } diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java index c3f5d23ab999..cf3528734c61 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java @@ -22,118 +22,64 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.impl.FileIteratingFirehose; -import io.druid.data.input.impl.StringInputRowParser; -import io.druid.java.util.common.CompressionUtils; +import io.druid.data.input.impl.PrefetcheableTextFilesFirehoseFactory; import io.druid.java.util.common.logger.Logger; -import io.druid.java.util.common.parsers.ParseException; import io.druid.storage.cloudfiles.CloudFilesByteSource; import io.druid.storage.cloudfiles.CloudFilesObjectApiProxy; -import org.apache.commons.io.IOUtils; -import org.apache.commons.io.LineIterator; import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi; -import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; -import java.util.Iterator; -import java.util.LinkedList; import java.util.List; -public class StaticCloudFilesFirehoseFactory implements FirehoseFactory +public class StaticCloudFilesFirehoseFactory extends PrefetcheableTextFilesFirehoseFactory { private static final Logger log = new Logger(StaticCloudFilesFirehoseFactory.class); private final CloudFilesApi cloudFilesApi; - private final List blobs; @JsonCreator public StaticCloudFilesFirehoseFactory( @JacksonInject("objectApi") CloudFilesApi cloudFilesApi, - @JsonProperty("blobs") CloudFilesBlob[] blobs + @JsonProperty("blobs") List blobs, + @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, + @JsonProperty("maxCacheCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("maxCacheCapacityBytes") Long prefetchTriggerBytes, + @JsonProperty("maxCacheCapacityBytes") Integer fetchTimeout, + @JsonProperty("maxCacheCapacityBytes") Integer maxFetchRetry ) { + super(blobs, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); this.cloudFilesApi = cloudFilesApi; - this.blobs = ImmutableList.copyOf(blobs); } @JsonProperty public List getBlobs() { - return blobs; + return getObjects(); } @Override - public Firehose connect(StringInputRowParser stringInputRowParser) throws IOException, ParseException + protected InputStream openStream(CloudFilesBlob object) throws IOException { - Preconditions.checkNotNull(cloudFilesApi, "null cloudFilesApi"); - - final LinkedList objectQueue = Lists.newLinkedList(blobs); - - return new FileIteratingFirehose( - new Iterator() - { - - @Override - public boolean hasNext() - { - return !objectQueue.isEmpty(); - } - - @Override - public LineIterator next() - { - final CloudFilesBlob nextURI = objectQueue.poll(); - - final String region = nextURI.getRegion(); - final String container = nextURI.getContainer(); - final String path = nextURI.getPath(); - - log.info("Retrieving file from region[%s], container[%s] and path [%s]", - region, container, path - ); - CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy( - cloudFilesApi, region, container); - final CloudFilesByteSource byteSource = new CloudFilesByteSource(objectApi, path); + final String region = object.getRegion(); + final String container = object.getContainer(); + final String path = object.getPath(); - try { - final InputStream innerInputStream = byteSource.openStream(); - final InputStream outerInputStream = path.endsWith(".gz") - ? CompressionUtils.gzipInputStream(innerInputStream) - : innerInputStream; - - return IOUtils.lineIterator( - new BufferedReader( - new InputStreamReader(outerInputStream, Charsets.UTF_8))); - } - catch (IOException e) { - log.error(e, - "Exception opening container[%s] blob[%s] from region[%s]", - container, path, region - ); - - throw Throwables.propagate(e); - } - } + log.info("Retrieving file from region[%s], container[%s] and path [%s]", + region, container, path + ); + CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy( + cloudFilesApi, region, container); + final CloudFilesByteSource byteSource = new CloudFilesByteSource(objectApi, path); - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } + return byteSource.openStream(); + } - }, - stringInputRowParser - ); + @Override + protected boolean isGzipped(CloudFilesBlob object) + { + return object.getPath().endsWith(".gz"); } } diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/GoogleBlob.java b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/GoogleBlob.java index 0b6b6cc6eba0..d0109973d87b 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/GoogleBlob.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/GoogleBlob.java @@ -41,5 +41,14 @@ public String getBucket() { public String getPath() { return path; } + + @Override + public String toString() + { + return "GoogleBlob {" + + "bucket=" + bucket + + ",path=" + path + + "}"; + } } diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java index 907571d58ffa..2c2705c75628 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java @@ -22,102 +22,53 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.impl.FileIteratingFirehose; -import io.druid.data.input.impl.StringInputRowParser; -import io.druid.java.util.common.CompressionUtils; -import io.druid.java.util.common.logger.Logger; +import io.druid.data.input.impl.PrefetcheableTextFilesFirehoseFactory; import io.druid.storage.google.GoogleByteSource; import io.druid.storage.google.GoogleStorage; -import org.apache.commons.io.IOUtils; -import org.apache.commons.io.LineIterator; -import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; -import java.util.Iterator; -import java.util.LinkedList; import java.util.List; -public class StaticGoogleBlobStoreFirehoseFactory implements FirehoseFactory { - private static final Logger LOG = new Logger(StaticGoogleBlobStoreFirehoseFactory.class); - +public class StaticGoogleBlobStoreFirehoseFactory extends PrefetcheableTextFilesFirehoseFactory +{ private final GoogleStorage storage; - private final List blobs; @JsonCreator public StaticGoogleBlobStoreFirehoseFactory( @JacksonInject GoogleStorage storage, - @JsonProperty("blobs") GoogleBlob[] blobs - ) { + @JsonProperty("blobs") List blobs, + @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, + @JsonProperty("maxCacheCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("maxCacheCapacityBytes") Long prefetchTriggerBytes, + @JsonProperty("maxCacheCapacityBytes") Integer fetchTimeout, + @JsonProperty("maxCacheCapacityBytes") Integer maxFetchRetry + ) + { + super(blobs, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); this.storage = storage; - this.blobs = ImmutableList.copyOf(blobs); } @JsonProperty public List getBlobs() { - return blobs; + return getObjects(); } @Override - public Firehose connect(StringInputRowParser stringInputRowParser) throws IOException { - Preconditions.checkNotNull(storage, "null storage"); - - final LinkedList objectQueue = Lists.newLinkedList(blobs); - - return new FileIteratingFirehose( - new Iterator() { - @Override - public boolean hasNext() { - return !objectQueue.isEmpty(); - } - - @Override - public LineIterator next() { - final GoogleBlob nextURI = objectQueue.poll(); - - final String bucket = nextURI.getBucket(); - final String path = nextURI.getPath().startsWith("/") - ? nextURI.getPath().substring(1) - : nextURI.getPath(); - - try { - final InputStream innerInputStream = new GoogleByteSource(storage, bucket, path).openStream(); - - final InputStream outerInputStream = path.endsWith(".gz") - ? CompressionUtils.gzipInputStream(innerInputStream) - : innerInputStream; - - return IOUtils.lineIterator( - new BufferedReader( - new InputStreamReader(outerInputStream, Charsets.UTF_8) - ) - ); - } catch (Exception e) { - LOG.error(e, - "Exception opening bucket[%s] blob[%s]", - bucket, - path - ); - - throw Throwables.propagate(e); - } - } + protected InputStream openStream(GoogleBlob object) throws IOException + { + final String bucket = object.getBucket(); + final String path = object.getPath().startsWith("/") + ? object.getPath().substring(1) + : object.getPath(); + + return new GoogleByteSource(storage, bucket, path).openStream(); + } - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }, - stringInputRowParser - ); + @Override + protected boolean isGzipped(GoogleBlob object) + { + return object.getPath().endsWith(".gz"); } } diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index b2ccd54d73f1..9f3ff350d7b7 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -22,52 +22,41 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.impl.FileIteratingFirehose; -import io.druid.data.input.impl.StringInputRowParser; -import io.druid.java.util.common.CompressionUtils; +import io.druid.data.input.impl.PrefetcheableTextFilesFirehoseFactory; import io.druid.java.util.common.logger.Logger; - -import org.apache.commons.io.IOUtils; -import org.apache.commons.io.LineIterator; +import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; -import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; import java.net.URI; -import java.util.Iterator; -import java.util.LinkedList; import java.util.List; /** * Builds firehoses that read from a predefined list of S3 objects and then dry up. */ -public class StaticS3FirehoseFactory implements FirehoseFactory +public class StaticS3FirehoseFactory extends PrefetcheableTextFilesFirehoseFactory { private static final Logger log = new Logger(StaticS3FirehoseFactory.class); private final RestS3Service s3Client; - private final List uris; @JsonCreator public StaticS3FirehoseFactory( @JacksonInject("s3Client") RestS3Service s3Client, - @JsonProperty("uris") List uris + @JsonProperty("uris") List uris, + @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, + @JsonProperty("maxCacheCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("maxCacheCapacityBytes") Long prefetchTriggerBytes, + @JsonProperty("maxCacheCapacityBytes") Integer fetchTimeout, + @JsonProperty("maxCacheCapacityBytes") Integer maxFetchRetry ) { - this.s3Client = s3Client; - this.uris = ImmutableList.copyOf(uris); + super(uris, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); + this.s3Client = Preconditions.checkNotNull(s3Client, "null s3Client");; for (final URI inputURI : uris) { Preconditions.checkArgument(inputURI.getScheme().equals("s3"), "input uri scheme == s3 (%s)", inputURI); @@ -77,75 +66,33 @@ public StaticS3FirehoseFactory( @JsonProperty public List getUris() { - return uris; + return getObjects(); } @Override - public Firehose connect(StringInputRowParser firehoseParser) throws IOException + protected InputStream openStream(URI object) throws IOException { - Preconditions.checkNotNull(s3Client, "null s3Client"); - - final LinkedList objectQueue = Lists.newLinkedList(uris); - - return new FileIteratingFirehose( - new Iterator() - { - @Override - public boolean hasNext() - { - return !objectQueue.isEmpty(); - } - - @Override - public LineIterator next() - { - final URI nextURI = objectQueue.poll(); - - final String s3Bucket = nextURI.getAuthority(); - final S3Object s3Object = new S3Object( - nextURI.getPath().startsWith("/") - ? nextURI.getPath().substring(1) - : nextURI.getPath() - ); - - log.info("Reading from bucket[%s] object[%s] (%s)", s3Bucket, s3Object.getKey(), nextURI); - - try { - final InputStream innerInputStream = s3Client.getObject( - new S3Bucket(s3Bucket), s3Object.getKey() - ) - .getDataInputStream(); - - final InputStream outerInputStream = s3Object.getKey().endsWith(".gz") - ? CompressionUtils.gzipInputStream(innerInputStream) - : innerInputStream; - - return IOUtils.lineIterator( - new BufferedReader( - new InputStreamReader(outerInputStream, Charsets.UTF_8) - ) - ); - } - catch (Exception e) { - log.error( - e, - "Exception reading from bucket[%s] object[%s]", - s3Bucket, - s3Object.getKey() - ); - - throw Throwables.propagate(e); - } - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }, - firehoseParser + final String s3Bucket = object.getAuthority(); + final S3Object s3Object = new S3Object( + object.getPath().startsWith("/") + ? object.getPath().substring(1) + : object.getPath() ); + + log.info("Reading from bucket[%s] object[%s] (%s)", s3Bucket, s3Object.getKey(), object); + + try { + return s3Client.getObject(new S3Bucket(s3Bucket), s3Object.getKey()).getDataInputStream(); + } + catch (ServiceException e) { + throw new IOException(e); + } + } + + @Override + protected boolean isGzipped(URI object) + { + return object.getPath().endsWith(".gz"); } @Override @@ -160,13 +107,13 @@ public boolean equals(Object o) StaticS3FirehoseFactory factory = (StaticS3FirehoseFactory) o; - return !(uris != null ? !uris.equals(factory.uris) : factory.uris != null); + return getUris().equals(factory.getUris()); } @Override public int hashCode() { - return uris != null ? uris.hashCode() : 0; + return getUris().hashCode(); } } diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java index 25950e334037..49c6938b61e6 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java @@ -23,6 +23,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import io.druid.jackson.DefaultObjectMapper; +import org.easymock.EasyMock; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.junit.Assert; import org.junit.Test; @@ -54,6 +56,8 @@ public void testSerde() throws Exception ); Assert.assertEquals(factory, outputFact); + Assert.assertEquals(uris, outputFact.getUris()); + Assert.assertTrue(uris.stream().allMatch(outputFact::isGzipped)); } // This class is a workaround for the injectable value that StaticS3FirehoseFactory requires @@ -64,7 +68,7 @@ public TestStaticS3FirehoseFactory( @JsonProperty("uris") List uris ) { - super(null, uris); + super(EasyMock.niceMock(RestS3Service.class), uris, null, null, null, null, null); } } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 75a86d8b84bb..790894f151e8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -80,7 +80,6 @@ import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; -import io.druid.segment.realtime.firehose.ReplayableFirehoseFactory; import io.druid.segment.realtime.plumber.Committers; import io.druid.segment.realtime.plumber.NoopSegmentHandoffNotifierFactory; import io.druid.timeline.DataSegment; @@ -175,19 +174,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ((IngestSegmentFirehoseFactory) delegateFirehoseFactory).setTaskToolbox(toolbox); } - final FirehoseFactory firehoseFactory; - if (ingestionSchema.getIOConfig().isSkipFirehoseCaching() - || delegateFirehoseFactory instanceof ReplayableFirehoseFactory) { - firehoseFactory = delegateFirehoseFactory; - } else { - firehoseFactory = new ReplayableFirehoseFactory( - delegateFirehoseFactory, - ingestionSchema.getTuningConfig().isReportParseExceptions(), - null, - null, - smileMapper - ); - } + final FirehoseFactory firehoseFactory = delegateFirehoseFactory; final Map> shardSpecs = determineShardSpecs(toolbox, firehoseFactory); @@ -584,22 +571,18 @@ public IndexTuningConfig getTuningConfig() public static class IndexIOConfig implements IOConfig { private static final boolean DEFAULT_APPEND_TO_EXISTING = false; - private static final boolean DEFAULT_SKIP_FIREHOSE_CACHING = false; private final FirehoseFactory firehoseFactory; private final boolean appendToExisting; - private final boolean skipFirehoseCaching; @JsonCreator public IndexIOConfig( @JsonProperty("firehose") FirehoseFactory firehoseFactory, - @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting, - @JsonProperty("skipFirehoseCaching") @Nullable Boolean skipFirehoseCaching + @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting ) { this.firehoseFactory = firehoseFactory; this.appendToExisting = appendToExisting == null ? DEFAULT_APPEND_TO_EXISTING : appendToExisting; - this.skipFirehoseCaching = skipFirehoseCaching == null ? DEFAULT_SKIP_FIREHOSE_CACHING : skipFirehoseCaching; } @JsonProperty("firehose") @@ -613,12 +596,6 @@ public boolean isAppendToExisting() { return appendToExisting; } - - @JsonProperty("skipFirehoseCaching") - public boolean isSkipFirehoseCaching() - { - return skipFirehoseCaching; - } } @JsonTypeName("index") diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 81e85c70dba1..139625f24705 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -479,7 +479,7 @@ private IndexTask.IndexIngestionSpec createIngestionSpec( baseDir, "druid*", null - ), appendToExisting, null + ), appendToExisting ), new IndexTask.IndexTuningConfig( targetPartitionSize, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index c7ec7b849d31..4fcc4fea3db1 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -77,6 +77,10 @@ public TaskSerdeTest() for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { jsonMapper.registerModule(jacksonModule); } + + final File baseDir = new File("lol"); + baseDir.mkdir(); + baseDir.deleteOnExit(); } @Test @@ -88,7 +92,6 @@ public void testIndexTaskIOConfigDefaults() throws Exception ); Assert.assertEquals(false, ioConfig.isAppendToExisting()); - Assert.assertEquals(false, ioConfig.isSkipFirehoseCaching()); } @Test @@ -186,7 +189,7 @@ public void testIndexTaskSerde() throws Exception ), jsonMapper ), - new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true, true), + new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), new IndexTask.IndexTuningConfig(10000, 10, 9999, null, indexSpec, 3, true, true, true) ), null, @@ -210,7 +213,6 @@ public void testIndexTaskSerde() throws Exception Assert.assertTrue(taskIoConfig.getFirehoseFactory() instanceof LocalFirehoseFactory); Assert.assertTrue(task2IoConfig.getFirehoseFactory() instanceof LocalFirehoseFactory); Assert.assertEquals(taskIoConfig.isAppendToExisting(), task2IoConfig.isAppendToExisting()); - Assert.assertEquals(taskIoConfig.isSkipFirehoseCaching(), task2IoConfig.isSkipFirehoseCaching()); IndexTask.IndexTuningConfig taskTuningConfig = task.getIngestionSchema().getTuningConfig(); IndexTask.IndexTuningConfig task2TuningConfig = task2.getIngestionSchema().getTuningConfig(); @@ -251,7 +253,7 @@ public void testIndexTaskwithResourceSerde() throws Exception ), jsonMapper ), - new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true, null), + new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), new IndexTask.IndexTuningConfig(10000, 10, null, null, indexSpec, 3, true, true, true) ), null, diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java index 3aee1a97c911..1574ea7650d8 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java @@ -49,6 +49,7 @@ import org.junit.Before; import org.junit.Test; +import java.io.File; import java.util.ArrayList; import java.util.Collection; import java.util.Set; @@ -71,6 +72,8 @@ public class RemoteTaskRunnerTest private Task task; private Worker worker; + private File baseDir = new File("lol"); + @Before public void setUp() throws Exception { @@ -79,6 +82,9 @@ public void setUp() throws Exception cf = rtrTestUtils.getCuratorFramework(); task = TestTasks.unending("task"); + + final File baseDir = new File("lol"); + baseDir.mkdir(); } @After @@ -88,6 +94,7 @@ public void tearDown() throws Exception remoteTaskRunner.stop(); } rtrTestUtils.tearDown(); + baseDir.delete(); } @Test diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index a4373d5be9a2..7a60901154b1 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -652,7 +652,7 @@ public void testIndexTask() throws Exception ), mapper ), - new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false, null), + new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false), new IndexTask.IndexTuningConfig(10000, 10, null, null, indexSpec, 3, true, true, true) ), null, @@ -710,7 +710,7 @@ public void testIndexTaskFailure() throws Exception ), mapper ), - new IndexTask.IndexIOConfig(new MockExceptionalFirehoseFactory(), false, null), + new IndexTask.IndexIOConfig(new MockExceptionalFirehoseFactory(), false), new IndexTask.IndexTuningConfig(10000, 10, null, null, indexSpec, 3, true, true, true) ), null, @@ -1069,7 +1069,7 @@ public void testResumeTasks() throws Exception ), mapper ), - new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false, null), + new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false), new IndexTask.IndexTuningConfig(10000, 10, null, null, indexSpec, null, false, null, null) ), null, diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java index 1d160e99bff6..c88f0d09af36 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java @@ -49,6 +49,10 @@ public TaskAnnouncementTest() { TestUtils testUtils = new TestUtils(); jsonMapper = testUtils.getTestObjectMapper(); + + final File baseDir = new File("lol"); + baseDir.mkdir(); + baseDir.deleteOnExit(); } @Test diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java index 381fefeed14f..3fccd0564a8c 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java @@ -21,30 +21,21 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; +import com.google.common.base.Preconditions; import com.metamx.emitter.EmittingLogger; -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.impl.FileIteratingFirehose; +import io.druid.data.input.impl.AbstractTextFilesFirehoseFactory; import io.druid.data.input.impl.StringInputRowParser; -import io.druid.java.util.common.IAE; -import io.druid.java.util.common.ISE; - import org.apache.commons.io.FileUtils; -import org.apache.commons.io.LineIterator; import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.commons.io.filefilter.WildcardFileFilter; import java.io.File; import java.io.IOException; -import java.util.Collection; -import java.util.Iterator; -import java.util.LinkedList; +import java.io.InputStream; /** */ -public class LocalFirehoseFactory implements FirehoseFactory +public class LocalFirehoseFactory extends AbstractTextFilesFirehoseFactory { private static final EmittingLogger log = new EmittingLogger(LocalFirehoseFactory.class); @@ -60,9 +51,18 @@ public LocalFirehoseFactory( @JsonProperty("parser") StringInputRowParser parser ) { + super( + FileUtils.listFiles( + Preconditions.checkNotNull(baseDir).getAbsoluteFile(), + new WildcardFileFilter(filter), + TrueFileFilter.INSTANCE + ) + ); this.baseDir = baseDir; this.filter = filter; this.parser = parser; + + log.info("Initialized with " + getObjects() + " files"); } @JsonProperty @@ -84,55 +84,14 @@ public StringInputRowParser getParser() } @Override - public Firehose connect(StringInputRowParser firehoseParser) throws IOException + protected InputStream openStream(File object) throws IOException { - if (baseDir == null) { - throw new IAE("baseDir is null"); - } - log.info("Searching for all [%s] in and beneath [%s]", filter, baseDir.getAbsoluteFile()); - - Collection foundFiles = FileUtils.listFiles( - baseDir.getAbsoluteFile(), - new WildcardFileFilter(filter), - TrueFileFilter.INSTANCE - ); - - if (foundFiles == null || foundFiles.isEmpty()) { - throw new ISE("Found no files to ingest! Check your schema."); - } - log.info ("Found files: " + foundFiles); - - final LinkedList files = Lists.newLinkedList( - foundFiles - ); - - return new FileIteratingFirehose( - new Iterator() - { - @Override - public boolean hasNext() - { - return !files.isEmpty(); - } - - @Override - public LineIterator next() - { - try { - return FileUtils.lineIterator(files.poll()); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } + return FileUtils.openInputStream(object); + } - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }, - firehoseParser - ); + @Override + protected boolean isGzipped(File object) + { + return object.getPath().endsWith(".gz"); } } diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java new file mode 100644 index 000000000000..9161b8806593 --- /dev/null +++ b/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java @@ -0,0 +1,123 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.realtime.firehose; + +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import io.druid.data.input.Firehose; +import io.druid.data.input.Row; +import io.druid.data.input.impl.CSVParseSpec; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.TimestampSpec; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.Writer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +public class LocalFirehoseFactoryTest +{ + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private LocalFirehoseFactory factory; + + @Before + public void setup() throws IOException + { + for (int i = 0; i < 5; i++) { + try (final Writer writer = new BufferedWriter( + new FileWriter(temporaryFolder.newFile("test_" + i)) + )) { + writer.write((20171225 + i) + "," + i + "th test file\n"); + } + } + + for (int i = 0; i < 5; i++) { + try (final Writer writer = new BufferedWriter( + new FileWriter(temporaryFolder.newFile("filtered_" + i)) + )) { + writer.write((20171225 + i) + "," + i + "th filtered file\n"); + } + } + + factory = new LocalFirehoseFactory(temporaryFolder.getRoot(), "test_*", null); + } + + @Test + public void testGetObjects() throws InterruptedException + { + final List files = factory.getObjects(); + Assert.assertEquals(5, files.size()); + } + + @Test + public void testIsGzipped() + { + Assert.assertTrue(factory.isGzipped(new File("test.gz"))); + Assert.assertFalse(factory.isGzipped(new File("test.g"))); + } + + @Test + public void testConnect() throws IOException + { + try (final Firehose firehose = factory.connect(new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec( + "timestamp", + "auto", + null + ), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a")), + Lists.newArrayList(), + Lists.newArrayList() + ), + ",", + Arrays.asList("timestamp", "a") + ), + Charsets.UTF_8.name() + ))) { + final List rows = new ArrayList<>(); + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + + Assert.assertEquals(5, rows.size()); + rows.sort(Comparator.comparing(Row::getTimestamp)); + for (int i = 0; i < 5; i++) { + final List dimVals = rows.get(i).getDimension("a"); + Assert.assertEquals(1, dimVals.size()); + Assert.assertEquals(i + "th test file", dimVals.get(0)); + } + } + } +} diff --git a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java index b83a93bdbe9d..37f7a3556c2b 100644 --- a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java @@ -141,6 +141,9 @@ public void testTaskValidator() throws Exception jsonMapper.registerModule(jacksonModule); } + final File file = new File("lol"); + file.mkdir(); + final RealtimeIndexTask task = new RealtimeIndexTask( null, new TaskResource("rofl", 2), @@ -191,6 +194,7 @@ public Plumber findPlumber( jsonMapper.writeValue(tmp, task); parseCommand("validator", "-f", tmp.getAbsolutePath(), "-t", "task").run(); + file.delete(); } @After From 3a45119626eec7d5c2d2e566efde9b53ceee42c9 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 22 Apr 2017 08:51:21 +0900 Subject: [PATCH 02/26] fix comment --- .../data/input/impl/PrefetcheableTextFilesFirehoseFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java index d2031c82581d..2db9ca826aa9 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java @@ -52,7 +52,7 @@ /** * PrefetcheableTextFilesFirehoseFactory is an abstract firehose factory for reading text files. The firehose returned - * by this class provides two key functionalities. + * by this class provides three key functionalities. * *
    *
  • Caching: for the first call of {@link #connect(StringInputRowParser)}, it caches objects in a local disk From bbdba216ba043c8d8dcbea0982a92bca3282de6c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 24 Apr 2017 00:35:42 +0900 Subject: [PATCH 03/26] exception handling --- ...PrefetcheableTextFilesFirehoseFactory.java | 58 ++++++++++--------- 1 file changed, 31 insertions(+), 27 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java index 2db9ca826aa9..3dcb5e780fbf 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java @@ -43,8 +43,10 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -104,8 +106,8 @@ public abstract class PrefetcheableTextFilesFirehoseFactory private final int maxFetchRetry; private volatile int nextFetchIndex; - private volatile boolean beingFetched; - private volatile Exception fetchException; + + private Future fetchFuture; public PrefetcheableTextFilesFirehoseFactory( Collection objects, @@ -167,28 +169,16 @@ private void cache() * This is for simplifying design, and should be improved when our client implementations for cloud storages like S3 * support range scan. */ - private void fetch() + private void fetch() throws Exception { - if (!beingFetched) { - beingFetched = true; - final List objects = getObjects(); - try { - for (int i = nextFetchIndex; i < objects.size() && fetchedBytes.get() <= maxFetchCapacityBytes; i++) { - final ObjectType object = objects.get(i); - LOG.info("Fetching object[%s]", object); - final File outFile = File.createTempFile("fetch-", null, baseDir); - fetchedBytes.addAndGet(download(object, outFile, 0)); - fetchFiles.put(new FetchedFile(outFile, isGzipped(object))); - nextFetchIndex++; - } - } - catch (Exception e) { - fetchException = e; - throw Throwables.propagate(e); - } - finally { - beingFetched = false; - } + final List objects = getObjects(); + for (int i = nextFetchIndex; i < objects.size() && fetchedBytes.get() <= maxFetchCapacityBytes; i++) { + final ObjectType object = objects.get(i); + LOG.info("Fetching object[%s]", object); + final File outFile = File.createTempFile("fetch-", null, baseDir); + fetchedBytes.addAndGet(download(object, outFile, 0)); + fetchFiles.put(new FetchedFile(outFile, isGzipped(object))); + nextFetchIndex++; } } @@ -258,8 +248,14 @@ public boolean hasNext() private void fetchIfNeeded(long remainingBytes) { - if (!beingFetched && remainingBytes <= prefetchTriggerBytes) { - fetchExecutor.submit(() -> fetch()); + if ((fetchFuture == null || fetchFuture.isDone()) + && remainingBytes <= prefetchTriggerBytes) { + fetchFuture = fetchExecutor.submit( + () -> { + fetch(); + return null; + } + ); } } @@ -270,8 +266,16 @@ public LineIterator next() throw new NoSuchElementException(); } - if (fetchException != null) { - throw Throwables.propagate(fetchException); + // If fetch() fails, hasNext() always returns true because nextFetchIndex must be smaller than the number + // of objects which means next() is always called. The below block checks that fetch() threw an exception + // and propagates it if exists. + if (fetchFuture != null && fetchFuture.isDone()) { + try { + fetchFuture.get(); + } + catch (InterruptedException | ExecutionException e) { + Throwables.propagate(e); + } } final FetchedFile fetchedFile; From 4e7ab6a028effd2d88f487720554abfe82d34e89 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 25 Apr 2017 13:23:18 +0900 Subject: [PATCH 04/26] Fix wrong json property --- .../azure/StaticAzureBlobStoreFirehoseFactory.java | 8 ++++---- .../cloudfiles/StaticCloudFilesFirehoseFactory.java | 8 ++++---- .../google/StaticGoogleBlobStoreFirehoseFactory.java | 8 ++++---- .../io/druid/firehose/s3/StaticS3FirehoseFactory.java | 8 ++++---- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java index c995ed00366b..69a1f34da32c 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java @@ -42,10 +42,10 @@ public StaticAzureBlobStoreFirehoseFactory( @JacksonInject("azureStorage") AzureStorage azureStorage, @JsonProperty("blobs") List blobs, @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, - @JsonProperty("maxCacheCapacityBytes") Long maxFetchCapacityBytes, - @JsonProperty("maxCacheCapacityBytes") Long prefetchTriggerBytes, - @JsonProperty("maxCacheCapacityBytes") Integer fetchTimeout, - @JsonProperty("maxCacheCapacityBytes") Integer maxFetchRetry + @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, + @JsonProperty("fetchTimeout") Integer fetchTimeout, + @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { super(blobs, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java index cf3528734c61..04a8a85a6883 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java @@ -43,10 +43,10 @@ public StaticCloudFilesFirehoseFactory( @JacksonInject("objectApi") CloudFilesApi cloudFilesApi, @JsonProperty("blobs") List blobs, @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, - @JsonProperty("maxCacheCapacityBytes") Long maxFetchCapacityBytes, - @JsonProperty("maxCacheCapacityBytes") Long prefetchTriggerBytes, - @JsonProperty("maxCacheCapacityBytes") Integer fetchTimeout, - @JsonProperty("maxCacheCapacityBytes") Integer maxFetchRetry + @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, + @JsonProperty("fetchTimeout") Integer fetchTimeout, + @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { super(blobs, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java index 2c2705c75628..9b6caff6b93a 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java @@ -39,10 +39,10 @@ public StaticGoogleBlobStoreFirehoseFactory( @JacksonInject GoogleStorage storage, @JsonProperty("blobs") List blobs, @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, - @JsonProperty("maxCacheCapacityBytes") Long maxFetchCapacityBytes, - @JsonProperty("maxCacheCapacityBytes") Long prefetchTriggerBytes, - @JsonProperty("maxCacheCapacityBytes") Integer fetchTimeout, - @JsonProperty("maxCacheCapacityBytes") Integer maxFetchRetry + @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, + @JsonProperty("fetchTimeout") Integer fetchTimeout, + @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { super(blobs, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index 9f3ff350d7b7..ddb253fb45af 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -49,10 +49,10 @@ public StaticS3FirehoseFactory( @JacksonInject("s3Client") RestS3Service s3Client, @JsonProperty("uris") List uris, @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, - @JsonProperty("maxCacheCapacityBytes") Long maxFetchCapacityBytes, - @JsonProperty("maxCacheCapacityBytes") Long prefetchTriggerBytes, - @JsonProperty("maxCacheCapacityBytes") Integer fetchTimeout, - @JsonProperty("maxCacheCapacityBytes") Integer maxFetchRetry + @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, + @JsonProperty("fetchTimeout") Integer fetchTimeout, + @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { super(uris, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); From cb65cde75f83ce34284d4690f781331dc93dc3ee Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 26 Apr 2017 03:36:20 +0900 Subject: [PATCH 05/26] Remove ReplayableFirehoseFactory and fix misspelling --- .../AbstractTextFilesFirehoseFactory.java | 40 +- .../input/impl/FileIteratingFirehose.java | 18 +- ...PrefetchableTextFilesFirehoseFactory.java} | 56 +-- ...etchableTextFilesFirehoseFactoryTest.java} | 64 ++- docs/content/ingestion/tasks.md | 1 - .../StaticAzureBlobStoreFirehoseFactory.java | 20 +- .../StaticCloudFilesFirehoseFactory.java | 17 +- .../StaticGoogleBlobStoreFirehoseFactory.java | 16 +- .../firehose/s3/StaticS3FirehoseFactory.java | 18 +- .../s3/StaticS3FirehoseFactoryTest.java | 1 - .../druid/indexing/common/task/IndexTask.java | 8 +- .../java/io/druid/guice/FirehoseModule.java | 2 - .../firehose/ReplayableFirehoseFactory.java | 319 ------------- .../ReplayableFirehoseFactoryTest.java | 447 ------------------ 14 files changed, 102 insertions(+), 925 deletions(-) rename api/src/main/java/io/druid/data/input/impl/{PrefetcheableTextFilesFirehoseFactory.java => PrefetchableTextFilesFirehoseFactory.java} (88%) rename api/src/test/java/io/druid/data/input/impl/{PrefetcheableTextFilesFirehoseFactoryTest.java => PrefetchableTextFilesFirehoseFactoryTest.java} (78%) delete mode 100644 server/src/main/java/io/druid/segment/realtime/firehose/ReplayableFirehoseFactory.java delete mode 100644 server/src/test/java/io/druid/realtime/firehose/ReplayableFirehoseFactoryTest.java diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java index 54bdcceee051..881c6dadf39a 100644 --- a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableList; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; -import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.logger.Logger; import org.apache.commons.io.Charsets; import org.apache.commons.io.IOUtils; @@ -54,7 +53,7 @@ public abstract class AbstractTextFilesFirehoseFactory public AbstractTextFilesFirehoseFactory(Collection objects) { - this.objects = ImmutableList.copyOf(Preconditions.checkNotNull(objects)); + this.objects = ImmutableList.copyOf(Preconditions.checkNotNull(objects, "objects")); } @Override @@ -78,10 +77,9 @@ public LineIterator next() } final ObjectType object = iterator.next(); try { - final InputStream stream = openStream(object); return IOUtils.lineIterator( new BufferedReader( - new InputStreamReader(wrapIfNeeded(object, stream), Charsets.UTF_8) + new InputStreamReader(openStream(object), Charsets.UTF_8) ) ); } @@ -104,33 +102,8 @@ public List getObjects() return objects; } - /** - * Wrap the given stream if needed, currently when the given object is compressed with gzip. - * - * @param object an object - * @param innerStream input stream for object - * - * @return wrapped input stream if the object is gzipped - * - * @throws IOException - */ - protected InputStream wrapIfNeeded(ObjectType object, InputStream innerStream) throws IOException - { - return wrapIfNeeded(innerStream, isGzipped(object)); - } - - protected static InputStream wrapIfNeeded(InputStream innerStream, boolean isGzipped) throws IOException - { - if (isGzipped) { - return CompressionUtils.gzipInputStream(innerStream); - } - - return innerStream; - } - /** * Open an input stream from the given object. - * The result input stream must not be wrapped even if the object is compressed with gzip. * * @param object an object to be read * @@ -139,13 +112,4 @@ protected static InputStream wrapIfNeeded(InputStream innerStream, boolean isGzi * @throws IOException */ protected abstract InputStream openStream(ObjectType object) throws IOException; - - /** - * Check if the object is compressed with gzip. - * - * @param object an object to be checked - * - * @return true if the object is compressed with gzip - */ - protected abstract boolean isGzipped(ObjectType object); } diff --git a/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java b/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java index e4d313e9ee72..e4ff057ab689 100644 --- a/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java +++ b/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java @@ -101,10 +101,22 @@ public Runnable commit() @Override public void close() throws IOException { - if (lineIterator != null) { - lineIterator.close(); + try { + if (lineIterator != null) { + lineIterator.close(); + } + } + catch (Throwable t) { + try { + if (closer != null) { + closer.close(); + } + } + catch (Exception e) { + t.addSuppressed(e); + } + throw t; } - if (closer != null) { closer.close(); } diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java similarity index 88% rename from api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java rename to api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 3dcb5e780fbf..3f856877f3ce 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -24,6 +24,7 @@ import com.google.common.io.CountingOutputStream; import com.google.common.io.Files; import io.druid.data.input.Firehose; +import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.logger.Logger; import org.apache.commons.io.Charsets; import org.apache.commons.io.FileUtils; @@ -53,7 +54,7 @@ import java.util.concurrent.atomic.AtomicLong; /** - * PrefetcheableTextFilesFirehoseFactory is an abstract firehose factory for reading text files. The firehose returned + * PrefetchableTextFilesFirehoseFactory is an abstract firehose factory for reading text files. The firehose returned * by this class provides three key functionalities. * *
      @@ -72,14 +73,14 @@ * IndexTask can read the whole data twice for determining partition specs and generating segments if the intervals of * GranularitySpec is not specified. */ -public abstract class PrefetcheableTextFilesFirehoseFactory +public abstract class PrefetchableTextFilesFirehoseFactory extends AbstractTextFilesFirehoseFactory { - private static final Logger LOG = new Logger(PrefetcheableTextFilesFirehoseFactory.class); + private static final Logger LOG = new Logger(PrefetchableTextFilesFirehoseFactory.class); private static final long DEFAULT_MAX_CACHE_CAPACITY_BYTES = 1024 * 1024 * 1024; // 1GB private static final long DEFAULT_MAX_FETCH_CAPACITY_BYTES = 1024 * 1024 * 1024; // 1GB + private static final long DEFAULT_FETCH_TIMEOUT = 60_000; // 60 secs private static final int DEFAULT_MAX_FETCH_RETRY = 3; - private static final int DEFAULT_FETCH_TIMEOUT = 60_000; // 60 secs // The below two variables are roughly the max size of total cached/fetched objects, but the actual cached/fetched // size can be larger. The reason is our current client implementations for cloud storages like s3 don't support range @@ -92,15 +93,15 @@ public abstract class PrefetcheableTextFilesFirehoseFactory private final long prefetchTriggerBytes; private File baseDir; // Directory for cached and fetched files - private final List cacheFiles; - private final LinkedBlockingQueue fetchFiles; + private final List cacheFiles; + private final LinkedBlockingQueue fetchFiles; // Number of bytes currently fetched files. // This is updated when fetch a file is successfully fetched or a fetched file is deleted. private final AtomicLong fetchedBytes = new AtomicLong(0); // timeout for fetching an object from the remote site - private final int fetchTimeout; + private final long fetchTimeout; // maximum retry for fetching an object from the remote site private final int maxFetchRetry; @@ -109,12 +110,12 @@ public abstract class PrefetcheableTextFilesFirehoseFactory private Future fetchFuture; - public PrefetcheableTextFilesFirehoseFactory( + public PrefetchableTextFilesFirehoseFactory( Collection objects, Long maxCacheCapacityBytes, Long maxFetchCapacityBytes, Long prefetchTriggerBytes, - Integer fetchTimeout, + Long fetchTimeout, Integer maxFetchRetry ) { @@ -136,7 +137,7 @@ public PrefetcheableTextFilesFirehoseFactory( } @VisibleForTesting - List getCacheFiles() + List getCacheFiles() { return cacheFiles; } @@ -154,7 +155,7 @@ private void cache() LOG.info("Caching object[%s]", object); final File outFile = File.createTempFile("cache-", null, baseDir); totalFetchedBytes += download(object, outFile, 0); - cacheFiles.add(new FetchedFile(outFile, isGzipped(object))); + cacheFiles.add(outFile); nextFetchIndex++; } } @@ -177,7 +178,7 @@ private void fetch() throws Exception LOG.info("Fetching object[%s]", object); final File outFile = File.createTempFile("fetch-", null, baseDir); fetchedBytes.addAndGet(download(object, outFile, 0)); - fetchFiles.put(new FetchedFile(outFile, isGzipped(object))); + fetchFiles.put(outFile); nextFetchIndex++; } } @@ -229,9 +230,9 @@ public Firehose connect(StringInputRowParser firehoseParser) throws IOException return new FileIteratingFirehose( new Iterator() { - final Iterator cacheFileIterator = cacheFiles.iterator(); + final Iterator cacheFileIterator = cacheFiles.iterator(); long remainingCachedBytes = cacheFiles.stream() - .mapToLong(fetchedFile -> fetchedFile.file.length()) + .mapToLong(File::length) .sum(); { @@ -278,12 +279,12 @@ public LineIterator next() } } - final FetchedFile fetchedFile; + final File fetchedFile; final Closeable closeable; // Check cache first if (cacheFileIterator.hasNext()) { fetchedFile = cacheFileIterator.next(); - remainingCachedBytes -= fetchedFile.file.length(); + remainingCachedBytes -= fetchedFile.length(); fetchIfNeeded(remainingCachedBytes); closeable = () -> { }; @@ -306,18 +307,21 @@ public LineIterator next() } } closeable = () -> { - final long fileSize = fetchedFile.file.length(); - fetchedFile.file.delete(); + final long fileSize = fetchedFile.length(); + fetchedFile.delete(); fetchedBytes.addAndGet(-fileSize); }; } try { - final InputStream stream = FileUtils.openInputStream(fetchedFile.file); + InputStream stream = FileUtils.openInputStream(fetchedFile); + if (fetchedFile.getName().endsWith(".gz")) { + stream = CompressionUtils.gzipInputStream(stream); + } return new ResourceCloseableLineIterator( new BufferedReader( - new InputStreamReader(wrapIfNeeded(stream, fetchedFile.isGzipped), Charsets.UTF_8) + new InputStreamReader(stream, Charsets.UTF_8) ), closeable ); @@ -332,18 +336,6 @@ public LineIterator next() ); } - static class FetchedFile - { - private final File file; - private final boolean isGzipped; - - FetchedFile(File file, boolean isGzipped) - { - this.file = file; - this.isGzipped = isGzipped; - } - } - /** * This class calls the {@link Closeable#close()} method of the resourceCloser when it is closed. */ diff --git a/api/src/test/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java similarity index 78% rename from api/src/test/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactoryTest.java rename to api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java index 2a5e86713836..e940f0a6dc23 100644 --- a/api/src/test/java/io/druid/data/input/impl/PrefetcheableTextFilesFirehoseFactoryTest.java +++ b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java @@ -46,7 +46,7 @@ import java.util.List; import java.util.concurrent.TimeoutException; -public class PrefetcheableTextFilesFirehoseFactoryTest +public class PrefetchableTextFilesFirehoseFactoryTest { private static File testDir; @@ -74,7 +74,7 @@ public class PrefetcheableTextFilesFirehoseFactoryTest @BeforeClass public static void setup() throws IOException { - testDir = File.createTempFile(PrefetcheableTextFilesFirehoseFactoryTest.class.getSimpleName(), ""); + testDir = File.createTempFile(PrefetchableTextFilesFirehoseFactoryTest.class.getSimpleName(), ""); FileUtils.forceDelete(testDir); FileUtils.forceMkdir(testDir); @@ -126,8 +126,8 @@ private static void assertResult(List rows) @Test public void testWithoutCache() throws IOException { - final TestPrefetcheableTextFilesFirehoseFactory factory = - TestPrefetcheableTextFilesFirehoseFactory.withoutCache(testDir); + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.withoutCache(testDir); final List rows = new ArrayList<>(); try (Firehose firehose = factory.connect(parser)) { @@ -143,8 +143,8 @@ public void testWithoutCache() throws IOException @Test public void testWithZeroFetchCapacity() throws IOException { - final TestPrefetcheableTextFilesFirehoseFactory factory = - TestPrefetcheableTextFilesFirehoseFactory.withZeroFetchCapacity(testDir); + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.withZeroFetchCapacity(testDir); final List rows = new ArrayList<>(); try (Firehose firehose = factory.connect(parser)) { @@ -159,8 +159,8 @@ public void testWithZeroFetchCapacity() throws IOException @Test public void testWithCacheAndFetch() throws IOException { - final TestPrefetcheableTextFilesFirehoseFactory factory = - TestPrefetcheableTextFilesFirehoseFactory.of(testDir); + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.of(testDir); final List rows = new ArrayList<>(); try (Firehose firehose = factory.connect(parser)) { @@ -175,8 +175,8 @@ public void testWithCacheAndFetch() throws IOException @Test public void testRetry() throws IOException { - final TestPrefetcheableTextFilesFirehoseFactory factory = - TestPrefetcheableTextFilesFirehoseFactory.withOpenExceptions(testDir, 1); + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.withOpenExceptions(testDir, 1); final List rows = new ArrayList<>(); try (Firehose firehose = factory.connect(parser)) { @@ -195,8 +195,8 @@ public void testMaxRetry() throws IOException expectedException.expectCause(CoreMatchers.instanceOf(IOException.class)); expectedException.expectMessage("Exception for retry test"); - final TestPrefetcheableTextFilesFirehoseFactory factory = - TestPrefetcheableTextFilesFirehoseFactory.withOpenExceptions(testDir, 5); + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.withOpenExceptions(testDir, 5); final List rows = new ArrayList<>(); try (Firehose firehose = factory.connect(parser)) { @@ -212,8 +212,8 @@ public void testTimeout() throws IOException expectedException.expect(RuntimeException.class); expectedException.expectCause(CoreMatchers.instanceOf(TimeoutException.class)); - final TestPrefetcheableTextFilesFirehoseFactory factory = - TestPrefetcheableTextFilesFirehoseFactory.withSleepMillis(testDir, 1000); + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.withSleepMillis(testDir, 1000); final List rows = new ArrayList<>(); try (Firehose firehose = factory.connect(parser)) { @@ -226,8 +226,8 @@ public void testTimeout() throws IOException @Test public void testReconnect() throws IOException { - final TestPrefetcheableTextFilesFirehoseFactory factory = - TestPrefetcheableTextFilesFirehoseFactory.of(testDir); + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.of(testDir); for (int i = 0; i < 5; i++) { final List rows = new ArrayList<>(); @@ -240,14 +240,14 @@ public void testReconnect() throws IOException } } - static class TestPrefetcheableTextFilesFirehoseFactory extends PrefetcheableTextFilesFirehoseFactory + static class TestPrefetchableTextFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private long sleepMillis; private int openExceptionCount; - static TestPrefetcheableTextFilesFirehoseFactory withoutCache(File baseDir) + static TestPrefetchableTextFilesFirehoseFactory withoutCache(File baseDir) { - return new TestPrefetcheableTextFilesFirehoseFactory( + return new TestPrefetchableTextFilesFirehoseFactory( baseDir, 1024, 0, @@ -259,9 +259,9 @@ static TestPrefetcheableTextFilesFirehoseFactory withoutCache(File baseDir) ); } - static TestPrefetcheableTextFilesFirehoseFactory withZeroFetchCapacity(File baseDir) + static TestPrefetchableTextFilesFirehoseFactory withZeroFetchCapacity(File baseDir) { - return new TestPrefetcheableTextFilesFirehoseFactory( + return new TestPrefetchableTextFilesFirehoseFactory( baseDir, 1024, 2048, @@ -273,9 +273,9 @@ static TestPrefetcheableTextFilesFirehoseFactory withZeroFetchCapacity(File base ); } - static TestPrefetcheableTextFilesFirehoseFactory of(File baseDir) + static TestPrefetchableTextFilesFirehoseFactory of(File baseDir) { - return new TestPrefetcheableTextFilesFirehoseFactory( + return new TestPrefetchableTextFilesFirehoseFactory( baseDir, 1024, 2048, @@ -287,9 +287,9 @@ static TestPrefetcheableTextFilesFirehoseFactory of(File baseDir) ); } - static TestPrefetcheableTextFilesFirehoseFactory withOpenExceptions(File baseDir, int count) + static TestPrefetchableTextFilesFirehoseFactory withOpenExceptions(File baseDir, int count) { - return new TestPrefetcheableTextFilesFirehoseFactory( + return new TestPrefetchableTextFilesFirehoseFactory( baseDir, 1024, 2048, @@ -301,9 +301,9 @@ static TestPrefetcheableTextFilesFirehoseFactory withOpenExceptions(File baseDir ); } - static TestPrefetcheableTextFilesFirehoseFactory withSleepMillis(File baseDir, long ms) + static TestPrefetchableTextFilesFirehoseFactory withSleepMillis(File baseDir, long ms) { - return new TestPrefetcheableTextFilesFirehoseFactory( + return new TestPrefetchableTextFilesFirehoseFactory( baseDir, 1024, 2048, @@ -315,12 +315,12 @@ static TestPrefetcheableTextFilesFirehoseFactory withSleepMillis(File baseDir, l ); } - public TestPrefetcheableTextFilesFirehoseFactory( + public TestPrefetchableTextFilesFirehoseFactory( File baseDir, long prefetchTriggerThreshold, long maxCacheCapacityBytes, long maxFetchCapacityBytes, - int timeout, + long timeout, int maxRetry, int openExceptionCount, long sleepMillis @@ -359,11 +359,5 @@ protected InputStream openStream(File object) throws IOException } return FileUtils.openInputStream(object); } - - @Override - protected boolean isGzipped(File object) - { - return object.getPath().endsWith(".gz"); - } } } diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index 6e01e82a386c..4e72e993c162 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -104,7 +104,6 @@ See [Ingestion](../ingestion/index.html) |type|The task type, this should always be "index".|none|yes| |firehose|Specify a [Firehose](../ingestion/firehose.html) here.|none|yes| |appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This will only work if the existing segment set has extendable-type shardSpecs (which can be forced by setting 'forceExtendableShardSpecs' in the tuning config).|false|no| -|skipFirehoseCaching|By default the IndexTask will fully read the supplied firehose to disk before processing the data. This prevents the task from doing multiple remote fetches and enforces determinism if more than one pass through the data is required. It also allows the task to retry fetching the data if the firehose throws an exception during reading. This requires sufficient disk space for the temporary cache.|false|no| #### TuningConfig diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java index 69a1f34da32c..8b2728d7843d 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java @@ -22,7 +22,8 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.data.input.impl.PrefetcheableTextFilesFirehoseFactory; +import io.druid.data.input.impl.PrefetchableTextFilesFirehoseFactory; +import io.druid.java.util.common.CompressionUtils; import io.druid.storage.azure.AzureByteSource; import io.druid.storage.azure.AzureStorage; @@ -33,7 +34,7 @@ /** * This class is heavily inspired by the StaticS3FirehoseFactory class in the io.druid.firehose.s3 package */ -public class StaticAzureBlobStoreFirehoseFactory extends PrefetcheableTextFilesFirehoseFactory +public class StaticAzureBlobStoreFirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private final AzureStorage azureStorage; @@ -44,7 +45,7 @@ public StaticAzureBlobStoreFirehoseFactory( @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, - @JsonProperty("fetchTimeout") Integer fetchTimeout, + @JsonProperty("fetchTimeout") Long fetchTimeout, @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { @@ -61,13 +62,12 @@ public List getBlobs() @Override protected InputStream openStream(AzureBlob object) throws IOException { - return makeByteSource(azureStorage, object).openStream(); - } - - @Override - protected boolean isGzipped(AzureBlob object) - { - return object.getPath().endsWith(".gz"); + final InputStream stream = makeByteSource(azureStorage, object).openStream(); + if (object.getPath().endsWith(".gz")) { + return CompressionUtils.gzipInputStream(stream); + } else { + return stream; + } } private static AzureByteSource makeByteSource(AzureStorage azureStorage, AzureBlob object) diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java index 04a8a85a6883..1672a225cd49 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java @@ -22,7 +22,8 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.data.input.impl.PrefetcheableTextFilesFirehoseFactory; +import io.druid.data.input.impl.PrefetchableTextFilesFirehoseFactory; +import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.logger.Logger; import io.druid.storage.cloudfiles.CloudFilesByteSource; import io.druid.storage.cloudfiles.CloudFilesObjectApiProxy; @@ -32,7 +33,7 @@ import java.io.InputStream; import java.util.List; -public class StaticCloudFilesFirehoseFactory extends PrefetcheableTextFilesFirehoseFactory +public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private static final Logger log = new Logger(StaticCloudFilesFirehoseFactory.class); @@ -45,7 +46,7 @@ public StaticCloudFilesFirehoseFactory( @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, - @JsonProperty("fetchTimeout") Integer fetchTimeout, + @JsonProperty("fetchTimeout") Long fetchTimeout, @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { @@ -73,13 +74,7 @@ protected InputStream openStream(CloudFilesBlob object) throws IOException cloudFilesApi, region, container); final CloudFilesByteSource byteSource = new CloudFilesByteSource(objectApi, path); - return byteSource.openStream(); + final InputStream stream = byteSource.openStream(); + return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } - - @Override - protected boolean isGzipped(CloudFilesBlob object) - { - return object.getPath().endsWith(".gz"); - } - } diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java index 9b6caff6b93a..d99d7d5b7e36 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java @@ -22,7 +22,8 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.data.input.impl.PrefetcheableTextFilesFirehoseFactory; +import io.druid.data.input.impl.PrefetchableTextFilesFirehoseFactory; +import io.druid.java.util.common.CompressionUtils; import io.druid.storage.google.GoogleByteSource; import io.druid.storage.google.GoogleStorage; @@ -30,7 +31,7 @@ import java.io.InputStream; import java.util.List; -public class StaticGoogleBlobStoreFirehoseFactory extends PrefetcheableTextFilesFirehoseFactory +public class StaticGoogleBlobStoreFirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private final GoogleStorage storage; @@ -41,7 +42,7 @@ public StaticGoogleBlobStoreFirehoseFactory( @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, - @JsonProperty("fetchTimeout") Integer fetchTimeout, + @JsonProperty("fetchTimeout") Long fetchTimeout, @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { @@ -62,13 +63,8 @@ protected InputStream openStream(GoogleBlob object) throws IOException ? object.getPath().substring(1) : object.getPath(); - return new GoogleByteSource(storage, bucket, path).openStream(); - } - - @Override - protected boolean isGzipped(GoogleBlob object) - { - return object.getPath().endsWith(".gz"); + final InputStream stream = new GoogleByteSource(storage, bucket, path).openStream(); + return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } } diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index ddb253fb45af..8093f60649df 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -23,7 +23,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import io.druid.data.input.impl.PrefetcheableTextFilesFirehoseFactory; +import io.druid.data.input.impl.PrefetchableTextFilesFirehoseFactory; +import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.logger.Logger; import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -38,7 +39,7 @@ /** * Builds firehoses that read from a predefined list of S3 objects and then dry up. */ -public class StaticS3FirehoseFactory extends PrefetcheableTextFilesFirehoseFactory +public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private static final Logger log = new Logger(StaticS3FirehoseFactory.class); @@ -51,12 +52,12 @@ public StaticS3FirehoseFactory( @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, - @JsonProperty("fetchTimeout") Integer fetchTimeout, + @JsonProperty("fetchTimeout") Long fetchTimeout, @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { super(uris, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); - this.s3Client = Preconditions.checkNotNull(s3Client, "null s3Client");; + this.s3Client = Preconditions.checkNotNull(s3Client, "null s3Client"); for (final URI inputURI : uris) { Preconditions.checkArgument(inputURI.getScheme().equals("s3"), "input uri scheme == s3 (%s)", inputURI); @@ -82,19 +83,14 @@ protected InputStream openStream(URI object) throws IOException log.info("Reading from bucket[%s] object[%s] (%s)", s3Bucket, s3Object.getKey(), object); try { - return s3Client.getObject(new S3Bucket(s3Bucket), s3Object.getKey()).getDataInputStream(); + final InputStream stream = s3Client.getObject(new S3Bucket(s3Bucket), s3Object.getKey()).getDataInputStream(); + return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } catch (ServiceException e) { throw new IOException(e); } } - @Override - protected boolean isGzipped(URI object) - { - return object.getPath().endsWith(".gz"); - } - @Override public boolean equals(Object o) { diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java index 49c6938b61e6..0663b40de613 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java @@ -57,7 +57,6 @@ public void testSerde() throws Exception Assert.assertEquals(factory, outputFact); Assert.assertEquals(uris, outputFact.getUris()); - Assert.assertTrue(uris.stream().allMatch(outputFact::isGzipped)); } // This class is a workaround for the injectable value that StaticS3FirehoseFactory requires diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 790894f151e8..0c20106ef72a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -167,15 +167,13 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception .bucketIntervals() .isPresent(); - final FirehoseFactory delegateFirehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); + final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); - if (delegateFirehoseFactory instanceof IngestSegmentFirehoseFactory) { + if (firehoseFactory instanceof IngestSegmentFirehoseFactory) { // pass toolbox to Firehose - ((IngestSegmentFirehoseFactory) delegateFirehoseFactory).setTaskToolbox(toolbox); + ((IngestSegmentFirehoseFactory) firehoseFactory).setTaskToolbox(toolbox); } - final FirehoseFactory firehoseFactory = delegateFirehoseFactory; - final Map> shardSpecs = determineShardSpecs(toolbox, firehoseFactory); final String version; diff --git a/server/src/main/java/io/druid/guice/FirehoseModule.java b/server/src/main/java/io/druid/guice/FirehoseModule.java index 2c9fb462d1fa..dd8a605af969 100644 --- a/server/src/main/java/io/druid/guice/FirehoseModule.java +++ b/server/src/main/java/io/druid/guice/FirehoseModule.java @@ -30,7 +30,6 @@ import io.druid.segment.realtime.firehose.FixedCountFirehoseFactory; import io.druid.segment.realtime.firehose.IrcFirehoseFactory; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; -import io.druid.segment.realtime.firehose.ReplayableFirehoseFactory; import io.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; import java.util.Arrays; @@ -57,7 +56,6 @@ public List getJacksonModules() new NamedType(LocalFirehoseFactory.class, "local"), new NamedType(EventReceiverFirehoseFactory.class, "receiver"), new NamedType(CombiningFirehoseFactory.class, "combining"), - new NamedType(ReplayableFirehoseFactory.class, "replayable"), new NamedType(FixedCountFirehoseFactory.class, "fixedCount") ) ); diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/ReplayableFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/ReplayableFirehoseFactory.java deleted file mode 100644 index 53a586ce4f28..000000000000 --- a/server/src/main/java/io/druid/segment/realtime/firehose/ReplayableFirehoseFactory.java +++ /dev/null @@ -1,319 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonFactory; -import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.fasterxml.jackson.dataformat.smile.SmileGenerator; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.Iterators; -import com.google.common.collect.Ordering; -import com.google.common.io.CountingOutputStream; -import com.google.common.io.Files; -import com.metamx.emitter.EmittingLogger; -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.InputRow; -import io.druid.data.input.Row; -import io.druid.data.input.Rows; -import io.druid.data.input.impl.InputRowParser; -import io.druid.guice.annotations.Smile; -import io.druid.java.util.common.parsers.ParseException; -import io.druid.utils.Runnables; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - -/** - * Creates a wrapper firehose that writes from another firehose to disk and then serves nextRow() from disk. Useful for - * tasks that require multiple passes through the data to prevent multiple remote fetches. Also has support for - * retrying fetches if the underlying firehose throws an exception while the local cache is being generated. - */ -public class ReplayableFirehoseFactory implements FirehoseFactory -{ - private static final EmittingLogger log = new EmittingLogger(ReplayableFirehoseFactory.class); - private static final boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = false; - private static final int DEFAULT_MAX_TEMP_FILE_SIZE = 250000000; - private static final int DEFAULT_READ_FIREHOSE_RETRIES = 3; - - private final FirehoseFactory delegateFactory; - private final boolean reportParseExceptions; - - // This is *roughly* the max size of the temp files that will be generated, but they may be slightly larger. The - // reason for the approximation is that we're not forcing flushes after writing to the generator so the number of - // bytes written to the stream won't be updated until the flush occurs. It's probably more important to optimize for - // I/O speed rather than maintaining a strict max on the size of the temp file before it's split. - private final int maxTempFileSize; - - private final int readFirehoseRetries; - private final ObjectMapper smileMapper; - - private ReplayableFirehose firehose; - - @JsonCreator - public ReplayableFirehoseFactory( - @JsonProperty("delegate") FirehoseFactory delegateFactory, - @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, - @JsonProperty("maxTempFileSize") Integer maxTempFileSize, - @JsonProperty("readFirehoseRetries") Integer readFirehoseRetries, - @Smile @JacksonInject ObjectMapper smileMapper - ) - { - Preconditions.checkNotNull(delegateFactory, "delegate cannot be null"); - Preconditions.checkArgument( - !(delegateFactory instanceof ReplayableFirehoseFactory), - "Refusing to wrap another ReplayableFirehoseFactory" - ); - - this.delegateFactory = delegateFactory; - this.reportParseExceptions = reportParseExceptions == null - ? DEFAULT_REPORT_PARSE_EXCEPTIONS - : reportParseExceptions; - this.maxTempFileSize = maxTempFileSize == null ? DEFAULT_MAX_TEMP_FILE_SIZE : maxTempFileSize; - this.readFirehoseRetries = readFirehoseRetries == null ? DEFAULT_READ_FIREHOSE_RETRIES : readFirehoseRetries; - - this.smileMapper = smileMapper; - - log.info(this.toString()); - } - - @Override - public Firehose connect(InputRowParser parser) throws IOException - { - if (firehose == null) { - firehose = new ReplayableFirehose(parser); - } else { - log.info("Rewinding and returning existing firehose"); - firehose.rewind(); - } - - return firehose; - } - - public class ReplayableFirehose implements Firehose - { - private final List files = new ArrayList<>(); - private final List dimensions; - - private int fileIndex = 0; - private JsonFactory jsonFactory; - private JsonParser jsonParser; - private Iterator it; - - public ReplayableFirehose(InputRowParser parser) throws IOException - { - jsonFactory = smileMapper.getFactory(); - - if (jsonFactory instanceof SmileFactory) { - jsonFactory = ((SmileFactory) jsonFactory).enable(SmileGenerator.Feature.CHECK_SHARED_STRING_VALUES); - } - - long counter = 0, totalBytes = 0, unparseable = 0, retryCount = 0; - Set dimensionScratch = new HashSet<>(); - - File tmpDir = Files.createTempDir(); - tmpDir.deleteOnExit(); - - long startTime = System.nanoTime(); - boolean isDone = false; - do { - deleteTempFiles(); - try (Firehose delegateFirehose = delegateFactory.connect(parser)) { - while (delegateFirehose.hasMore()) { - File tmpFile = File.createTempFile("replayable-", null, tmpDir); - tmpFile.deleteOnExit(); - - files.add(tmpFile); - log.debug("Created file [%s]", tmpFile.getAbsolutePath()); - - try (CountingOutputStream cos = new CountingOutputStream(new FileOutputStream(tmpFile)); - JsonGenerator generator = jsonFactory.createGenerator(cos)) { - - while (delegateFirehose.hasMore() && cos.getCount() < getMaxTempFileSize()) { - try { - InputRow row = delegateFirehose.nextRow(); - generator.writeObject(row); - dimensionScratch.addAll(row.getDimensions()); - counter++; - } - catch (ParseException e) { - if (reportParseExceptions) { - throw e; - } - unparseable++; - } - } - - totalBytes += cos.getCount(); - } - } - isDone = true; - } - catch (Exception e) { - if (++retryCount <= readFirehoseRetries && !(e instanceof ParseException)) { - log.error(e, "Delegate firehose threw an exception, retrying (%d of %d)", retryCount, readFirehoseRetries); - } else { - log.error(e, "Delegate firehose threw an exception, retries exhausted, aborting"); - Throwables.propagate(e); - } - } - } while (!isDone); - - log.info( - "Finished reading from firehose in [%,dms], [%,d] events parsed, [%,d] bytes written, [%,d] events unparseable", - (System.nanoTime() - startTime) / 1000000, - counter, - totalBytes, - unparseable - ); - - dimensions = Ordering.natural().immutableSortedCopy(dimensionScratch); - - if (counter == 0) { - log.warn("Firehose contains no events!"); - deleteTempFiles(); - it = Iterators.emptyIterator(); - } else { - jsonParser = jsonFactory.createParser(files.get(fileIndex)); - it = jsonParser.readValuesAs(Row.class); - } - } - - @Override - public boolean hasMore() - { - if (it.hasNext()) { - return true; - } - - try { - if (jsonParser != null) { - jsonParser.close(); - } - - if (++fileIndex >= files.size() || files.get(fileIndex).length() == 0) { - return false; - } - - jsonParser = jsonFactory.createParser(files.get(fileIndex)); - it = jsonParser.readValuesAs(Row.class); - return true; - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public InputRow nextRow() - { - return Rows.toCaseInsensitiveInputRow(it.next(), dimensions); - } - - @Override - public Runnable commit() - { - return Runnables.getNoopRunnable(); - } - - /** - * Closes the firehose by closing the input stream and setting an empty iterator. The underlying cache files - * backing the firehose are retained for when the firehose is "replayed" by calling rewind(). The cache files are - * deleted by File.deleteOnExit() when the process exits. - */ - @Override - public void close() throws IOException - { - if (jsonParser != null) { - jsonParser.close(); - } - it = Iterators.emptyIterator(); - } - - private void rewind() throws IOException - { - close(); - - if (!files.isEmpty()) { - fileIndex = 0; - jsonParser = jsonFactory.createParser(files.get(fileIndex)); - it = jsonParser.readValuesAs(Row.class); - } - } - - private void deleteTempFiles() - { - for (File file : files) { - log.debug("Deleting temp file: %s", file.getAbsolutePath()); - file.delete(); - } - - files.clear(); - } - } - - @JsonProperty("delegate") - public FirehoseFactory getDelegateFactory() - { - return delegateFactory; - } - - @JsonProperty("reportParseExceptions") - public boolean isReportParseExceptions() - { - return reportParseExceptions; - } - - @JsonProperty("maxTempFileSize") - public int getMaxTempFileSize() - { - return maxTempFileSize; - } - - @JsonProperty("readFirehoseRetries") - public int getReadFirehoseRetries() - { - return readFirehoseRetries; - } - - @Override - public String toString() - { - return "ReplayableFirehoseFactory{" + - "delegateFactory=" + delegateFactory + - ", reportParseExceptions=" + reportParseExceptions + - ", maxTempFileSize=" + maxTempFileSize + - ", readFirehoseRetries=" + readFirehoseRetries + - '}'; - } -} diff --git a/server/src/test/java/io/druid/realtime/firehose/ReplayableFirehoseFactoryTest.java b/server/src/test/java/io/druid/realtime/firehose/ReplayableFirehoseFactoryTest.java deleted file mode 100644 index f1631049c4cb..000000000000 --- a/server/src/test/java/io/druid/realtime/firehose/ReplayableFirehoseFactoryTest.java +++ /dev/null @@ -1,447 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.realtime.firehose; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.InputRow; -import io.druid.data.input.MapBasedInputRow; -import io.druid.data.input.impl.InputRowParser; -import io.druid.data.input.impl.MapInputRowParser; -import io.druid.data.input.impl.TimeAndDimsParseSpec; -import io.druid.jackson.DefaultObjectMapper; -import io.druid.java.util.common.parsers.ParseException; -import io.druid.segment.realtime.firehose.ReplayableFirehoseFactory; -import org.easymock.EasyMockSupport; -import org.easymock.IAnswer; -import org.joda.time.DateTime; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; -import java.util.List; - -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.expectLastCall; - -public class ReplayableFirehoseFactoryTest extends EasyMockSupport -{ - private FirehoseFactory delegateFactory = createMock(FirehoseFactory.class); - private Firehose delegateFirehose = createMock(Firehose.class); - private InputRowParser parser = new MapInputRowParser(new TimeAndDimsParseSpec(null, null)); - private ObjectMapper mapper = new DefaultObjectMapper(); - - - private List testRows = Lists.newArrayList( - new MapBasedInputRow( - DateTime.now(), Lists.newArrayList("dim1", "dim2"), - ImmutableMap.of("dim1", "val1", "dim2", "val2", "met1", 1) - ), - new MapBasedInputRow( - DateTime.now(), Lists.newArrayList("dim1", "dim2"), - ImmutableMap.of("dim1", "val5", "dim2", "val2", "met1", 2) - ), - new MapBasedInputRow( - DateTime.now(), Lists.newArrayList("dim2", "dim3"), - ImmutableMap.of("dim2", "val1", "dim3", "val2", "met1", 3) - ) - ); - - private ReplayableFirehoseFactory replayableFirehoseFactory; - - @Before - public void setup() - { - replayableFirehoseFactory = new ReplayableFirehoseFactory( - delegateFactory, - true, - 10000, - 3, - mapper - ); - } - - @Test - public void testConstructor() throws Exception - { - Assert.assertEquals(delegateFactory, replayableFirehoseFactory.getDelegateFactory()); - Assert.assertEquals(10000, replayableFirehoseFactory.getMaxTempFileSize()); - Assert.assertEquals(3, replayableFirehoseFactory.getReadFirehoseRetries()); - Assert.assertEquals(true, replayableFirehoseFactory.isReportParseExceptions()); - } - - @Test - public void testReplayableFirehoseNoEvents() throws Exception - { - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andReturn(false); - delegateFirehose.close(); - replayAll(); - - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - Assert.assertFalse(firehose.hasMore()); - } - verifyAll(); - } - - @Test - public void testReplayableFirehoseWithEvents() throws Exception - { - final boolean hasMore[] = {true}; - - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andAnswer( - new IAnswer() - { - @Override - public Boolean answer() throws Throwable - { - return hasMore[0]; - } - } - ).anyTimes(); - expect(delegateFirehose.nextRow()) - .andReturn(testRows.get(0)) - .andReturn(testRows.get(1)) - .andAnswer( - new IAnswer() - { - @Override - public InputRow answer() throws Throwable - { - hasMore[0] = false; - return testRows.get(2); - } - } - ); - - delegateFirehose.close(); - replayAll(); - - List rows = Lists.newArrayList(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRows, rows); - - // now replay! - rows.clear(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRows, rows); - - verifyAll(); - } - - @Test - public void testReplayableFirehoseWithoutReportParseExceptions() throws Exception - { - final boolean hasMore[] = {true}; - replayableFirehoseFactory = new ReplayableFirehoseFactory( - delegateFactory, - false, - 10000, - 3, - mapper - ); - - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andAnswer( - new IAnswer() - { - @Override - public Boolean answer() throws Throwable - { - return hasMore[0]; - } - } - ).anyTimes(); - expect(delegateFirehose.nextRow()) - .andReturn(testRows.get(0)) - .andReturn(testRows.get(1)) - .andThrow(new ParseException("unparseable!")) - .andAnswer( - new IAnswer() - { - @Override - public InputRow answer() throws Throwable - { - hasMore[0] = false; - return testRows.get(2); - } - } - ); - - delegateFirehose.close(); - replayAll(); - - List rows = Lists.newArrayList(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRows, rows); - - verifyAll(); - } - - @Test(expected = ParseException.class) - public void testReplayableFirehoseWithReportParseExceptions() throws Exception - { - final boolean hasMore[] = {true}; - - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andAnswer( - new IAnswer() - { - @Override - public Boolean answer() throws Throwable - { - return hasMore[0]; - } - } - ).anyTimes(); - expect(delegateFirehose.nextRow()) - .andReturn(testRows.get(0)) - .andReturn(testRows.get(1)) - .andThrow(new ParseException("unparseable!")) - .andAnswer( - new IAnswer() - { - @Override - public InputRow answer() throws Throwable - { - hasMore[0] = false; - return testRows.get(2); - } - } - ); - - delegateFirehose.close(); - replayAll(); - - replayableFirehoseFactory.connect(parser); - verifyAll(); - } - - @Test - public void testReplayableFirehoseWithConnectRetries() throws Exception - { - final boolean hasMore[] = {true}; - - expect(delegateFactory.connect(parser)).andThrow(new IOException()) - .andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andAnswer( - new IAnswer() - { - @Override - public Boolean answer() throws Throwable - { - return hasMore[0]; - } - } - ).anyTimes(); - expect(delegateFirehose.nextRow()) - .andReturn(testRows.get(0)) - .andReturn(testRows.get(1)) - .andAnswer( - new IAnswer() - { - @Override - public InputRow answer() throws Throwable - { - hasMore[0] = false; - return testRows.get(2); - } - } - ); - delegateFirehose.close(); - replayAll(); - - List rows = Lists.newArrayList(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRows, rows); - - verifyAll(); - } - - @Test - public void testReplayableFirehoseWithNextRowRetries() throws Exception - { - final boolean hasMore[] = {true}; - - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose).times(2); - expect(delegateFirehose.hasMore()).andAnswer( - new IAnswer() - { - @Override - public Boolean answer() throws Throwable - { - return hasMore[0]; - } - } - ).anyTimes(); - expect(delegateFirehose.nextRow()) - .andReturn(testRows.get(0)) - .andThrow(new RuntimeException()) - .andReturn(testRows.get(0)) - .andReturn(testRows.get(1)) - .andAnswer( - new IAnswer() - { - @Override - public InputRow answer() throws Throwable - { - hasMore[0] = false; - return testRows.get(2); - } - } - ); - delegateFirehose.close(); - expectLastCall().times(2); - replayAll(); - - List rows = Lists.newArrayList(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRows, rows); - - verifyAll(); - } - - @Test(expected = TestReadingException.class) - public void testReplayableFirehoseWithNoRetries() throws Exception - { - replayableFirehoseFactory = new ReplayableFirehoseFactory( - delegateFactory, - false, - 10000, - 0, - mapper - ); - - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andReturn(true).times(2); - expect(delegateFirehose.nextRow()).andThrow(new TestReadingException()); - - delegateFirehose.close(); - expectLastCall(); - replayAll(); - - replayableFirehoseFactory.connect(parser); - verifyAll(); - } - - @Test - public void testReplayableFirehoseWithMultipleFiles() throws Exception - { - replayableFirehoseFactory = new ReplayableFirehoseFactory(delegateFactory, false, 1, 3, mapper); - - final boolean hasMore[] = {true}; - final int multiplicationFactor = 500; - - final InputRow finalRow = new MapBasedInputRow( - DateTime.now(), Lists.newArrayList("dim4", "dim5"), - ImmutableMap.of("dim4", "val12", "dim5", "val20", "met1", 30) - ); - - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andAnswer( - new IAnswer() - { - @Override - public Boolean answer() throws Throwable - { - return hasMore[0]; - } - } - ).anyTimes(); - - expect(delegateFirehose.nextRow()) - .andReturn(testRows.get(0)).times(multiplicationFactor) - .andReturn(testRows.get(1)).times(multiplicationFactor) - .andReturn(testRows.get(2)).times(multiplicationFactor) - .andAnswer( - new IAnswer() - { - @Override - public InputRow answer() throws Throwable - { - hasMore[0] = false; - return finalRow; - } - } - ); - - delegateFirehose.close(); - replayAll(); - - List testRowsMultiplied = Lists.newArrayList(); - for (InputRow row : testRows) { - for (int i = 0; i < multiplicationFactor; i++) { - testRowsMultiplied.add(row); - } - } - testRowsMultiplied.add(finalRow); - - List rows = Lists.newArrayList(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRowsMultiplied, rows); - - // now replay! - rows.clear(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRowsMultiplied, rows); - - verifyAll(); - } - - private class TestReadingException extends RuntimeException - { - } -} - - - From aa5f28d87e9bf157dd628141bdbf01d8faa91d9c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 26 Apr 2017 11:13:32 +0900 Subject: [PATCH 06/26] Defer object initialization --- .../AbstractTextFilesFirehoseFactory.java | 20 +++--- .../PrefetchableTextFilesFirehoseFactory.java | 18 +++-- .../input/impl/FileIteratingFirehoseTest.java | 69 +++++++++++++++---- ...fetchableTextFilesFirehoseFactoryTest.java | 20 ++++-- .../StaticAzureBlobStoreFirehoseFactory.java | 13 +++- .../StaticCloudFilesFirehoseFactory.java | 13 +++- .../StaticGoogleBlobStoreFirehoseFactory.java | 13 +++- .../firehose/s3/StaticS3FirehoseFactory.java | 13 +++- .../indexing/common/task/TaskSerdeTest.java | 4 -- .../overlord/RemoteTaskRunnerTest.java | 7 -- .../indexing/worker/TaskAnnouncementTest.java | 4 -- .../firehose/LocalFirehoseFactory.java | 26 +++---- .../firehose/LocalFirehoseFactoryTest.java | 15 ---- .../cli/validate/DruidJsonValidatorTest.java | 4 -- 14 files changed, 144 insertions(+), 95 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java index 881c6dadf39a..b187dcbcd030 100644 --- a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java @@ -49,16 +49,10 @@ public abstract class AbstractTextFilesFirehoseFactory { private static final Logger LOG = new Logger(AbstractTextFilesFirehoseFactory.class); - private final List objects; - - public AbstractTextFilesFirehoseFactory(Collection objects) - { - this.objects = ImmutableList.copyOf(Preconditions.checkNotNull(objects, "objects")); - } - @Override public Firehose connect(StringInputRowParser firehoseParser) throws IOException { + final List objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects")); final Iterator iterator = objects.iterator(); return new FileIteratingFirehose( new Iterator() @@ -97,10 +91,14 @@ public LineIterator next() ); } - public List getObjects() - { - return objects; - } + /** + * Initialize objects to be read by this firehose. Since firehose factories are constructed whenever + * io.druid.indexing.common.task.Task objects are deserialized, actual initialization of objects is deferred + * until {@link #connect(StringInputRowParser)} is called. + * + * @return a collection of initialized objects. + */ + protected abstract Collection initObjects(); /** * Open an input stream from the given object. diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 3f856877f3ce..c77e26b15a5a 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -20,7 +20,9 @@ package io.druid.data.input.impl; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.io.CountingOutputStream; import com.google.common.io.Files; import io.druid.data.input.Firehose; @@ -40,7 +42,6 @@ import java.io.InputStreamReader; import java.io.Reader; import java.util.ArrayList; -import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -111,7 +112,6 @@ public abstract class PrefetchableTextFilesFirehoseFactory private Future fetchFuture; public PrefetchableTextFilesFirehoseFactory( - Collection objects, Long maxCacheCapacityBytes, Long maxFetchCapacityBytes, Long prefetchTriggerBytes, @@ -119,7 +119,6 @@ public PrefetchableTextFilesFirehoseFactory( Integer maxFetchRetry ) { - super(objects); this.maxCacheCapacityBytes = maxCacheCapacityBytes == null ? DEFAULT_MAX_CACHE_CAPACITY_BYTES : maxCacheCapacityBytes; @@ -145,10 +144,9 @@ List getCacheFiles() /** * Cache objects in a local disk up to {@link #maxCacheCapacityBytes}. */ - private void cache() + private void cache(final List objects) { double totalFetchedBytes = 0; - final List objects = getObjects(); try { for (int i = 0; i < objects.size() && totalFetchedBytes < maxCacheCapacityBytes; i++) { final ObjectType object = objects.get(i); @@ -170,9 +168,8 @@ private void cache() * This is for simplifying design, and should be improved when our client implementations for cloud storages like S3 * support range scan. */ - private void fetch() throws Exception + private void fetch(final List objects) throws Exception { - final List objects = getObjects(); for (int i = nextFetchIndex; i < objects.size() && fetchedBytes.get() <= maxFetchCapacityBytes; i++) { final ObjectType object = objects.get(i); LOG.info("Fetching object[%s]", object); @@ -216,10 +213,11 @@ private long download(ObjectType object, File outFile, int retry) throws IOExcep @Override public Firehose connect(StringInputRowParser firehoseParser) throws IOException { + final List objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "objects")); if (baseDir == null) { baseDir = Files.createTempDir(); baseDir.deleteOnExit(); - cache(); + cache(objects); } else { nextFetchIndex = cacheFiles.size(); } @@ -244,7 +242,7 @@ public boolean hasNext() { return cacheFileIterator.hasNext() || !fetchFiles.isEmpty() - || nextFetchIndex < getObjects().size(); + || nextFetchIndex < objects.size(); } private void fetchIfNeeded(long remainingBytes) @@ -253,7 +251,7 @@ private void fetchIfNeeded(long remainingBytes) && remainingBytes <= prefetchTriggerBytes) { fetchFuture = fetchExecutor.submit( () -> { - fetch(); + fetch(objects); return null; } ); diff --git a/api/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java b/api/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java index 87f5d20fcd5c..a6916c4094b8 100644 --- a/api/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java +++ b/api/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java @@ -23,13 +23,14 @@ import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; - import io.druid.java.util.common.Pair; -import junit.framework.Assert; - import org.apache.commons.io.LineIterator; +import org.junit.Assert; import org.junit.Test; +import java.io.Closeable; +import java.io.IOException; +import java.io.Reader; import java.io.StringReader; import java.util.Arrays; import java.util.List; @@ -46,6 +47,16 @@ public class FileIteratingFirehoseTest Pair.of(new String[]{}, ImmutableList.of()) ); + private static final StringInputRowParser parser = new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec("ts", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x")), null, null), + ",", + ImmutableList.of("ts", "x") + ), + null + ); + @Test public void testFirehose() throws Exception { @@ -62,16 +73,6 @@ public LineIterator apply(String s) } ); - final StringInputRowParser parser = new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("ts", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x")), null, null), - ",", - ImmutableList.of("ts", "x") - ), - null - ); - final FileIteratingFirehose firehose = new FileIteratingFirehose(lineIterators.iterator(), parser); final List results = Lists.newArrayList(); @@ -82,4 +83,46 @@ public LineIterator apply(String s) Assert.assertEquals(fixture.rhs, results); } } + + @Test(expected = RuntimeException.class) + public void testClose() throws IOException + { + final LineIterator lineIterator = new LineIterator(new Reader() + { + @Override + public int read(char[] cbuf, int off, int len) throws IOException + { + final char[] chs = "\n".toCharArray(); + System.arraycopy(chs, 0, cbuf, 0, chs.length); + return chs.length; + } + + @Override + public void close() throws IOException + { + throw new RuntimeException("close test for FileIteratingFirehose"); + } + }); + + final TestCloseable closeable = new TestCloseable(); + final FileIteratingFirehose firehose = new FileIteratingFirehose( + ImmutableList.of(lineIterator).iterator(), + parser, + closeable + ); + firehose.hasMore(); // initialize lineIterator + firehose.close(); + Assert.assertTrue(closeable.closed); + } + + private static final class TestCloseable implements Closeable + { + private boolean closed; + + @Override + public void close() throws IOException + { + closed = true; + } + } } diff --git a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java index e940f0a6dc23..cd203e29ba28 100644 --- a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java @@ -43,6 +43,7 @@ import java.io.Writer; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.concurrent.TimeoutException; @@ -242,7 +243,8 @@ public void testReconnect() throws IOException static class TestPrefetchableTextFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory { - private long sleepMillis; + private final long sleepMillis; + private final File baseDir; private int openExceptionCount; static TestPrefetchableTextFilesFirehoseFactory withoutCache(File baseDir) @@ -327,11 +329,6 @@ public TestPrefetchableTextFilesFirehoseFactory( ) { super( - FileUtils.listFiles( - Preconditions.checkNotNull(baseDir).getAbsoluteFile(), - TrueFileFilter.INSTANCE, - TrueFileFilter.INSTANCE - ), maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerThreshold, @@ -340,6 +337,17 @@ public TestPrefetchableTextFilesFirehoseFactory( ); this.openExceptionCount = openExceptionCount; this.sleepMillis = sleepMillis; + this.baseDir = baseDir; + } + + @Override + protected Collection initObjects() + { + return FileUtils.listFiles( + Preconditions.checkNotNull(baseDir).getAbsoluteFile(), + TrueFileFilter.INSTANCE, + TrueFileFilter.INSTANCE + ); } @Override diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java index 8b2728d7843d..899f15893f05 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java @@ -29,6 +29,7 @@ import java.io.IOException; import java.io.InputStream; +import java.util.Collection; import java.util.List; /** @@ -37,6 +38,7 @@ public class StaticAzureBlobStoreFirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private final AzureStorage azureStorage; + private final List blobs; @JsonCreator public StaticAzureBlobStoreFirehoseFactory( @@ -49,14 +51,21 @@ public StaticAzureBlobStoreFirehoseFactory( @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { - super(blobs, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); + super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); + this.blobs = blobs; this.azureStorage = azureStorage; } @JsonProperty public List getBlobs() { - return getObjects(); + return blobs; + } + + @Override + protected Collection initObjects() + { + return blobs; } @Override diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java index 1672a225cd49..ff38f425af81 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java @@ -31,6 +31,7 @@ import java.io.IOException; import java.io.InputStream; +import java.util.Collection; import java.util.List; public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory @@ -38,6 +39,7 @@ public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFireho private static final Logger log = new Logger(StaticCloudFilesFirehoseFactory.class); private final CloudFilesApi cloudFilesApi; + private final List blobs; @JsonCreator public StaticCloudFilesFirehoseFactory( @@ -50,14 +52,21 @@ public StaticCloudFilesFirehoseFactory( @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { - super(blobs, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); + super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); this.cloudFilesApi = cloudFilesApi; + this.blobs = blobs; } @JsonProperty public List getBlobs() { - return getObjects(); + return blobs; + } + + @Override + protected Collection initObjects() + { + return blobs; } @Override diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java index d99d7d5b7e36..c95f88adecbd 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java @@ -29,11 +29,13 @@ import java.io.IOException; import java.io.InputStream; +import java.util.Collection; import java.util.List; public class StaticGoogleBlobStoreFirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private final GoogleStorage storage; + private final List blobs; @JsonCreator public StaticGoogleBlobStoreFirehoseFactory( @@ -46,13 +48,20 @@ public StaticGoogleBlobStoreFirehoseFactory( @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { - super(blobs, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); + super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); this.storage = storage; + this.blobs = blobs; } @JsonProperty public List getBlobs() { - return getObjects(); + return blobs; + } + + @Override + protected Collection initObjects() + { + return blobs; } @Override diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index 8093f60649df..78a6569ccd61 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -34,6 +34,7 @@ import java.io.IOException; import java.io.InputStream; import java.net.URI; +import java.util.Collection; import java.util.List; /** @@ -44,6 +45,7 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor private static final Logger log = new Logger(StaticS3FirehoseFactory.class); private final RestS3Service s3Client; + private final List uris; @JsonCreator public StaticS3FirehoseFactory( @@ -56,8 +58,9 @@ public StaticS3FirehoseFactory( @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { - super(uris, maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); + super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); this.s3Client = Preconditions.checkNotNull(s3Client, "null s3Client"); + this.uris = uris; for (final URI inputURI : uris) { Preconditions.checkArgument(inputURI.getScheme().equals("s3"), "input uri scheme == s3 (%s)", inputURI); @@ -67,7 +70,13 @@ public StaticS3FirehoseFactory( @JsonProperty public List getUris() { - return getObjects(); + return uris; + } + + @Override + protected Collection initObjects() + { + return uris; } @Override diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 4fcc4fea3db1..515b3913f2f5 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -77,10 +77,6 @@ public TaskSerdeTest() for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { jsonMapper.registerModule(jacksonModule); } - - final File baseDir = new File("lol"); - baseDir.mkdir(); - baseDir.deleteOnExit(); } @Test diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java index 1574ea7650d8..3aee1a97c911 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java @@ -49,7 +49,6 @@ import org.junit.Before; import org.junit.Test; -import java.io.File; import java.util.ArrayList; import java.util.Collection; import java.util.Set; @@ -72,8 +71,6 @@ public class RemoteTaskRunnerTest private Task task; private Worker worker; - private File baseDir = new File("lol"); - @Before public void setUp() throws Exception { @@ -82,9 +79,6 @@ public void setUp() throws Exception cf = rtrTestUtils.getCuratorFramework(); task = TestTasks.unending("task"); - - final File baseDir = new File("lol"); - baseDir.mkdir(); } @After @@ -94,7 +88,6 @@ public void tearDown() throws Exception remoteTaskRunner.stop(); } rtrTestUtils.tearDown(); - baseDir.delete(); } @Test diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java index c88f0d09af36..1d160e99bff6 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java @@ -49,10 +49,6 @@ public TaskAnnouncementTest() { TestUtils testUtils = new TestUtils(); jsonMapper = testUtils.getTestObjectMapper(); - - final File baseDir = new File("lol"); - baseDir.mkdir(); - baseDir.deleteOnExit(); } @Test diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java index 3fccd0564a8c..4c4e5af3cc2e 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java @@ -25,6 +25,7 @@ import com.metamx.emitter.EmittingLogger; import io.druid.data.input.impl.AbstractTextFilesFirehoseFactory; import io.druid.data.input.impl.StringInputRowParser; +import io.druid.java.util.common.CompressionUtils; import org.apache.commons.io.FileUtils; import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.commons.io.filefilter.WildcardFileFilter; @@ -32,6 +33,7 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.util.Collection; /** */ @@ -51,18 +53,9 @@ public LocalFirehoseFactory( @JsonProperty("parser") StringInputRowParser parser ) { - super( - FileUtils.listFiles( - Preconditions.checkNotNull(baseDir).getAbsoluteFile(), - new WildcardFileFilter(filter), - TrueFileFilter.INSTANCE - ) - ); this.baseDir = baseDir; this.filter = filter; this.parser = parser; - - log.info("Initialized with " + getObjects() + " files"); } @JsonProperty @@ -84,14 +77,21 @@ public StringInputRowParser getParser() } @Override - protected InputStream openStream(File object) throws IOException + protected Collection initObjects() { - return FileUtils.openInputStream(object); + final Collection files = FileUtils.listFiles( + Preconditions.checkNotNull(baseDir).getAbsoluteFile(), + new WildcardFileFilter(filter), + TrueFileFilter.INSTANCE + ); + log.info("Initialized with " + files + " files"); + return files; } @Override - protected boolean isGzipped(File object) + protected InputStream openStream(File object) throws IOException { - return object.getPath().endsWith(".gz"); + final InputStream stream = FileUtils.openInputStream(object); + return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } } diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java index 9161b8806593..f09eb88401b0 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java @@ -34,7 +34,6 @@ import org.junit.rules.TemporaryFolder; import java.io.BufferedWriter; -import java.io.File; import java.io.FileWriter; import java.io.IOException; import java.io.Writer; @@ -72,20 +71,6 @@ public void setup() throws IOException factory = new LocalFirehoseFactory(temporaryFolder.getRoot(), "test_*", null); } - @Test - public void testGetObjects() throws InterruptedException - { - final List files = factory.getObjects(); - Assert.assertEquals(5, files.size()); - } - - @Test - public void testIsGzipped() - { - Assert.assertTrue(factory.isGzipped(new File("test.gz"))); - Assert.assertFalse(factory.isGzipped(new File("test.g"))); - } - @Test public void testConnect() throws IOException { diff --git a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java index 37f7a3556c2b..b83a93bdbe9d 100644 --- a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java @@ -141,9 +141,6 @@ public void testTaskValidator() throws Exception jsonMapper.registerModule(jacksonModule); } - final File file = new File("lol"); - file.mkdir(); - final RealtimeIndexTask task = new RealtimeIndexTask( null, new TaskResource("rofl", 2), @@ -194,7 +191,6 @@ public Plumber findPlumber( jsonMapper.writeValue(tmp, task); parseCommand("validator", "-f", tmp.getAbsolutePath(), "-t", "task").run(); - file.delete(); } @After From 7ea130425efdab26d659ebbf3896f2fe34f931cf Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 26 Apr 2017 14:40:45 +0900 Subject: [PATCH 07/26] Add a temporaryDirectory parameter to FirehoseFactory.connect() --- .../io/druid/data/input/FirehoseFactory.java | 11 ++++- .../AbstractTextFilesFirehoseFactory.java | 7 ++-- .../PrefetchableTextFilesFirehoseFactory.java | 42 +++++++++++++------ ...fetchableTextFilesFirehoseFactoryTest.java | 24 +++++++---- .../TwitterSpritzerFirehoseFactory.java | 3 +- .../rocketmq/RocketMQFirehoseFactory.java | 6 ++- .../rabbitmq/RabbitMQFirehoseFactory.java | 3 +- .../kafka/KafkaEightFirehoseFactory.java | 3 +- .../druid/indexing/common/task/IndexTask.java | 33 +++++++++++---- .../druid/indexing/common/task/NoopTask.java | 4 +- .../common/task/RealtimeIndexTask.java | 8 +++- .../IngestSegmentFirehoseFactory.java | 2 +- .../common/task/RealtimeIndexTaskTest.java | 2 +- .../IngestSegmentFirehoseFactoryTest.java | 2 +- ...estSegmentFirehoseFactoryTimelineTest.java | 2 +- .../indexing/overlord/TaskLifecycleTest.java | 4 +- .../segment/realtime/FireDepartment.java | 5 ++- .../segment/realtime/RealtimeManager.java | 24 +++++++---- .../firehose/ClippedFirehoseFactory.java | 5 ++- .../firehose/CombiningFirehoseFactory.java | 13 +++--- .../EventReceiverFirehoseFactory.java | 3 +- .../firehose/FixedCountFirehoseFactory.java | 5 ++- .../realtime/firehose/IrcFirehoseFactory.java | 5 +-- .../firehose/TimedShutoffFirehoseFactory.java | 11 ++--- .../CombiningFirehoseFactoryTest.java | 5 ++- .../segment/realtime/RealtimeManagerTest.java | 3 +- .../firehose/EventReceiverFirehoseTest.java | 7 ++-- .../firehose/LocalFirehoseFactoryTest.java | 2 +- 28 files changed, 160 insertions(+), 84 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/FirehoseFactory.java b/api/src/main/java/io/druid/data/input/FirehoseFactory.java index 43cbf01111f9..8579ccd9067d 100644 --- a/api/src/main/java/io/druid/data/input/FirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/FirehoseFactory.java @@ -20,10 +20,11 @@ package io.druid.data.input; import com.fasterxml.jackson.annotation.JsonTypeInfo; - import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.parsers.ParseException; +import java.io.File; import java.io.IOException; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @@ -36,7 +37,13 @@ public interface FirehoseFactory * If this method returns null, then any attempt to call hasMore(), nextRow(), commit() and close() on the return * value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on * invalid configuration is preferred over returning null. + *

      + * Some fire hoses like {@link PrefetchableTextFilesFirehoseFactory} may use a temporary + * directory to cache data in it. + * + * @param parser an input row parser + * @param temporaryDirectory a directory where temporary files are stored */ - public Firehose connect(T parser) throws IOException, ParseException; + Firehose connect(T parser, File temporaryDirectory) throws IOException, ParseException; } diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java index b187dcbcd030..cd32371f779b 100644 --- a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java @@ -30,6 +30,7 @@ import org.apache.commons.io.LineIterator; import java.io.BufferedReader; +import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; @@ -40,7 +41,7 @@ /** * This is an abstract class for firehose factory for making firehoses reading text files. - * It provides an unified {@link #connect(StringInputRowParser)} implementation for its subclasses. + * It provides an unified {@link #connect(StringInputRowParser, File)} implementation for its subclasses. * * @param object type representing input data */ @@ -50,7 +51,7 @@ public abstract class AbstractTextFilesFirehoseFactory private static final Logger LOG = new Logger(AbstractTextFilesFirehoseFactory.class); @Override - public Firehose connect(StringInputRowParser firehoseParser) throws IOException + public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirectory) throws IOException { final List objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects")); final Iterator iterator = objects.iterator(); @@ -94,7 +95,7 @@ public LineIterator next() /** * Initialize objects to be read by this firehose. Since firehose factories are constructed whenever * io.druid.indexing.common.task.Task objects are deserialized, actual initialization of objects is deferred - * until {@link #connect(StringInputRowParser)} is called. + * until {@link #connect(StringInputRowParser, File)} is called. * * @return a collection of initialized objects. */ diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index c77e26b15a5a..3f52a9bacc3b 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -24,7 +24,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.io.CountingOutputStream; -import com.google.common.io.Files; import io.druid.data.input.Firehose; import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.logger.Logger; @@ -59,7 +58,7 @@ * by this class provides three key functionalities. * *

        - *
      • Caching: for the first call of {@link #connect(StringInputRowParser)}, it caches objects in a local disk + *
      • Caching: for the first call of {@link #connect(StringInputRowParser, File)}, it caches objects in a local disk * up to {@link #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 @@ -82,6 +81,8 @@ public abstract class PrefetchableTextFilesFirehoseFactory private static final long DEFAULT_MAX_FETCH_CAPACITY_BYTES = 1024 * 1024 * 1024; // 1GB private static final long DEFAULT_FETCH_TIMEOUT = 60_000; // 60 secs private static final int DEFAULT_MAX_FETCH_RETRY = 3; + private static final String CACHE_FILE_PREFIX = "cache-"; + private static final String FETCH_FILE_PREFIX = "fetch-"; // The below two variables are roughly the max size of total cached/fetched objects, but the actual cached/fetched // size can be larger. The reason is our current client implementations for cloud storages like s3 don't support range @@ -92,7 +93,6 @@ public abstract class PrefetchableTextFilesFirehoseFactory private final long maxFetchCapacityBytes; private final long prefetchTriggerBytes; - private File baseDir; // Directory for cached and fetched files private final List cacheFiles; private final LinkedBlockingQueue fetchFiles; @@ -111,6 +111,8 @@ public abstract class PrefetchableTextFilesFirehoseFactory private Future fetchFuture; + private boolean needCache = true; + public PrefetchableTextFilesFirehoseFactory( Long maxCacheCapacityBytes, Long maxFetchCapacityBytes, @@ -144,14 +146,14 @@ List getCacheFiles() /** * Cache objects in a local disk up to {@link #maxCacheCapacityBytes}. */ - private void cache(final List objects) + private void cache(final List objects, File baseDir) { double totalFetchedBytes = 0; try { for (int i = 0; i < objects.size() && totalFetchedBytes < maxCacheCapacityBytes; i++) { final ObjectType object = objects.get(i); LOG.info("Caching object[%s]", object); - final File outFile = File.createTempFile("cache-", null, baseDir); + final File outFile = File.createTempFile(CACHE_FILE_PREFIX, null, baseDir); totalFetchedBytes += download(object, outFile, 0); cacheFiles.add(outFile); nextFetchIndex++; @@ -168,12 +170,12 @@ private void cache(final List objects) * This is for simplifying design, and should be improved when our client implementations for cloud storages like S3 * support range scan. */ - private void fetch(final List objects) throws Exception + private void fetch(final List objects, File baseDir) throws Exception { for (int i = nextFetchIndex; i < objects.size() && fetchedBytes.get() <= maxFetchCapacityBytes; i++) { final ObjectType object = objects.get(i); LOG.info("Fetching object[%s]", object); - final File outFile = File.createTempFile("fetch-", null, baseDir); + final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, baseDir); fetchedBytes.addAndGet(download(object, outFile, 0)); fetchFiles.put(outFile); nextFetchIndex++; @@ -211,13 +213,27 @@ private long download(ObjectType object, File outFile, int retry) throws IOExcep } @Override - public Firehose connect(StringInputRowParser firehoseParser) throws IOException + public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirectory) throws IOException { final List objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "objects")); - if (baseDir == null) { - baseDir = Files.createTempDir(); - baseDir.deleteOnExit(); - cache(objects); +// if (baseDir == null) { +// baseDir = Files.createTempDir(); +// baseDir.deleteOnExit(); +// cache(objects); +// } else { +// nextFetchIndex = cacheFiles.size(); +// } + + Preconditions.checkState(temporaryDirectory.exists(), "temporaryDirectory[%s] does not exist", temporaryDirectory); + Preconditions.checkState( + temporaryDirectory.isDirectory(), + "temporaryDirectory[%s] is not a directory", + temporaryDirectory + ); + + if (needCache) { + cache(objects, temporaryDirectory); + needCache = false; } else { nextFetchIndex = cacheFiles.size(); } @@ -251,7 +267,7 @@ private void fetchIfNeeded(long remainingBytes) && remainingBytes <= prefetchTriggerBytes) { fetchFuture = fetchExecutor.submit( () -> { - fetch(objects); + fetch(objects, temporaryDirectory); return null; } ); diff --git a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java index cd203e29ba28..1ffcd192e5ab 100644 --- a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java @@ -50,8 +50,9 @@ public class PrefetchableTextFilesFirehoseFactoryTest { private static File testDir; + private static File firehoseTempDir; - private final StringInputRowParser parser = new StringInputRowParser( + private static final StringInputRowParser parser = new StringInputRowParser( new CSVParseSpec( new TimestampSpec( "timestamp", @@ -75,10 +76,14 @@ public class PrefetchableTextFilesFirehoseFactoryTest @BeforeClass public static void setup() throws IOException { - testDir = File.createTempFile(PrefetchableTextFilesFirehoseFactoryTest.class.getSimpleName(), ""); + testDir = File.createTempFile(PrefetchableTextFilesFirehoseFactoryTest.class.getSimpleName(), "testDir"); FileUtils.forceDelete(testDir); FileUtils.forceMkdir(testDir); + firehoseTempDir = File.createTempFile(PrefetchableTextFilesFirehoseFactoryTest.class.getSimpleName(), "baseDir"); + FileUtils.forceDelete(firehoseTempDir); + FileUtils.forceMkdir(firehoseTempDir); + for (int i = 0; i < 10; i++) { // Each file is 1390 bytes try (final Writer writer = new BufferedWriter( @@ -96,6 +101,7 @@ public static void setup() throws IOException public static void teardown() throws IOException { FileUtils.forceDelete(testDir); + FileUtils.forceDelete(firehoseTempDir); } private static void assertResult(List rows) @@ -131,7 +137,7 @@ public void testWithoutCache() throws IOException TestPrefetchableTextFilesFirehoseFactory.withoutCache(testDir); final List rows = new ArrayList<>(); - try (Firehose firehose = factory.connect(parser)) { + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { Assert.assertEquals(0, factory.getCacheFiles().size()); while (firehose.hasMore()) { rows.add(firehose.nextRow()); @@ -148,7 +154,7 @@ public void testWithZeroFetchCapacity() throws IOException TestPrefetchableTextFilesFirehoseFactory.withZeroFetchCapacity(testDir); final List rows = new ArrayList<>(); - try (Firehose firehose = factory.connect(parser)) { + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { while (firehose.hasMore()) { rows.add(firehose.nextRow()); } @@ -164,7 +170,7 @@ public void testWithCacheAndFetch() throws IOException TestPrefetchableTextFilesFirehoseFactory.of(testDir); final List rows = new ArrayList<>(); - try (Firehose firehose = factory.connect(parser)) { + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { while (firehose.hasMore()) { rows.add(firehose.nextRow()); } @@ -180,7 +186,7 @@ public void testRetry() throws IOException TestPrefetchableTextFilesFirehoseFactory.withOpenExceptions(testDir, 1); final List rows = new ArrayList<>(); - try (Firehose firehose = factory.connect(parser)) { + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { while (firehose.hasMore()) { rows.add(firehose.nextRow()); } @@ -200,7 +206,7 @@ public void testMaxRetry() throws IOException TestPrefetchableTextFilesFirehoseFactory.withOpenExceptions(testDir, 5); final List rows = new ArrayList<>(); - try (Firehose firehose = factory.connect(parser)) { + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { while (firehose.hasMore()) { rows.add(firehose.nextRow()); } @@ -217,7 +223,7 @@ public void testTimeout() throws IOException TestPrefetchableTextFilesFirehoseFactory.withSleepMillis(testDir, 1000); final List rows = new ArrayList<>(); - try (Firehose firehose = factory.connect(parser)) { + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { while (firehose.hasMore()) { rows.add(firehose.nextRow()); } @@ -232,7 +238,7 @@ public void testReconnect() throws IOException for (int i = 0; i < 5; i++) { final List rows = new ArrayList<>(); - try (Firehose firehose = factory.connect(parser)) { + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { while (firehose.hasMore()) { rows.add(firehose.nextRow()); } diff --git a/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java b/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java index 9333e58126ee..4ac888fbb243 100644 --- a/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java +++ b/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java @@ -43,6 +43,7 @@ import twitter4j.User; import javax.annotation.Nullable; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -113,7 +114,7 @@ public TwitterSpritzerFirehoseFactory( } @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { final ConnectionLifeCycleListener connectionLifeCycleListener = new ConnectionLifeCycleListener() { diff --git a/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java b/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java index 94b7bb79cde8..279aa3eb4882 100644 --- a/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java +++ b/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java @@ -42,6 +42,7 @@ import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.ParseException; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -137,7 +138,10 @@ private boolean hasMessagesPending() } @Override - public Firehose connect(ByteBufferInputRowParser byteBufferInputRowParser) throws IOException, ParseException + public Firehose connect( + ByteBufferInputRowParser byteBufferInputRowParser, + File temporaryDirectory + ) throws IOException, ParseException { Set newDimExclus = Sets.union( diff --git a/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java b/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java index b48083fe2991..66c91770a2b7 100644 --- a/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java +++ b/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java @@ -41,6 +41,7 @@ import net.jodah.lyra.retry.RetryPolicy; import net.jodah.lyra.util.Duration; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.concurrent.BlockingQueue; @@ -134,7 +135,7 @@ public JacksonifiedConnectionFactory getConnectionFactory() } @Override - public Firehose connect(final ByteBufferInputRowParser firehoseParser) throws IOException + public Firehose connect(final ByteBufferInputRowParser firehoseParser, File temporaryDirectory) throws IOException { ConnectionOptions lyraOptions = new ConnectionOptions(this.connectionFactory); Config lyraConfig = new Config() diff --git a/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java b/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java index 3a87e15b2adc..dd63153ce92c 100644 --- a/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java +++ b/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java @@ -37,6 +37,7 @@ import kafka.javaapi.consumer.ConsumerConnector; import kafka.message.InvalidMessageException; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.List; @@ -68,7 +69,7 @@ public KafkaEightFirehoseFactory( } @Override - public Firehose connect(final ByteBufferInputRowParser firehoseParser) throws IOException + public Firehose connect(final ByteBufferInputRowParser firehoseParser, File temporaryDirectory) throws IOException { Set newDimExclus = Sets.union( firehoseParser.getParseSpec().getDimensionsSpec().getDimensionExclusions(), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 0c20106ef72a..5e2cdb128a8a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -38,6 +38,7 @@ import com.google.common.collect.Maps; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; +import com.google.common.io.Files; import io.druid.common.utils.JodaUtils; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; @@ -88,6 +89,7 @@ import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.ShardSpec; import io.druid.timeline.partition.ShardSpecLookup; +import org.codehaus.plexus.util.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; @@ -162,6 +164,9 @@ public IndexIngestionSpec getIngestionSchema() @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { + final File firehoseTempDir = Files.createTempDir(); + FileUtils.forceMkdir(firehoseTempDir); + final boolean determineIntervals = !ingestionSchema.getDataSchema() .getGranularitySpec() .bucketIntervals() @@ -174,7 +179,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ((IngestSegmentFirehoseFactory) firehoseFactory).setTaskToolbox(toolbox); } - final Map> shardSpecs = determineShardSpecs(toolbox, firehoseFactory); + final Map> shardSpecs = determineShardSpecs(toolbox, firehoseFactory, firehoseTempDir); final String version; final DataSchema dataSchema; @@ -196,10 +201,15 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception dataSchema = ingestionSchema.getDataSchema(); } - if (generateAndPublishSegments(toolbox, dataSchema, shardSpecs, version, firehoseFactory)) { - return TaskStatus.success(getId()); - } else { - return TaskStatus.failure(getId()); + try { + if (generateAndPublishSegments(toolbox, dataSchema, shardSpecs, version, firehoseFactory, firehoseTempDir)) { + return TaskStatus.success(getId()); + } else { + return TaskStatus.failure(getId()); + } + } + finally { + FileUtils.forceDelete(firehoseTempDir); } } @@ -209,7 +219,8 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception */ private Map> determineShardSpecs( final TaskToolbox toolbox, - final FirehoseFactory firehoseFactory + final FirehoseFactory firehoseFactory, + final File firehoseTempDir ) throws IOException { final ObjectMapper jsonMapper = toolbox.getObjectMapper(); @@ -253,7 +264,10 @@ private Map> determineShardSpecs( log.info("Determining intervals and shardSpecs"); long determineShardSpecsStartMillis = System.currentTimeMillis(); - try (final Firehose firehose = firehoseFactory.connect(ingestionSchema.getDataSchema().getParser())) { + try (final Firehose firehose = firehoseFactory.connect( + ingestionSchema.getDataSchema().getParser(), + firehoseTempDir) + ) { while (firehose.hasMore()) { final InputRow inputRow = firehose.nextRow(); @@ -333,7 +347,8 @@ private boolean generateAndPublishSegments( final DataSchema dataSchema, final Map> shardSpecs, final String version, - final FirehoseFactory firehoseFactory + final FirehoseFactory firehoseFactory, + final File firehoseTempDir ) throws IOException, InterruptedException { @@ -386,7 +401,7 @@ public SegmentIdentifier allocate(DateTime timestamp, String sequenceName, Strin segmentAllocator, fireDepartmentMetrics ); - final Firehose firehose = firehoseFactory.connect(dataSchema.getParser()) + final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) ) { final Supplier committerSupplier = Committers.supplierFromFirehose(firehose); final Map shardSpecLookups = Maps.newHashMap(); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java index 0eac7c88ec2a..aa041f09e335 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; - import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.indexing.common.TaskStatus; @@ -30,7 +29,6 @@ import io.druid.indexing.common.actions.TaskActionClient; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; - import org.joda.time.DateTime; import java.util.Map; @@ -139,7 +137,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception if (firehoseFactory != null) { log.info("Connecting firehose"); } - try (Firehose firehose = firehoseFactory != null ? firehoseFactory.connect(null) : null) { + try (Firehose firehose = firehoseFactory != null ? firehoseFactory.connect(null, null) : null) { log.info("Running noop task[%s]", getId()); log.info("Sleeping for %,d millis.", runTime); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 60bf62a9d3af..f48fba1bdc73 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -26,6 +26,7 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.io.Files; import com.google.common.primitives.Ints; import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Committer; @@ -63,6 +64,7 @@ import io.druid.segment.realtime.plumber.VersioningPolicy; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.timeline.DataSegment; +import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -324,6 +326,7 @@ public String getVersion(final Interval interval) this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, metrics); Supplier committerSupplier = null; + final File firehoseTempDir = Files.createTempDir(); try { plumber.startJob(); @@ -332,13 +335,14 @@ public String getVersion(final Interval interval) toolbox.getMonitorScheduler().addMonitor(metricsMonitor); // Delay firehose connection to avoid claiming input resources while the plumber is starting up. + FileUtils.forceMkdir(firehoseTempDir); final FirehoseFactory firehoseFactory = spec.getIOConfig().getFirehoseFactory(); final boolean firehoseDrainableByClosing = isFirehoseDrainableByClosing(firehoseFactory); // Skip connecting firehose if we've been stopped before we got started. synchronized (this) { if (!gracefullyStopped) { - firehose = firehoseFactory.connect(spec.getDataSchema().getParser()); + firehose = firehoseFactory.connect(spec.getDataSchema().getParser(), firehoseTempDir); committerSupplier = Committers.supplierFromFirehose(firehose); } } @@ -410,6 +414,8 @@ public void run() plumber.finishJob(); } } + + FileUtils.forceDelete(firehoseTempDir); } catch (InterruptedException e) { log.debug(e, "Interrupted while finishing the job"); diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 25028f0f5c0f..11bc9bdfa26c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -127,7 +127,7 @@ public void setTaskToolbox(TaskToolbox taskToolbox) } @Override - public Firehose connect(InputRowParser inputRowParser) throws IOException, ParseException + public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) throws IOException, ParseException { log.info("Connecting firehose: dataSource[%s], interval[%s]", dataSource, interval); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 7f4464f7d540..15e6bc8a7989 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -219,7 +219,7 @@ public TestFirehoseFactory() } @Override - public Firehose connect(InputRowParser parser) throws IOException, ParseException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException, ParseException { return new TestFirehose(); } diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index b7431f6f3abd..81ee7aa2a9ed 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -518,7 +518,7 @@ public void simpleFirehoseReadingTest() throws IOException Integer rowcount = 0; try (final IngestSegmentFirehose firehose = (IngestSegmentFirehose) - factory.connect(rowParser)) { + factory.connect(rowParser, null)) { while (firehose.hasMore()) { InputRow row = firehose.nextRow(); Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().toArray()); diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 165b29fc7fee..7d29d73a39bf 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -144,7 +144,7 @@ public void testSimple() throws Exception int count = 0; long sum = 0; - try (final Firehose firehose = factory.connect(ROW_PARSER)) { + try (final Firehose firehose = factory.connect(ROW_PARSER, null)) { while (firehose.hasMore()) { final InputRow row = firehose.nextRow(); count++; diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 7a60901154b1..16eeef86799b 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -238,7 +238,7 @@ private static InputRow IR(String dt, String dim1, String dim2, float met) private static class MockExceptionalFirehoseFactory implements FirehoseFactory { @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { return new Firehose() { @@ -288,7 +288,7 @@ public MockFirehoseFactory(@JsonProperty("usedByRealtimeIdxTask") boolean usedBy } @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { final Iterator inputRowIterator = usedByRealtimeIdxTask ? realtimeIdxTaskInputRows.iterator() diff --git a/server/src/main/java/io/druid/segment/realtime/FireDepartment.java b/server/src/main/java/io/druid/segment/realtime/FireDepartment.java index 8114d903256b..1882a3850067 100644 --- a/server/src/main/java/io/druid/segment/realtime/FireDepartment.java +++ b/server/src/main/java/io/druid/segment/realtime/FireDepartment.java @@ -30,6 +30,7 @@ import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.realtime.plumber.Plumber; +import java.io.File; import java.io.IOException; /** @@ -99,9 +100,9 @@ public boolean checkFirehoseV2() return ioConfig.getFirehoseFactoryV2() != null; } - public Firehose connect() throws IOException + public Firehose connect(File temporaryDirectory) throws IOException { - return ioConfig.getFirehoseFactory().connect(dataSchema.getParser()); + return ioConfig.getFirehoseFactory().connect(dataSchema.getParser(), temporaryDirectory); } public FirehoseV2 connect(Object metaData) throws IOException diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index f13752b6006c..8ddd78ae205c 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -25,6 +25,7 @@ import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; +import com.google.common.io.Files; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; @@ -50,9 +51,11 @@ import io.druid.segment.realtime.plumber.Committers; import io.druid.segment.realtime.plumber.Plumber; import io.druid.segment.realtime.plumber.Plumbers; +import org.apache.commons.io.FileUtils; import org.joda.time.Interval; import java.io.Closeable; +import java.io.File; import java.io.IOException; import java.util.HashMap; import java.util.List; @@ -223,13 +226,13 @@ public FireChief(FireDepartment fireDepartment, QueryRunnerFactoryConglomerate c this.metrics = fireDepartment.getMetrics(); } - public Firehose initFirehose() + public Firehose initFirehose(File firehoseTempDir) { synchronized (this) { if (firehose == null) { try { log.info("Calling the FireDepartment and getting a Firehose."); - firehose = fireDepartment.connect(); + firehose = fireDepartment.connect(firehoseTempDir); log.info("Firehose acquired!"); } catch (IOException e) { @@ -288,6 +291,7 @@ public void run() { plumber = initPlumber(); + File firehoseV1TempDir = null; try { Object metadata = plumber.startJob(); @@ -295,22 +299,25 @@ public void run() firehoseV2 = initFirehoseV2(metadata); runFirehoseV2(firehoseV2); } else { - firehose = initFirehose(); + firehoseV1TempDir = Files.createTempDir(); + FileUtils.forceMkdir(firehoseV1TempDir); + firehose = initFirehose(firehoseV1TempDir); runFirehose(firehose); } } - catch (RuntimeException e) { + catch (IOException | RuntimeException e) { log.makeAlert( e, - "RuntimeException aborted realtime processing[%s]", + "[%s] aborted realtime processing[%s]", + e.getClass().getSimpleName(), fireDepartment.getDataSchema().getDataSource() ).emit(); normalExit = false; - throw e; + throw Throwables.propagate(e); } catch (Error e) { - log.makeAlert(e, "Exception aborted realtime processing[%s]", fireDepartment.getDataSchema().getDataSource()) + log.makeAlert(e, "Error aborted realtime processing[%s]", fireDepartment.getDataSchema().getDataSource()) .emit(); normalExit = false; throw e; @@ -322,6 +329,9 @@ public void run() plumber = null; firehose = null; } + if (firehoseV1TempDir != null) { + FileUtils.deleteQuietly(firehoseV1TempDir); + } } } diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/ClippedFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/ClippedFirehoseFactory.java index ef9a8ad03fee..9bcd7761152d 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/ClippedFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/ClippedFirehoseFactory.java @@ -28,6 +28,7 @@ import io.druid.data.input.impl.InputRowParser; import org.joda.time.Interval; +import java.io.File; import java.io.IOException; /** @@ -61,10 +62,10 @@ public Interval getInterval() } @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { return new PredicateFirehose( - delegate.connect(parser), + delegate.connect(parser, temporaryDirectory), new Predicate() { @Override diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java index 536c1edf88c7..4dc3c9627528 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java @@ -29,6 +29,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.impl.InputRowParser; +import java.io.File; import java.io.IOException; import java.util.Iterator; import java.util.List; @@ -52,9 +53,9 @@ public CombiningFirehoseFactory( } @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { - return new CombiningFirehose(parser); + return new CombiningFirehose(parser, temporaryDirectory); } @JsonProperty("delegates") @@ -63,16 +64,18 @@ public List getDelegateFactoryList() return delegateFactoryList; } - public class CombiningFirehose implements Firehose + class CombiningFirehose implements Firehose { private final InputRowParser parser; + private final File temporaryDirectory; private final Iterator firehoseFactoryIterator; private volatile Firehose currentFirehose; - public CombiningFirehose(InputRowParser parser) throws IOException + CombiningFirehose(InputRowParser parser, File temporaryDirectory) throws IOException { this.firehoseFactoryIterator = delegateFactoryList.iterator(); this.parser = parser; + this.temporaryDirectory = temporaryDirectory; nextFirehose(); } @@ -84,7 +87,7 @@ private void nextFirehose() currentFirehose.close(); } - currentFirehose = firehoseFactoryIterator.next().connect(parser); + currentFirehose = firehoseFactoryIterator.next().connect(parser, temporaryDirectory); } catch (IOException e) { if (currentFirehose != null) { diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java index 3f20139fd8ec..1cf9911c7c26 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java @@ -54,6 +54,7 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import java.io.File; import java.io.IOException; import java.io.InputStream; import java.util.Collection; @@ -103,7 +104,7 @@ public EventReceiverFirehoseFactory( } @Override - public Firehose connect(MapInputRowParser firehoseParser) throws IOException + public Firehose connect(MapInputRowParser firehoseParser, File temporaryDirectory) throws IOException { log.info("Connecting firehose: %s", serviceName); final EventReceiverFirehose firehose = new EventReceiverFirehose(firehoseParser); diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java index 300b6d139229..fbf498ce5bac 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java @@ -27,6 +27,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.impl.InputRowParser; +import java.io.File; import java.io.IOException; /** @@ -60,12 +61,12 @@ public int getCount() } @Override - public Firehose connect(final InputRowParser parser) throws IOException + public Firehose connect(final InputRowParser parser, File temporaryDirectory) throws IOException { return new Firehose() { private int i = 0; - private Firehose delegateFirehose = delegate.connect(parser); + private Firehose delegateFirehose = delegate.connect(parser, temporaryDirectory); @Override public boolean hasMore() diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java index 3da59a8f5fda..099b7c12a3e1 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java @@ -30,15 +30,14 @@ import com.ircclouds.irc.api.domain.messages.ChannelPrivMsg; import com.ircclouds.irc.api.listeners.VariousMessageListenerAdapter; import com.ircclouds.irc.api.state.IIRCState; - import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; - import org.joda.time.DateTime; +import java.io.File; import java.io.IOException; import java.util.List; import java.util.UUID; @@ -101,7 +100,7 @@ public List getChannels() } @Override - public Firehose connect(final IrcInputRowParser firehoseParser) throws IOException + public Firehose connect(final IrcInputRowParser firehoseParser, File temporaryDirectory) throws IOException { final IRCApi irc = new IRCApiImpl(false); final LinkedBlockingQueue> queue = new LinkedBlockingQueue>(); diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java index 8b64ae1d3381..5ac8a57cbb93 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java @@ -29,6 +29,7 @@ import io.druid.data.input.impl.InputRowParser; import org.joda.time.DateTime; +import java.io.File; import java.io.IOException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -53,21 +54,21 @@ public TimedShutoffFirehoseFactory( } @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { - return new TimedShutoffFirehose(parser); + return new TimedShutoffFirehose(parser, temporaryDirectory); } - public class TimedShutoffFirehose implements Firehose + class TimedShutoffFirehose implements Firehose { private final Firehose firehose; private final ScheduledExecutorService exec; private final Object shutdownLock = new Object(); private volatile boolean shutdown = false; - public TimedShutoffFirehose(InputRowParser parser) throws IOException + TimedShutoffFirehose(InputRowParser parser, File temporaryDirectory) throws IOException { - firehose = delegateFactory.connect(parser); + firehose = delegateFactory.connect(parser, temporaryDirectory); exec = Execs.scheduledSingleThreaded("timed-shutoff-firehose-%d"); diff --git a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java index 79d213186ccc..3904eca6bff4 100644 --- a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java +++ b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java @@ -33,6 +33,7 @@ import org.junit.Assert; import org.junit.Test; +import java.io.File; import java.io.IOException; import java.util.Arrays; import java.util.Iterator; @@ -52,7 +53,7 @@ public void testCombiningfirehose() throws IOException new ListFirehoseFactory(list2) ) ); - final Firehose firehose = combiningFactory.connect(null); + final Firehose firehose = combiningFactory.connect(null, null); for (int i = 1; i < 6; i++) { Assert.assertTrue(firehose.hasMore()); final InputRow inputRow = firehose.nextRow(); @@ -126,7 +127,7 @@ public static class ListFirehoseFactory implements FirehoseFactory iterator = rows.iterator(); return new Firehose() diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 2484030ae135..ae760e5effc8 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -81,6 +81,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import java.io.File; import java.io.IOException; import java.util.Arrays; import java.util.Iterator; @@ -143,7 +144,7 @@ public void setUp() throws Exception new FirehoseFactory() { @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { return new TestFirehose(rows.iterator()); } diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java index 3517e7eb3e18..2e9a7d3716d6 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; - import io.druid.concurrent.Execs; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; @@ -88,7 +87,8 @@ public void setUp() throws Exception null, null ) - ) + ), + null ); } @@ -220,7 +220,8 @@ public void testDuplicateRegistering() throws IOException null, null ) - ) + ), + null ); } diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java index f09eb88401b0..957a9725c001 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java @@ -90,7 +90,7 @@ public void testConnect() throws IOException Arrays.asList("timestamp", "a") ), Charsets.UTF_8.name() - ))) { + ), null)) { final List rows = new ArrayList<>(); while (firehose.hasMore()) { rows.add(firehose.nextRow()); From 19662b3ca4a572872a19444236867e6675fc00aa Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 3 May 2017 09:25:11 +0900 Subject: [PATCH 08/26] fix when cache and fetch are disabled --- .../impl/PrefetchableTextFilesFirehoseFactory.java | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 3f52a9bacc3b..5f3545b3b53d 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -215,14 +215,11 @@ private long download(ObjectType object, File outFile, int retry) throws IOExcep @Override public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirectory) throws IOException { + if (maxCacheCapacityBytes == 0 && maxFetchCapacityBytes == 0) { + return super.connect(firehoseParser, temporaryDirectory); + } + final List objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "objects")); -// if (baseDir == null) { -// baseDir = Files.createTempDir(); -// baseDir.deleteOnExit(); -// cache(objects); -// } else { -// nextFetchIndex = cacheFiles.size(); -// } Preconditions.checkState(temporaryDirectory.exists(), "temporaryDirectory[%s] does not exist", temporaryDirectory); Preconditions.checkState( From 0c97820dd0bdfc668da7f3c6b38a170e9dbcf174 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 4 May 2017 18:04:27 +0900 Subject: [PATCH 09/26] Address comments --- .../data/input/impl/PrefetchableTextFilesFirehoseFactory.java | 2 +- .../java/io/druid/indexing/common/task/RealtimeIndexTask.java | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 5f3545b3b53d..00685db6bb32 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -183,7 +183,7 @@ private void fetch(final List objects, File baseDir) throws Exceptio } /** - * Downloads an object. It retires downloading {@link #maxFetchRetry} times and throws that exception. + * Downloads an object. It retries downloading {@link #maxFetchRetry} times and throws that exception. * * @param object an object to be downloaded * @param outFile a file which the object data is stored diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index f48fba1bdc73..f3f5deb4755d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -414,8 +414,6 @@ public void run() plumber.finishJob(); } } - - FileUtils.forceDelete(firehoseTempDir); } catch (InterruptedException e) { log.debug(e, "Interrupted while finishing the job"); @@ -428,6 +426,7 @@ public void run() if (firehose != null) { CloseQuietly.close(firehose); } + FileUtils.forceDelete(firehoseTempDir); toolbox.getMonitorScheduler().removeMonitor(metricsMonitor); } } From 2e8b6b0cd4f6c50c7177a6cee2cac5837659fe72 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 4 May 2017 18:41:38 +0900 Subject: [PATCH 10/26] Add more test --- .../PrefetchableTextFilesFirehoseFactory.java | 7 ++-- ...fetchableTextFilesFirehoseFactoryTest.java | 41 ++++++++++--------- 2 files changed, 25 insertions(+), 23 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 00685db6bb32..52297b1344ae 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -109,8 +109,6 @@ public abstract class PrefetchableTextFilesFirehoseFactory private volatile int nextFetchIndex; - private Future fetchFuture; - private boolean needCache = true; public PrefetchableTextFilesFirehoseFactory( @@ -241,10 +239,11 @@ public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirec return new FileIteratingFirehose( new Iterator() { - final Iterator cacheFileIterator = cacheFiles.iterator(); - long remainingCachedBytes = cacheFiles.stream() + private final Iterator cacheFileIterator = cacheFiles.iterator(); + private long remainingCachedBytes = cacheFiles.stream() .mapToLong(File::length) .sum(); + private Future fetchFuture; { fetchIfNeeded(remainingCachedBytes); diff --git a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java index 1ffcd192e5ab..271abab16054 100644 --- a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java @@ -130,11 +130,28 @@ private static void assertResult(List rows) } } + @Test + public void testWithoutCacheAndFetch() throws IOException + { + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.with(testDir, 0, 0); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + Assert.assertEquals(0, factory.getCacheFiles().size()); + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + @Test public void testWithoutCache() throws IOException { final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.withoutCache(testDir); + TestPrefetchableTextFilesFirehoseFactory.with(testDir, 0, 2048); final List rows = new ArrayList<>(); try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { @@ -151,7 +168,7 @@ public void testWithoutCache() throws IOException public void testWithZeroFetchCapacity() throws IOException { final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.withZeroFetchCapacity(testDir); + TestPrefetchableTextFilesFirehoseFactory.with(testDir, 2048, 0); final List rows = new ArrayList<>(); try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { @@ -253,27 +270,13 @@ static class TestPrefetchableTextFilesFirehoseFactory extends PrefetchableTextFi private final File baseDir; private int openExceptionCount; - static TestPrefetchableTextFilesFirehoseFactory withoutCache(File baseDir) + static TestPrefetchableTextFilesFirehoseFactory with(File baseDir, long cacheCapacity, long fetchCapacity) { return new TestPrefetchableTextFilesFirehoseFactory( baseDir, 1024, - 0, - 2048, - 1000, - 3, - 0, - 0 - ); - } - - static TestPrefetchableTextFilesFirehoseFactory withZeroFetchCapacity(File baseDir) - { - return new TestPrefetchableTextFilesFirehoseFactory( - baseDir, - 1024, - 2048, - 0, + cacheCapacity, + fetchCapacity, 1000, 3, 0, From f6bbb56bddfe378aac1dfcbc1e574a5bed096d09 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 4 May 2017 19:02:39 +0900 Subject: [PATCH 11/26] Increase timeout for test --- .../impl/PrefetchableTextFilesFirehoseFactoryTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java index 271abab16054..fb4bc148a5a5 100644 --- a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java @@ -277,7 +277,7 @@ static TestPrefetchableTextFilesFirehoseFactory with(File baseDir, long cacheCap 1024, cacheCapacity, fetchCapacity, - 1000, + 5000, 3, 0, 0 @@ -291,7 +291,7 @@ static TestPrefetchableTextFilesFirehoseFactory of(File baseDir) 1024, 2048, 2048, - 1000, + 5000, 3, 0, 0 @@ -305,7 +305,7 @@ static TestPrefetchableTextFilesFirehoseFactory withOpenExceptions(File baseDir, 1024, 2048, 2048, - 1000, + 5000, 3, count, 0 From f598515d08ae0301690c714dd4fb3a7ffe1efc38 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 5 May 2017 10:59:44 +0900 Subject: [PATCH 12/26] Add wrapObjectStream --- .../AbstractTextFilesFirehoseFactory.java | 20 ++++++-- .../PrefetchableTextFilesFirehoseFactory.java | 51 +++++++++++++------ ...fetchableTextFilesFirehoseFactoryTest.java | 8 ++- .../StaticAzureBlobStoreFirehoseFactory.java | 15 +++--- .../StaticCloudFilesFirehoseFactory.java | 9 +++- .../StaticGoogleBlobStoreFirehoseFactory.java | 9 +++- .../firehose/s3/StaticS3FirehoseFactory.java | 11 ++-- .../firehose/LocalFirehoseFactory.java | 9 +++- 8 files changed, 97 insertions(+), 35 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java index cd32371f779b..3536876bfdad 100644 --- a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java @@ -74,7 +74,7 @@ public LineIterator next() try { return IOUtils.lineIterator( new BufferedReader( - new InputStreamReader(openStream(object), Charsets.UTF_8) + new InputStreamReader(wrapObjectStream(object, openObjectStream(object)), Charsets.UTF_8) ) ); } @@ -102,7 +102,9 @@ public LineIterator next() protected abstract Collection initObjects(); /** - * Open an input stream from the given object. + * TODO + * 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 * @@ -110,5 +112,17 @@ public LineIterator next() * * @throws IOException */ - protected abstract InputStream openStream(ObjectType object) throws IOException; + protected abstract InputStream openObjectStream(ObjectType object) throws IOException; + + /** + * TODO + * Wrap the given input stream if needed. The decompression logic should be applied to the given stream if the object + * is compressed. + * + * @param object an input object + * @param stream a stream for the object + * @return + * @throws IOException + */ + protected abstract InputStream wrapObjectStream(ObjectType object, InputStream stream) throws IOException; } diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 52297b1344ae..15ceae93b8e6 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.io.CountingOutputStream; import io.druid.data.input.Firehose; -import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.logger.Logger; import org.apache.commons.io.Charsets; import org.apache.commons.io.FileUtils; @@ -94,8 +93,8 @@ public abstract class PrefetchableTextFilesFirehoseFactory private final long prefetchTriggerBytes; - private final List cacheFiles; - private final LinkedBlockingQueue fetchFiles; + private final List cacheFiles; + private final LinkedBlockingQueue fetchFiles; // Number of bytes currently fetched files. // This is updated when fetch a file is successfully fetched or a fetched file is deleted. @@ -136,7 +135,7 @@ public PrefetchableTextFilesFirehoseFactory( } @VisibleForTesting - List getCacheFiles() + List getCacheFiles() { return cacheFiles; } @@ -153,7 +152,7 @@ private void cache(final List objects, File baseDir) LOG.info("Caching object[%s]", object); final File outFile = File.createTempFile(CACHE_FILE_PREFIX, null, baseDir); totalFetchedBytes += download(object, outFile, 0); - cacheFiles.add(outFile); + cacheFiles.add(new FetchedFile(object, outFile)); nextFetchIndex++; } } @@ -175,7 +174,7 @@ private void fetch(final List objects, File baseDir) throws Exceptio LOG.info("Fetching object[%s]", object); final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, baseDir); fetchedBytes.addAndGet(download(object, outFile, 0)); - fetchFiles.put(outFile); + fetchFiles.put(new FetchedFile(object, outFile)); nextFetchIndex++; } } @@ -193,7 +192,7 @@ private void fetch(final List objects, File baseDir) throws Exceptio */ private long download(ObjectType object, File outFile, int retry) throws IOException { - try (final InputStream is = openStream(object); + try (final InputStream is = openObjectStream(object); final CountingOutputStream cos = new CountingOutputStream(new FileOutputStream(outFile))) { IOUtils.copy(is, cos); return cos.getCount(); @@ -239,9 +238,9 @@ public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirec return new FileIteratingFirehose( new Iterator() { - private final Iterator cacheFileIterator = cacheFiles.iterator(); + private final Iterator cacheFileIterator = cacheFiles.iterator(); private long remainingCachedBytes = cacheFiles.stream() - .mapToLong(File::length) + .mapToLong(FetchedFile::length) .sum(); private Future fetchFuture; @@ -278,7 +277,7 @@ public LineIterator next() } // If fetch() fails, hasNext() always returns true because nextFetchIndex must be smaller than the number - // of objects which means next() is always called. The below block checks that fetch() threw an exception + // of objects, which means next() is always called. The below block checks that fetch() threw an exception // and propagates it if exists. if (fetchFuture != null && fetchFuture.isDone()) { try { @@ -289,7 +288,7 @@ public LineIterator next() } } - final File fetchedFile; + final FetchedFile fetchedFile; final Closeable closeable; // Check cache first if (cacheFileIterator.hasNext()) { @@ -324,10 +323,10 @@ public LineIterator next() } try { - InputStream stream = FileUtils.openInputStream(fetchedFile); - if (fetchedFile.getName().endsWith(".gz")) { - stream = CompressionUtils.gzipInputStream(stream); - } + final InputStream stream = wrapObjectStream( + fetchedFile.object, + FileUtils.openInputStream(fetchedFile.file) + ); return new ResourceCloseableLineIterator( new BufferedReader( @@ -371,4 +370,26 @@ public void close() } } } + + private class FetchedFile + { + private final ObjectType object; + private final File file; + + public FetchedFile(ObjectType object, File file) + { + this.object = object; + this.file = file; + } + + public long length() + { + return file.length(); + } + + public void delete() + { + file.delete(); + } + } } diff --git a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java index fb4bc148a5a5..0690f48b4c2e 100644 --- a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java @@ -360,7 +360,7 @@ protected Collection initObjects() } @Override - protected InputStream openStream(File object) throws IOException + protected InputStream openObjectStream(File object) throws IOException { if (openExceptionCount > 0) { openExceptionCount--; @@ -376,5 +376,11 @@ protected InputStream openStream(File object) throws IOException } return FileUtils.openInputStream(object); } + + @Override + protected InputStream wrapObjectStream(File object, InputStream stream) throws IOException + { + return stream; + } } } diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java index 899f15893f05..c891c8472b0e 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java @@ -69,14 +69,15 @@ protected Collection initObjects() } @Override - protected InputStream openStream(AzureBlob object) throws IOException + protected InputStream openObjectStream(AzureBlob object) throws IOException { - final InputStream stream = makeByteSource(azureStorage, object).openStream(); - if (object.getPath().endsWith(".gz")) { - return CompressionUtils.gzipInputStream(stream); - } else { - return stream; - } + return makeByteSource(azureStorage, object).openStream(); + } + + @Override + protected InputStream wrapObjectStream(AzureBlob object, InputStream stream) throws IOException + { + return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } private static AzureByteSource makeByteSource(AzureStorage azureStorage, AzureBlob object) diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java index ff38f425af81..29ecfb3bbc8a 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java @@ -70,7 +70,7 @@ protected Collection initObjects() } @Override - protected InputStream openStream(CloudFilesBlob object) throws IOException + protected InputStream openObjectStream(CloudFilesBlob object) throws IOException { final String region = object.getRegion(); final String container = object.getContainer(); @@ -83,7 +83,12 @@ protected InputStream openStream(CloudFilesBlob object) throws IOException cloudFilesApi, region, container); final CloudFilesByteSource byteSource = new CloudFilesByteSource(objectApi, path); - final InputStream stream = byteSource.openStream(); + return byteSource.openStream(); + } + + @Override + protected InputStream wrapObjectStream(CloudFilesBlob object, InputStream stream) throws IOException + { return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } } diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java index c95f88adecbd..4f8afca76ca3 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java @@ -65,14 +65,19 @@ protected Collection initObjects() } @Override - protected InputStream openStream(GoogleBlob object) throws IOException + protected InputStream openObjectStream(GoogleBlob object) throws IOException { final String bucket = object.getBucket(); final String path = object.getPath().startsWith("/") ? object.getPath().substring(1) : object.getPath(); - final InputStream stream = new GoogleByteSource(storage, bucket, path).openStream(); + return new GoogleByteSource(storage, bucket, path).openStream(); + } + + @Override + protected InputStream wrapObjectStream(GoogleBlob object, InputStream stream) throws IOException + { return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } } diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index 78a6569ccd61..f930c761ea7e 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -80,7 +80,7 @@ protected Collection initObjects() } @Override - protected InputStream openStream(URI object) throws IOException + protected InputStream openObjectStream(URI object) throws IOException { final String s3Bucket = object.getAuthority(); final S3Object s3Object = new S3Object( @@ -92,14 +92,19 @@ protected InputStream openStream(URI object) throws IOException log.info("Reading from bucket[%s] object[%s] (%s)", s3Bucket, s3Object.getKey(), object); try { - final InputStream stream = s3Client.getObject(new S3Bucket(s3Bucket), s3Object.getKey()).getDataInputStream(); - return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + return s3Client.getObject(new S3Bucket(s3Bucket), s3Object.getKey()).getDataInputStream(); } catch (ServiceException e) { throw new IOException(e); } } + @Override + protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException + { + return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + } + @Override public boolean equals(Object o) { diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java index 4c4e5af3cc2e..6b80f2e96fbc 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java @@ -89,9 +89,14 @@ protected Collection initObjects() } @Override - protected InputStream openStream(File object) throws IOException + protected InputStream openObjectStream(File object) throws IOException + { + return FileUtils.openInputStream(object); + } + + @Override + protected InputStream wrapObjectStream(File object, InputStream stream) throws IOException { - final InputStream stream = FileUtils.openInputStream(object); return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } } From c5bd32d5f0856021d1f5791827eb940e09f85b9d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 5 May 2017 13:51:00 +0900 Subject: [PATCH 13/26] Move methods to Firehose from PrefetchableFirehoseFactory --- .../PrefetchableTextFilesFirehoseFactory.java | 205 +++++++++--------- ...fetchableTextFilesFirehoseFactoryTest.java | 11 +- 2 files changed, 110 insertions(+), 106 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 15ceae93b8e6..3f0e577f869c 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -19,12 +19,12 @@ package io.druid.data.input.impl; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.io.CountingOutputStream; import io.druid.data.input.Firehose; +import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import org.apache.commons.io.Charsets; import org.apache.commons.io.FileUtils; @@ -88,25 +88,18 @@ public abstract class PrefetchableTextFilesFirehoseFactory // scan yet, so we must download the whole file at once. It's still possible for the size of cached/fetched data to // not exceed these variables by estimating the after-fetch size, but it makes us consider the case when any files // cannot be fetched due to their large size, which makes the implementation complicated. - private long maxCacheCapacityBytes; + private final long maxCacheCapacityBytes; private final long maxFetchCapacityBytes; private final long prefetchTriggerBytes; - private final List cacheFiles; - private final LinkedBlockingQueue fetchFiles; - - // Number of bytes currently fetched files. - // This is updated when fetch a file is successfully fetched or a fetched file is deleted. - private final AtomicLong fetchedBytes = new AtomicLong(0); - // timeout for fetching an object from the remote site private final long fetchTimeout; // maximum retry for fetching an object from the remote site private final int maxFetchRetry; - private volatile int nextFetchIndex; + private final List cacheFiles = new ArrayList<>(); private boolean needCache = true; @@ -129,84 +122,6 @@ public PrefetchableTextFilesFirehoseFactory( : prefetchTriggerBytes; this.fetchTimeout = fetchTimeout == null ? DEFAULT_FETCH_TIMEOUT : fetchTimeout; this.maxFetchRetry = maxFetchRetry == null ? DEFAULT_MAX_FETCH_RETRY : maxFetchRetry; - - cacheFiles = new ArrayList<>(); - fetchFiles = new LinkedBlockingQueue<>(); - } - - @VisibleForTesting - List getCacheFiles() - { - return cacheFiles; - } - - /** - * Cache objects in a local disk up to {@link #maxCacheCapacityBytes}. - */ - private void cache(final List objects, File baseDir) - { - double totalFetchedBytes = 0; - try { - for (int i = 0; i < objects.size() && totalFetchedBytes < maxCacheCapacityBytes; i++) { - final ObjectType object = objects.get(i); - LOG.info("Caching object[%s]", object); - final File outFile = File.createTempFile(CACHE_FILE_PREFIX, null, baseDir); - totalFetchedBytes += download(object, outFile, 0); - cacheFiles.add(new FetchedFile(object, outFile)); - nextFetchIndex++; - } - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - - /** - * Fetch objects to a local disk up to {@link #maxFetchCapacityBytes}. This method is not thread safe and must be - * called by a single thread. Note that even {@link #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. - */ - private void fetch(final List objects, File baseDir) throws Exception - { - for (int i = nextFetchIndex; i < objects.size() && fetchedBytes.get() <= maxFetchCapacityBytes; i++) { - final ObjectType object = objects.get(i); - LOG.info("Fetching object[%s]", object); - final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, baseDir); - fetchedBytes.addAndGet(download(object, outFile, 0)); - fetchFiles.put(new FetchedFile(object, outFile)); - nextFetchIndex++; - } - } - - /** - * Downloads an object. It retries downloading {@link #maxFetchRetry} times and throws that exception. - * - * @param object an object to be downloaded - * @param outFile a file which the object data is stored - * @param retry current retry count - * - * @return number of downloaded bytes - * - * @throws IOException - */ - private long download(ObjectType object, File outFile, int retry) throws IOException - { - try (final InputStream is = openObjectStream(object); - final CountingOutputStream cos = new CountingOutputStream(new FileOutputStream(outFile))) { - IOUtils.copy(is, cos); - return cos.getCount(); - } - catch (IOException e) { - if (retry < maxFetchRetry) { - LOG.error(e, "Failed to download object[%s], retrying (%d of %d)", object, retry + 1, maxFetchRetry); - outFile.delete(); - return download(object, outFile, retry + 1); - } else { - LOG.error(e, "Failed to download object[%s], retries exhausted, aborting", object); - throw e; - } - } } @Override @@ -225,29 +140,109 @@ public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirec temporaryDirectory ); - if (needCache) { - cache(objects, temporaryDirectory); - needCache = false; - } else { - nextFetchIndex = cacheFiles.size(); - } - // fetchExecutor is responsible for background data fetching final ExecutorService fetchExecutor = Executors.newSingleThreadExecutor(); return new FileIteratingFirehose( new Iterator() { - private final Iterator cacheFileIterator = cacheFiles.iterator(); - private long remainingCachedBytes = cacheFiles.stream() - .mapToLong(FetchedFile::length) - .sum(); + private final LinkedBlockingQueue fetchFiles = new LinkedBlockingQueue<>(); + + // Number of bytes currently fetched files. + // This is updated when a file is successfully fetched or a fetched file is deleted. + private final AtomicLong fetchedBytes = new AtomicLong(0); + + private final Iterator cacheFileIterator; + + private long remainingCachedBytes; private Future fetchFuture; + private volatile int nextFetchIndex; { + if (needCache) { + cache(objects, temporaryDirectory); + needCache = false; + } else { + nextFetchIndex = cacheFiles.size(); + } + cacheFileIterator = cacheFiles.iterator(); + remainingCachedBytes = cacheFiles.stream() + .mapToLong(FetchedFile::length) + .sum(); fetchIfNeeded(remainingCachedBytes); } + /** + * Cache objects in a local disk up to {@link PrefetchableTextFilesFirehoseFactory#maxCacheCapacityBytes}. + */ + private void cache(final List objects, File baseDir) + { + double totalFetchedBytes = 0; + try { + for (int i = 0; i < objects.size() && totalFetchedBytes < maxCacheCapacityBytes; i++) { + final ObjectType object = objects.get(i); + LOG.info("Caching object[%s]", object); + final File outFile = File.createTempFile(CACHE_FILE_PREFIX, null, baseDir); + totalFetchedBytes += download(object, outFile, 0); + cacheFiles.add(new FetchedFile(object, outFile)); + nextFetchIndex++; + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + /** + * Fetch objects to a local disk up to {@link PrefetchableTextFilesFirehoseFactory#maxFetchCapacityBytes}. + * This method is not thread safe and must be called by a single thread. Note that even + * {@link PrefetchableTextFilesFirehoseFactory#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. + */ + private void fetch(final List objects, File baseDir) throws Exception + { + for (int i = nextFetchIndex; i < objects.size() && fetchedBytes.get() <= maxFetchCapacityBytes; i++) { + final ObjectType object = objects.get(i); + LOG.info("Fetching object[%s], fetchedBytes[%d]", object, fetchedBytes.get()); + final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, baseDir); + fetchedBytes.addAndGet(download(object, outFile, 0)); + fetchFiles.put(new FetchedFile(object, outFile)); + nextFetchIndex++; + } + } + + /** + * Downloads an object. It retries downloading {@link PrefetchableTextFilesFirehoseFactory#maxFetchRetry} + * times and throws that exception. + * + * @param object an object to be downloaded + * @param outFile a file which the object data is stored + * @param retry current retry count + * + * @return number of downloaded bytes + * + * @throws IOException + */ + private long download(ObjectType object, File outFile, int retry) throws IOException + { + try (final InputStream is = openObjectStream(object); + final CountingOutputStream cos = new CountingOutputStream(new FileOutputStream(outFile))) { + IOUtils.copy(is, cos); + return cos.getCount(); + } + catch (IOException e) { + if (!Thread.currentThread().isInterrupted() && retry < maxFetchRetry) { + LOG.error(e, "Failed to download object[%s], retrying (%d of %d)", object, retry + 1, maxFetchRetry); + outFile.delete(); + return download(object, outFile, retry + 1); + } else { + LOG.error(e, "Failed to download object[%s], retries exhausted, aborting", object); + throw e; + } + } + } + @Override public boolean hasNext() { @@ -282,6 +277,7 @@ public LineIterator next() if (fetchFuture != null && fetchFuture.isDone()) { try { fetchFuture.get(); + fetchFuture = null; } catch (InterruptedException | ExecutionException e) { Throwables.propagate(e); @@ -341,7 +337,16 @@ public LineIterator next() } }, firehoseParser, - fetchExecutor::shutdown + () -> { + fetchExecutor.shutdownNow(); + try { + Preconditions.checkState(fetchExecutor.awaitTermination(fetchTimeout, TimeUnit.MILLISECONDS)); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ISE("Failed to shutdown fetch executor during close"); + } + } ); } diff --git a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java index 0690f48b4c2e..2e65d827b541 100644 --- a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java @@ -138,7 +138,6 @@ public void testWithoutCacheAndFetch() throws IOException final List rows = new ArrayList<>(); try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { - Assert.assertEquals(0, factory.getCacheFiles().size()); while (firehose.hasMore()) { rows.add(firehose.nextRow()); } @@ -155,7 +154,6 @@ public void testWithoutCache() throws IOException final List rows = new ArrayList<>(); try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { - Assert.assertEquals(0, factory.getCacheFiles().size()); while (firehose.hasMore()) { rows.add(firehose.nextRow()); } @@ -266,6 +264,7 @@ public void testReconnect() throws IOException static class TestPrefetchableTextFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory { + private static final long defaultTimeout = 5000; private final long sleepMillis; private final File baseDir; private int openExceptionCount; @@ -277,7 +276,7 @@ static TestPrefetchableTextFilesFirehoseFactory with(File baseDir, long cacheCap 1024, cacheCapacity, fetchCapacity, - 5000, + defaultTimeout, 3, 0, 0 @@ -291,7 +290,7 @@ static TestPrefetchableTextFilesFirehoseFactory of(File baseDir) 1024, 2048, 2048, - 5000, + defaultTimeout, 3, 0, 0 @@ -305,7 +304,7 @@ static TestPrefetchableTextFilesFirehoseFactory withOpenExceptions(File baseDir, 1024, 2048, 2048, - 5000, + defaultTimeout, 3, count, 0 @@ -371,7 +370,7 @@ protected InputStream openObjectStream(File object) throws IOException Thread.sleep(sleepMillis); } catch (InterruptedException e) { - throw new IOException(e); + throw new RuntimeException(e); } } return FileUtils.openInputStream(object); From d47abddada3714dd4d3f44746fc677ea39d6be55 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 5 May 2017 14:01:44 +0900 Subject: [PATCH 14/26] Cleanup comment --- .../druid/data/input/impl/AbstractTextFilesFirehoseFactory.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java index 3536876bfdad..8c55fb6dcf7a 100644 --- a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java @@ -102,7 +102,6 @@ public LineIterator next() protected abstract Collection initObjects(); /** - * TODO * 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)}. * @@ -115,7 +114,6 @@ public LineIterator next() protected abstract InputStream openObjectStream(ObjectType object) throws IOException; /** - * TODO * Wrap the given input stream if needed. The decompression logic should be applied to the given stream if the object * is compressed. * From 65892defd394efa8e2852cc6099a328f943e0e74 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 5 May 2017 23:14:00 +0900 Subject: [PATCH 15/26] add directory listing to s3 firehose --- .../AbstractTextFilesFirehoseFactory.java | 2 +- .../firehose/s3/StaticS3FirehoseFactory.java | 89 +++++++++++++++---- .../s3/StaticS3FirehoseFactoryTest.java | 2 +- 3 files changed, 73 insertions(+), 20 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java index 8c55fb6dcf7a..3f44cbd75e5d 100644 --- a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java @@ -99,7 +99,7 @@ public LineIterator next() * * @return a collection of initialized objects. */ - protected abstract Collection initObjects(); + protected abstract Collection initObjects() throws IOException; /** * Open an input stream from the given object. If the object is compressed, this method should return a byte stream diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index f930c761ea7e..9a01d9c21a4e 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -25,32 +25,38 @@ import com.google.common.base.Preconditions; import io.druid.data.input.impl.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.CompressionUtils; +import io.druid.java.util.common.IAE; import io.druid.java.util.common.logger.Logger; +import org.jets3t.service.S3ServiceException; import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; import java.io.IOException; import java.io.InputStream; import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.List; +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 +public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private static final Logger log = new Logger(StaticS3FirehoseFactory.class); private final RestS3Service s3Client; private final List uris; + private final List directories; @JsonCreator public StaticS3FirehoseFactory( @JacksonInject("s3Client") RestS3Service s3Client, @JsonProperty("uris") List uris, + @JsonProperty("directories") List directories, @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, @@ -60,9 +66,22 @@ public StaticS3FirehoseFactory( { super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); this.s3Client = Preconditions.checkNotNull(s3Client, "null s3Client"); - this.uris = uris; + this.uris = uris == null ? new ArrayList<>() : uris; + this.directories = directories == null ? new ArrayList<>() : directories; - for (final URI inputURI : uris) { + if (!this.uris.isEmpty() && !this.directories.isEmpty()) { + throw new IAE("uris and directories cannot be used together"); + } + + if (this.uris.isEmpty() && this.directories.isEmpty()) { + throw new IAE("uris or directories must be specified"); + } + + for (final URI inputURI : this.uris) { + Preconditions.checkArgument(inputURI.getScheme().equals("s3"), "input uri scheme == s3 (%s)", inputURI); + } + + for (final URI inputURI : this.directories) { Preconditions.checkArgument(inputURI.getScheme().equals("s3"), "input uri scheme == s3 (%s)", inputURI); } } @@ -73,26 +92,60 @@ public List getUris() return uris; } - @Override - protected Collection initObjects() + @JsonProperty("directories") + public List getDirectories() { - return uris; + return directories; } @Override - protected InputStream openObjectStream(URI object) throws IOException + protected Collection initObjects() throws IOException + { + // Here, the returned s3 objects contain minimal information without data. + // Getting data is deferred until openObjectStream() is called for each object. + if (!uris.isEmpty()) { + return uris.stream() + .map( + uri -> { + final String s3Bucket = uri.getAuthority(); + final S3Object s3Object = new S3Object(extractS3Key(uri)); + s3Object.setBucketName(s3Bucket); + return s3Object; + } + ) + .collect(Collectors.toList()); + } else { + final List objects = new ArrayList<>(); + for (URI uri : directories) { + final String bucket = uri.getAuthority(); + final String prefix = extractS3Key(uri); + try { + final S3Object[] listed = s3Client.listObjects(bucket, prefix, null); + objects.addAll(Arrays.asList(listed)); + } + catch (S3ServiceException e) { + throw new IOException(e); + } + } + return objects; + } + } + + private static String extractS3Key(URI uri) { - final String s3Bucket = object.getAuthority(); - final S3Object s3Object = new S3Object( - object.getPath().startsWith("/") - ? object.getPath().substring(1) - : object.getPath() - ); + return uri.getPath().startsWith("/") + ? uri.getPath().substring(1) + : uri.getPath(); + } - log.info("Reading from bucket[%s] object[%s] (%s)", s3Bucket, s3Object.getKey(), object); + @Override + protected InputStream openObjectStream(S3Object object) throws IOException + { + log.info("Reading from bucket[%s] object[%s] (%s)", object.getBucketName(), object.getKey(), object); try { - return s3Client.getObject(new S3Bucket(s3Bucket), s3Object.getKey()).getDataInputStream(); + // Get data of the given object and open an input stream + return s3Client.getObject(object.getBucketName(), object.getKey()).getDataInputStream(); } catch (ServiceException e) { throw new IOException(e); @@ -100,9 +153,9 @@ protected InputStream openObjectStream(URI object) throws IOException } @Override - protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException + protected InputStream wrapObjectStream(S3Object object, InputStream stream) throws IOException { - return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + return object.getKey().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } @Override diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java index 0663b40de613..79a03559e8b6 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java @@ -67,7 +67,7 @@ public TestStaticS3FirehoseFactory( @JsonProperty("uris") List uris ) { - super(EasyMock.niceMock(RestS3Service.class), uris, null, null, null, null, null); + super(EasyMock.niceMock(RestS3Service.class), uris, null, null, null, null, null, null); } } } From 185d6e9e8d12e3801a221fb53349dde6bfb5e288 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 7 May 2017 11:35:53 +0900 Subject: [PATCH 16/26] Rename a variable --- .../firehose/s3/StaticS3FirehoseFactory.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index 9a01d9c21a4e..4005173d7919 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -50,13 +50,13 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor private final RestS3Service s3Client; private final List uris; - private final List directories; + private final List prefixes; @JsonCreator public StaticS3FirehoseFactory( @JacksonInject("s3Client") RestS3Service s3Client, @JsonProperty("uris") List uris, - @JsonProperty("directories") List directories, + @JsonProperty("prefixes") List prefixes, @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, @@ -67,13 +67,13 @@ public StaticS3FirehoseFactory( super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); this.s3Client = Preconditions.checkNotNull(s3Client, "null s3Client"); this.uris = uris == null ? new ArrayList<>() : uris; - this.directories = directories == null ? new ArrayList<>() : directories; + this.prefixes = prefixes == null ? new ArrayList<>() : prefixes; - if (!this.uris.isEmpty() && !this.directories.isEmpty()) { + if (!this.uris.isEmpty() && !this.prefixes.isEmpty()) { throw new IAE("uris and directories cannot be used together"); } - if (this.uris.isEmpty() && this.directories.isEmpty()) { + if (this.uris.isEmpty() && this.prefixes.isEmpty()) { throw new IAE("uris or directories must be specified"); } @@ -81,7 +81,7 @@ public StaticS3FirehoseFactory( Preconditions.checkArgument(inputURI.getScheme().equals("s3"), "input uri scheme == s3 (%s)", inputURI); } - for (final URI inputURI : this.directories) { + for (final URI inputURI : this.prefixes) { Preconditions.checkArgument(inputURI.getScheme().equals("s3"), "input uri scheme == s3 (%s)", inputURI); } } @@ -92,10 +92,10 @@ public List getUris() return uris; } - @JsonProperty("directories") - public List getDirectories() + @JsonProperty("prefixes") + public List getPrefixes() { - return directories; + return prefixes; } @Override @@ -116,7 +116,7 @@ protected Collection initObjects() throws IOException .collect(Collectors.toList()); } else { final List objects = new ArrayList<>(); - for (URI uri : directories) { + for (URI uri : prefixes) { final String bucket = uri.getAuthority(); final String prefix = extractS3Key(uri); try { From 3dc991deb3c32b07569b7ac1d74ae99698dfbcf7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 10 May 2017 16:23:19 +0900 Subject: [PATCH 17/26] Addressing comments --- .../io/druid/data/input/FirehoseFactory.java | 17 +- .../PrefetchableTextFilesFirehoseFactory.java | 176 +++++++++++------- ...fetchableTextFilesFirehoseFactoryTest.java | 5 +- .../druid/indexing/kafka/KafkaIndexTask.java | 3 +- .../firehose/s3/StaticS3FirehoseFactory.java | 20 +- .../io/druid/indexing/common/TaskToolbox.java | 15 ++ .../druid/indexing/common/task/IndexTask.java | 23 +-- .../indexing/common/task/MergeTaskBase.java | 4 +- .../common/task/RealtimeIndexTask.java | 10 +- .../segment/realtime/FireDepartment.java | 5 +- .../segment/realtime/RealtimeManager.java | 17 +- 11 files changed, 179 insertions(+), 116 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/FirehoseFactory.java b/api/src/main/java/io/druid/data/input/FirehoseFactory.java index 8579ccd9067d..0600c7be189b 100644 --- a/api/src/main/java/io/druid/data/input/FirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/FirehoseFactory.java @@ -30,6 +30,22 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") public interface FirehoseFactory { + /** + * Initialization method that connects up the fire hose. If this method returns successfully it should be safe to + * call hasMore() on the returned Firehose (which might subsequently block). + *

        + * If this method returns null, then any attempt to call hasMore(), nextRow(), commit() and close() on the return + * value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on + * invalid configuration is preferred over returning null. + * + * @param parser an input row parser + */ + @Deprecated + default Firehose connect(T parser) throws IOException, ParseException + { + return connect(parser, null); + } + /** * Initialization method that connects up the fire hose. If this method returns successfully it should be safe to * call hasMore() on the returned Firehose (which might subsequently block). @@ -45,5 +61,4 @@ public interface FirehoseFactory * @param temporaryDirectory a directory where temporary files are stored */ Firehose connect(T parser, File temporaryDirectory) throws IOException, ParseException; - } diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 3f0e577f869c..9b6295a9ed58 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -23,6 +23,7 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.io.CountingOutputStream; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.druid.data.input.Firehose; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; @@ -100,8 +101,16 @@ public abstract class PrefetchableTextFilesFirehoseFactory private final int maxFetchRetry; private final List cacheFiles = new ArrayList<>(); + private long totalCachedBytes; - private boolean needCache = true; + private static ExecutorService createFetchExecutor() + { + return Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder() + .setNameFormat("firehose_fetch_%d") + .build() + ); + } public PrefetchableTextFilesFirehoseFactory( Long maxCacheCapacityBytes, @@ -141,7 +150,7 @@ public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirec ); // fetchExecutor is responsible for background data fetching - final ExecutorService fetchExecutor = Executors.newSingleThreadExecutor(); + final ExecutorService fetchExecutor = createFetchExecutor(); return new FileIteratingFirehose( new Iterator() @@ -151,45 +160,36 @@ public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirec // Number of bytes currently fetched files. // This is updated when a file is successfully fetched or a fetched file is deleted. private final AtomicLong fetchedBytes = new AtomicLong(0); + private final boolean cacheInitialized; - private final Iterator cacheFileIterator; - - private long remainingCachedBytes; private Future fetchFuture; + private int cacheIterateIndex; + + // nextFetchIndex indicates which object should be downloaded when fetch is triggered. + // This variable is only updated by the fetchExecutor thread, but read by the main thread (in hasNext()) + // and the fetchExecutor thread (in fetch()). To guarantee that the main thread can read the most fresh value, + // this variable should be volatile. private volatile int nextFetchIndex; { - if (needCache) { - cache(objects, temporaryDirectory); - needCache = false; - } else { + cacheInitialized = totalCachedBytes > 0; + + if (cacheInitialized) { nextFetchIndex = cacheFiles.size(); } - cacheFileIterator = cacheFiles.iterator(); - remainingCachedBytes = cacheFiles.stream() - .mapToLong(FetchedFile::length) - .sum(); - fetchIfNeeded(remainingCachedBytes); + fetchIfNeeded(totalCachedBytes); } - /** - * Cache objects in a local disk up to {@link PrefetchableTextFilesFirehoseFactory#maxCacheCapacityBytes}. - */ - private void cache(final List objects, File baseDir) + private void fetchIfNeeded(long remainingBytes) { - double totalFetchedBytes = 0; - try { - for (int i = 0; i < objects.size() && totalFetchedBytes < maxCacheCapacityBytes; i++) { - final ObjectType object = objects.get(i); - LOG.info("Caching object[%s]", object); - final File outFile = File.createTempFile(CACHE_FILE_PREFIX, null, baseDir); - totalFetchedBytes += download(object, outFile, 0); - cacheFiles.add(new FetchedFile(object, outFile)); - nextFetchIndex++; - } - } - catch (Exception e) { - throw Throwables.propagate(e); + if ((fetchFuture == null || fetchFuture.isDone()) + && remainingBytes <= prefetchTriggerBytes) { + fetchFuture = fetchExecutor.submit( + () -> { + fetch(objects, temporaryDirectory); + return null; + } + ); } } @@ -214,17 +214,17 @@ private void fetch(final List objects, File baseDir) throws Exceptio /** * Downloads an object. It retries downloading {@link PrefetchableTextFilesFirehoseFactory#maxFetchRetry} - * times and throws that exception. + * times and throws an exception. * - * @param object an object to be downloaded - * @param outFile a file which the object data is stored - * @param retry current retry count + * @param object an object to be downloaded + * @param outFile a file which the object data is stored + * @param tryCount current retry count * * @return number of downloaded bytes * * @throws IOException */ - private long download(ObjectType object, File outFile, int retry) throws IOException + private long download(ObjectType object, File outFile, int tryCount) throws IOException { try (final InputStream is = openObjectStream(object); final CountingOutputStream cos = new CountingOutputStream(new FileOutputStream(outFile))) { @@ -232,10 +232,11 @@ private long download(ObjectType object, File outFile, int retry) throws IOExcep return cos.getCount(); } catch (IOException e) { - if (!Thread.currentThread().isInterrupted() && retry < maxFetchRetry) { - LOG.error(e, "Failed to download object[%s], retrying (%d of %d)", object, retry + 1, maxFetchRetry); + final int nextTry = tryCount + 1; + if (!Thread.currentThread().isInterrupted() && nextTry < maxFetchRetry) { + LOG.error(e, "Failed to download object[%s], retrying (%d of %d)", object, nextTry, maxFetchRetry); outFile.delete(); - return download(object, outFile, retry + 1); + return download(object, outFile, nextTry); } else { LOG.error(e, "Failed to download object[%s], retries exhausted, aborting", object); throw e; @@ -246,21 +247,21 @@ private long download(ObjectType object, File outFile, int retry) throws IOExcep @Override public boolean hasNext() { - return cacheFileIterator.hasNext() + return (cacheInitialized && cacheIterateIndex < cacheFiles.size()) || !fetchFiles.isEmpty() || nextFetchIndex < objects.size(); } - private void fetchIfNeeded(long remainingBytes) + private void checkFetchException() { - if ((fetchFuture == null || fetchFuture.isDone()) - && remainingBytes <= prefetchTriggerBytes) { - fetchFuture = fetchExecutor.submit( - () -> { - fetch(objects, temporaryDirectory); - return null; - } - ); + if (fetchFuture != null && fetchFuture.isDone()) { + try { + fetchFuture.get(); + fetchFuture = null; + } + catch (InterruptedException | ExecutionException e) { + Throwables.propagate(e); + } } } @@ -272,31 +273,21 @@ public LineIterator next() } // If fetch() fails, hasNext() always returns true because nextFetchIndex must be smaller than the number - // of objects, which means next() is always called. The below block checks that fetch() threw an exception + // of objects, which means next() is always called. The below method checks that fetch() threw an exception // and propagates it if exists. - if (fetchFuture != null && fetchFuture.isDone()) { - try { - fetchFuture.get(); - fetchFuture = null; - } - catch (InterruptedException | ExecutionException e) { - Throwables.propagate(e); - } - } + checkFetchException(); final FetchedFile fetchedFile; final Closeable closeable; // Check cache first - if (cacheFileIterator.hasNext()) { - fetchedFile = cacheFileIterator.next(); - remainingCachedBytes -= fetchedFile.length(); - fetchIfNeeded(remainingCachedBytes); - closeable = () -> { - }; + if (cacheInitialized && cacheIterateIndex < cacheFiles.size()) { + fetchedFile = cacheFiles.get(cacheIterateIndex++); + closeable = getNoopCloseable(); } else { if (!fetchFiles.isEmpty()) { // If there are already fetched files, use them fetchedFile = fetchFiles.poll(); + closeable = cacheIfPossibleAndGetCloseable(fetchedFile, fetchedBytes); fetchIfNeeded(fetchedBytes.get()); } else { // Otherwise, wait for fetching @@ -304,18 +295,19 @@ public LineIterator next() fetchIfNeeded(fetchedBytes.get()); fetchedFile = fetchFiles.poll(fetchTimeout, TimeUnit.MILLISECONDS); if (fetchedFile == null) { + // Check the latest fetch is failed + checkFetchException(); + // Or throw a timeout exception throw new RuntimeException(new TimeoutException()); } + closeable = cacheIfPossibleAndGetCloseable(fetchedFile, fetchedBytes); + // trigger fetch again for subsequent next() calls + fetchIfNeeded(fetchedBytes.get()); } catch (InterruptedException e) { throw Throwables.propagate(e); } } - closeable = () -> { - final long fileSize = fetchedFile.length(); - fetchedFile.delete(); - fetchedBytes.addAndGet(-fileSize); - }; } try { @@ -350,6 +342,50 @@ public LineIterator next() ); } + private boolean cacheIfPossible(FetchedFile fetchedFile) + { + // maxCacheCapacityBytes is a rough limit, so if totalCachedBytes is larger than it, no more caching is + // allowed. + if (totalCachedBytes < maxCacheCapacityBytes) { + cacheFiles.add(fetchedFile); + totalCachedBytes += fetchedFile.length(); + return true; + } else { + return false; + } + } + + private Closeable cacheIfPossibleAndGetCloseable(FetchedFile fetchedFile, AtomicLong fetchedBytes) + { + final Closeable closeable; + if (cacheIfPossible(fetchedFile)) { + closeable = getNoopCloseable(); + // 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()); + } else { + closeable = getFetchedFileCleanupCloseable(fetchedFile, fetchedBytes); + } + return closeable; + } + + private Closeable getNoopCloseable() + { + return () -> {}; + } + + private Closeable getFetchedFileCleanupCloseable( + final FetchedFile fetchedFile, + final AtomicLong fetchedBytes + ) + { + return () -> { + final long fileSize = fetchedFile.length(); + fetchedFile.delete(); + fetchedBytes.addAndGet(-fileSize); + }; + } + /** * This class calls the {@link Closeable#close()} method of the resourceCloser when it is closed. */ diff --git a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java index 2e65d827b541..7481cfd6a4c5 100644 --- a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java @@ -45,6 +45,7 @@ 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 @@ -214,7 +215,7 @@ public void testRetry() throws IOException public void testMaxRetry() throws IOException { expectedException.expect(RuntimeException.class); - expectedException.expectCause(CoreMatchers.instanceOf(IOException.class)); + expectedException.expectCause(CoreMatchers.instanceOf(ExecutionException.class)); expectedException.expectMessage("Exception for retry test"); final TestPrefetchableTextFilesFirehoseFactory factory = @@ -264,7 +265,7 @@ public void testReconnect() throws IOException static class TestPrefetchableTextFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory { - private static final long defaultTimeout = 5000; + private static final long defaultTimeout = 1000; private final long sleepMillis; private final File baseDir; private int openExceptionCount; diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index b344b8a3e9b5..650ea746b031 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -89,7 +89,6 @@ import javax.ws.rs.QueryParam; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Map; @@ -823,7 +822,7 @@ private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox ioConfig.getStartPartitions().getPartitionOffsetMap().size()); return Appenderators.createRealtime( dataSchema, - tuningConfig.withBasePersistDirectory(new File(toolbox.getTaskWorkDir(), "persist")) + tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()) .withMaxRowsInMemory(maxRowsInMemoryPerPartition), metrics, toolbox.getSegmentPusher(), diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index 4005173d7919..95ae9394a70a 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -27,8 +27,8 @@ import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.IAE; import io.druid.java.util.common.logger.Logger; -import org.jets3t.service.S3ServiceException; import org.jets3t.service.ServiceException; +import org.jets3t.service.StorageObjectsChunk; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Object; @@ -47,6 +47,7 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private static final Logger log = new Logger(StaticS3FirehoseFactory.class); + private static final long MAX_LISTING_LENGTH = 1024; private final RestS3Service s3Client; private final List uris; @@ -120,10 +121,21 @@ protected Collection initObjects() throws IOException final String bucket = uri.getAuthority(); final String prefix = extractS3Key(uri); try { - final S3Object[] listed = s3Client.listObjects(bucket, prefix, null); - objects.addAll(Arrays.asList(listed)); + String lastKey = null; + StorageObjectsChunk objectsChunk; + do { + objectsChunk = s3Client.listObjectsChunked( + bucket, + prefix, + null, + MAX_LISTING_LENGTH, + lastKey + ); + Arrays.stream(objectsChunk.getObjects()).forEach(storageObject -> objects.add((S3Object) storageObject)); + lastKey = objectsChunk.getPriorLastKey(); + } while (!objectsChunk.isListingComplete()); } - catch (S3ServiceException e) { + catch (ServiceException e) { throw new IOException(e); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index d9a690eb65ba..dc1175a22053 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -253,4 +253,19 @@ public CacheConfig getCacheConfig() public IndexMergerV9 getIndexMergerV9() { return indexMergerV9; } + + public File getFirehoseTemporaryDir() + { + return new File(taskWorkDir, "firehose"); + } + + public File getMergeDir() + { + return new File(taskWorkDir, "merge"); + } + + public File getPersistDir() + { + return new File(taskWorkDir, "persist"); + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 5e2cdb128a8a..e9061fb19057 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -38,7 +38,6 @@ import com.google.common.collect.Maps; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; -import com.google.common.io.Files; import io.druid.common.utils.JodaUtils; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; @@ -164,9 +163,6 @@ public IndexIngestionSpec getIngestionSchema() @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { - final File firehoseTempDir = Files.createTempDir(); - FileUtils.forceMkdir(firehoseTempDir); - final boolean determineIntervals = !ingestionSchema.getDataSchema() .getGranularitySpec() .bucketIntervals() @@ -179,6 +175,10 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ((IngestSegmentFirehoseFactory) firehoseFactory).setTaskToolbox(toolbox); } + final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); + // Firehose temporary directory is automatically removed when this IndexTask completes. + FileUtils.forceMkdir(firehoseTempDir); + final Map> shardSpecs = determineShardSpecs(toolbox, firehoseFactory, firehoseTempDir); final String version; @@ -201,15 +201,10 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception dataSchema = ingestionSchema.getDataSchema(); } - try { - if (generateAndPublishSegments(toolbox, dataSchema, shardSpecs, version, firehoseFactory, firehoseTempDir)) { - return TaskStatus.success(getId()); - } else { - return TaskStatus.failure(getId()); - } - } - finally { - FileUtils.forceDelete(firehoseTempDir); + if (generateAndPublishSegments(toolbox, dataSchema, shardSpecs, version, firehoseFactory, firehoseTempDir)) { + return TaskStatus.success(getId()); + } else { + return TaskStatus.failure(getId()); } } @@ -507,7 +502,7 @@ private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox { return Appenderators.createOffline( dataSchema, - ingestionSchema.getTuningConfig().withBasePersistDirectory(new File(toolbox.getTaskWorkDir(), "persist")), + ingestionSchema.getTuningConfig().withBasePersistDirectory(toolbox.getPersistDir()), metrics, toolbox.getSegmentPusher(), toolbox.getObjectMapper(), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java index fdec7e63c1b3..de49180bf08b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java @@ -130,7 +130,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception final ServiceEmitter emitter = toolbox.getEmitter(); final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments); - final File taskDir = toolbox.getTaskWorkDir(); + final File mergeDir = toolbox.getMergeDir(); try { final long startTime = System.currentTimeMillis(); @@ -155,7 +155,7 @@ public String apply(DataSegment input) final Map gettedSegments = toolbox.fetchSegments(segments); // merge files together - final File fileToUpload = merge(toolbox, gettedSegments, new File(taskDir, "merged")); + final File fileToUpload = merge(toolbox, gettedSegments, mergeDir); emitter.emit(builder.build("merger/numMerged", segments.size())); emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime)); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index f3f5deb4755d..807781a6fd32 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -26,7 +26,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.io.Files; import com.google.common.primitives.Ints; import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Committer; @@ -285,7 +284,7 @@ public String getVersion(final Interval interval) DataSchema dataSchema = spec.getDataSchema(); RealtimeIOConfig realtimeIOConfig = spec.getIOConfig(); RealtimeTuningConfig tuningConfig = spec.getTuningConfig() - .withBasePersistDirectory(new File(toolbox.getTaskWorkDir(), "persist")) + .withBasePersistDirectory(toolbox.getPersistDir()) .withVersioningPolicy(versioningPolicy); final FireDepartment fireDepartment = new FireDepartment( @@ -326,7 +325,7 @@ public String getVersion(final Interval interval) this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, metrics); Supplier committerSupplier = null; - final File firehoseTempDir = Files.createTempDir(); + final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); try { plumber.startJob(); @@ -334,8 +333,10 @@ public String getVersion(final Interval interval) // Set up metrics emission toolbox.getMonitorScheduler().addMonitor(metricsMonitor); - // Delay firehose connection to avoid claiming input resources while the plumber is starting up. + // Firehose temporary directory is automatically removed when this RealtimeIndexTask completes. FileUtils.forceMkdir(firehoseTempDir); + + // Delay firehose connection to avoid claiming input resources while the plumber is starting up. final FirehoseFactory firehoseFactory = spec.getIOConfig().getFirehoseFactory(); final boolean firehoseDrainableByClosing = isFirehoseDrainableByClosing(firehoseFactory); @@ -426,7 +427,6 @@ public void run() if (firehose != null) { CloseQuietly.close(firehose); } - FileUtils.forceDelete(firehoseTempDir); toolbox.getMonitorScheduler().removeMonitor(metricsMonitor); } } diff --git a/server/src/main/java/io/druid/segment/realtime/FireDepartment.java b/server/src/main/java/io/druid/segment/realtime/FireDepartment.java index 1882a3850067..eca78928eccf 100644 --- a/server/src/main/java/io/druid/segment/realtime/FireDepartment.java +++ b/server/src/main/java/io/druid/segment/realtime/FireDepartment.java @@ -30,7 +30,6 @@ import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.realtime.plumber.Plumber; -import java.io.File; import java.io.IOException; /** @@ -100,9 +99,9 @@ public boolean checkFirehoseV2() return ioConfig.getFirehoseFactoryV2() != null; } - public Firehose connect(File temporaryDirectory) throws IOException + public Firehose connect() throws IOException { - return ioConfig.getFirehoseFactory().connect(dataSchema.getParser(), temporaryDirectory); + return ioConfig.getFirehoseFactory().connect(dataSchema.getParser(), null); } public FirehoseV2 connect(Object metaData) throws IOException diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index 8ddd78ae205c..d9fbe539ab20 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -25,7 +25,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; -import com.google.common.io.Files; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; @@ -51,11 +50,9 @@ import io.druid.segment.realtime.plumber.Committers; import io.druid.segment.realtime.plumber.Plumber; import io.druid.segment.realtime.plumber.Plumbers; -import org.apache.commons.io.FileUtils; import org.joda.time.Interval; import java.io.Closeable; -import java.io.File; import java.io.IOException; import java.util.HashMap; import java.util.List; @@ -226,13 +223,13 @@ public FireChief(FireDepartment fireDepartment, QueryRunnerFactoryConglomerate c this.metrics = fireDepartment.getMetrics(); } - public Firehose initFirehose(File firehoseTempDir) + public Firehose initFirehose() { synchronized (this) { if (firehose == null) { try { log.info("Calling the FireDepartment and getting a Firehose."); - firehose = fireDepartment.connect(firehoseTempDir); + firehose = fireDepartment.connect(); log.info("Firehose acquired!"); } catch (IOException e) { @@ -291,7 +288,6 @@ public void run() { plumber = initPlumber(); - File firehoseV1TempDir = null; try { Object metadata = plumber.startJob(); @@ -299,14 +295,12 @@ public void run() firehoseV2 = initFirehoseV2(metadata); runFirehoseV2(firehoseV2); } else { - firehoseV1TempDir = Files.createTempDir(); - FileUtils.forceMkdir(firehoseV1TempDir); - firehose = initFirehose(firehoseV1TempDir); + firehose = initFirehose(); runFirehose(firehose); } } - catch (IOException | RuntimeException e) { + catch (RuntimeException e) { log.makeAlert( e, "[%s] aborted realtime processing[%s]", @@ -329,9 +323,6 @@ public void run() plumber = null; firehose = null; } - if (firehoseV1TempDir != null) { - FileUtils.deleteQuietly(firehoseV1TempDir); - } } } From a8d9af31d883a74f6699172ef88077718b5a8bcb Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 10 May 2017 21:06:32 +0900 Subject: [PATCH 18/26] Update document --- .../impl/AbstractTextFilesFirehoseFactory.java | 6 +++++- .../impl/PrefetchableTextFilesFirehoseFactory.java | 13 ++++++++----- .../content/development/extensions-contrib/azure.md | 8 ++++++++ .../development/extensions-contrib/cloudfiles.md | 7 +++++++ .../development/extensions-contrib/google.md | 8 ++++++++ docs/content/development/extensions-core/s3.md | 11 ++++++++++- 6 files changed, 46 insertions(+), 7 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java index 3f44cbd75e5d..a42efb4dd53b 100644 --- a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java @@ -50,10 +50,14 @@ public abstract class AbstractTextFilesFirehoseFactory { private static final Logger LOG = new Logger(AbstractTextFilesFirehoseFactory.class); + private List objects; + @Override public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirectory) throws IOException { - final List objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects")); + if (objects == null) { + objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects")); + } final Iterator iterator = objects.iterator(); return new FileIteratingFirehose( new Iterator() diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 9b6295a9ed58..9d4cfdb29850 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -81,7 +81,6 @@ public abstract class PrefetchableTextFilesFirehoseFactory private static final long DEFAULT_MAX_FETCH_CAPACITY_BYTES = 1024 * 1024 * 1024; // 1GB private static final long DEFAULT_FETCH_TIMEOUT = 60_000; // 60 secs private static final int DEFAULT_MAX_FETCH_RETRY = 3; - private static final String CACHE_FILE_PREFIX = "cache-"; private static final String FETCH_FILE_PREFIX = "fetch-"; // The below two variables are roughly the max size of total cached/fetched objects, but the actual cached/fetched @@ -103,6 +102,8 @@ public abstract class PrefetchableTextFilesFirehoseFactory private final List cacheFiles = new ArrayList<>(); private long totalCachedBytes; + private List objects; + private static ExecutorService createFetchExecutor() { return Executors.newSingleThreadExecutor( @@ -140,7 +141,9 @@ public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirec return super.connect(firehoseParser, temporaryDirectory); } - final List objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "objects")); + if (objects == null) { + objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "objects")); + } Preconditions.checkState(temporaryDirectory.exists(), "temporaryDirectory[%s] does not exist", temporaryDirectory); Preconditions.checkState( @@ -186,7 +189,7 @@ private void fetchIfNeeded(long remainingBytes) && remainingBytes <= prefetchTriggerBytes) { fetchFuture = fetchExecutor.submit( () -> { - fetch(objects, temporaryDirectory); + fetch(); return null; } ); @@ -200,12 +203,12 @@ private void fetchIfNeeded(long remainingBytes) * This is for simplifying design, and should be improved when our client implementations for cloud storages * like S3 support range scan. */ - private void fetch(final List objects, File baseDir) throws Exception + private void fetch() throws Exception { for (int i = nextFetchIndex; i < objects.size() && fetchedBytes.get() <= maxFetchCapacityBytes; i++) { final ObjectType object = objects.get(i); LOG.info("Fetching object[%s], fetchedBytes[%d]", object, fetchedBytes.get()); - final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, baseDir); + final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, temporaryDirectory); fetchedBytes.addAndGet(download(object, outFile, 0)); fetchFiles.put(new FetchedFile(object, outFile)); nextFetchIndex++; diff --git a/docs/content/development/extensions-contrib/azure.md b/docs/content/development/extensions-contrib/azure.md index 497735c7ddf9..cb9f090cc8ee 100644 --- a/docs/content/development/extensions-contrib/azure.md +++ b/docs/content/development/extensions-contrib/azure.md @@ -51,10 +51,18 @@ Sample spec: } ``` +This firehose provides caching and prefetching features. In IndexTask, a firehose can be read twice if intervals or +shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow. + |property|description|default|required?| |--------|-----------|-------|---------| |type|This should be "static-azure-blobstore".|N/A|yes| |blobs|JSON array of [Azure blobs](https://msdn.microsoft.com/en-us/library/azure/ee691964.aspx).|N/A|yes| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes|1073741824|no| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes|1073741824|no| +|prefetchTriggerBytes|Threshold to trigger prefetching Azure objects|maxFetchCapacityBytes / 2|no| +|fetchTimeout|Timeout for fetching an Azure object|60000|no| +|maxFetchRetry|Maximum retry for fetching an Azure object|3|no| Azure Blobs: diff --git a/docs/content/development/extensions-contrib/cloudfiles.md b/docs/content/development/extensions-contrib/cloudfiles.md index de56c6cc79cc..788a5cef7a28 100644 --- a/docs/content/development/extensions-contrib/cloudfiles.md +++ b/docs/content/development/extensions-contrib/cloudfiles.md @@ -51,11 +51,18 @@ Sample spec: ] } ``` +This firehose provides caching and prefetching features. In IndexTask, a firehose can be read twice if intervals or +shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow. |property|description|default|required?| |--------|-----------|-------|---------| |type|This should be "static-cloudfiles".|N/A|yes| |blobs|JSON array of Cloud Files blobs.|N/A|yes| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes|1073741824|no| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes|1073741824|no| +|prefetchTriggerBytes|Threshold to trigger prefetching Cloud Files objects|maxFetchCapacityBytes / 2|no| +|fetchTimeout|Timeout for fetching a Cloud Files object|60000|no| +|maxFetchRetry|Maximum retry for fetching a Cloud Files object|3|no| Cloud Files Blobs: diff --git a/docs/content/development/extensions-contrib/google.md b/docs/content/development/extensions-contrib/google.md index 1c080c5c3dd4..66416ff8fa71 100644 --- a/docs/content/development/extensions-contrib/google.md +++ b/docs/content/development/extensions-contrib/google.md @@ -41,10 +41,18 @@ Sample spec: } ``` +This firehose provides caching and prefetching features. In IndexTask, a firehose can be read twice if intervals or +shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow. + |property|description|default|required?| |--------|-----------|-------|---------| |type|This should be "static-google-blobstore".|N/A|yes| |blobs|JSON array of Google Blobs.|N/A|yes| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes|1073741824|no| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes|1073741824|no| +|prefetchTriggerBytes|Threshold to trigger prefetching Google blobs|maxFetchCapacityBytes / 2|no| +|fetchTimeout|Timeout for fetching a Google blob|60000|no| +|maxFetchRetry|Maximum retry for fetching a Google blob|3|no| Google Blobs: diff --git a/docs/content/development/extensions-core/s3.md b/docs/content/development/extensions-core/s3.md index 59f6a7084b3b..8804cf3cb577 100644 --- a/docs/content/development/extensions-core/s3.md +++ b/docs/content/development/extensions-core/s3.md @@ -32,7 +32,16 @@ Sample spec: } ``` +This firehose provides caching and prefetching features. In IndexTask, a firehose can be read twice if intervals or +shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow. + |property|description|default|required?| |--------|-----------|-------|---------| |type|This should be "static-s3"|N/A|yes| -|uris|JSON array of URIs where s3 files to be ingested are located.|N/A|yes| +|uris|JSON array of URIs where s3 files to be ingested are located.|N/A|`uris` or `prefixes` must be set| +|prefixes|JSON array of URI prefixes for the locations of s3 files to be ingested|N/A|`uris` or `prefixes` must be set| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes|1073741824|no| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes|1073741824|no| +|prefetchTriggerBytes|Threshold to trigger prefetching s3 objects|maxFetchCapacityBytes / 2|no| +|fetchTimeout|Timeout for fetching an s3 object|60000|no| +|maxFetchRetry|Maximum retry for fetching an s3 object|3|no| \ No newline at end of file From 02d22c96b8c98968f584b506ad7cbe553453a025 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 10 May 2017 23:13:54 +0900 Subject: [PATCH 19/26] Support disabling prefetch --- .../PrefetchableTextFilesFirehoseFactory.java | 171 ++++++++++++------ .../development/extensions-contrib/azure.md | 12 +- .../extensions-contrib/cloudfiles.md | 12 +- .../development/extensions-contrib/google.md | 12 +- .../content/development/extensions-core/s3.md | 14 +- 5 files changed, 143 insertions(+), 78 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 9d4cfdb29850..35e42c2b67ad 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -164,6 +164,7 @@ public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirec // This is updated when a file is successfully fetched or a fetched file is deleted. private final AtomicLong fetchedBytes = new AtomicLong(0); private final boolean cacheInitialized; + private final boolean prefetchEnabled; private Future fetchFuture; private int cacheIterateIndex; @@ -176,11 +177,14 @@ public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirec { cacheInitialized = totalCachedBytes > 0; + prefetchEnabled = maxFetchCapacityBytes > 0; if (cacheInitialized) { nextFetchIndex = cacheFiles.size(); } - fetchIfNeeded(totalCachedBytes); + if (prefetchEnabled) { + fetchIfNeeded(totalCachedBytes); + } } private void fetchIfNeeded(long remainingBytes) @@ -255,19 +259,6 @@ public boolean hasNext() || nextFetchIndex < objects.size(); } - private void checkFetchException() - { - if (fetchFuture != null && fetchFuture.isDone()) { - try { - fetchFuture.get(); - fetchFuture = null; - } - catch (InterruptedException | ExecutionException e) { - Throwables.propagate(e); - } - } - } - @Override public LineIterator next() { @@ -280,56 +271,109 @@ public LineIterator next() // and propagates it if exists. checkFetchException(); - final FetchedFile fetchedFile; - final Closeable closeable; - // Check cache first - if (cacheInitialized && cacheIterateIndex < cacheFiles.size()) { - fetchedFile = cacheFiles.get(cacheIterateIndex++); - closeable = getNoopCloseable(); - } else { - if (!fetchFiles.isEmpty()) { - // If there are already fetched files, use them - fetchedFile = fetchFiles.poll(); - closeable = cacheIfPossibleAndGetCloseable(fetchedFile, fetchedBytes); - fetchIfNeeded(fetchedBytes.get()); + final OpenedObject openedObject; + + try { + // Check cache first + if (cacheInitialized && cacheIterateIndex < cacheFiles.size()) { + final FetchedFile fetchedFile = cacheFiles.get(cacheIterateIndex++); + openedObject = new OpenedObject(fetchedFile, getNoopCloser()); + } else if (prefetchEnabled) { + openedObject = openObjectFromLocal(); } else { - // Otherwise, wait for fetching - try { - fetchIfNeeded(fetchedBytes.get()); - fetchedFile = fetchFiles.poll(fetchTimeout, TimeUnit.MILLISECONDS); - if (fetchedFile == null) { - // Check the latest fetch is failed - checkFetchException(); - // Or throw a timeout exception - throw new RuntimeException(new TimeoutException()); - } - closeable = cacheIfPossibleAndGetCloseable(fetchedFile, fetchedBytes); - // trigger fetch again for subsequent next() calls - fetchIfNeeded(fetchedBytes.get()); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } + openedObject = openObjectFromRemote(); } - } - try { final InputStream stream = wrapObjectStream( - fetchedFile.object, - FileUtils.openInputStream(fetchedFile.file) + openedObject.object, + openedObject.objectStream ); return new ResourceCloseableLineIterator( new BufferedReader( new InputStreamReader(stream, Charsets.UTF_8) ), - closeable + openedObject.resourceCloser ); } catch (IOException e) { throw Throwables.propagate(e); } } + + private void checkFetchException() + { + if (fetchFuture != null && fetchFuture.isDone()) { + try { + fetchFuture.get(); + fetchFuture = null; + } + catch (InterruptedException | ExecutionException e) { + throw Throwables.propagate(e); + } + } + } + + private OpenedObject openObjectFromLocal() throws IOException + { + final FetchedFile fetchedFile; + final Closeable resourceCloser; + + if (!fetchFiles.isEmpty()) { + // If there are already fetched files, use them + fetchedFile = fetchFiles.poll(); + resourceCloser = cacheIfPossibleAndGetCloser(fetchedFile, fetchedBytes); + fetchIfNeeded(fetchedBytes.get()); + } else { + // Otherwise, wait for fetching + try { + fetchIfNeeded(fetchedBytes.get()); + fetchedFile = fetchFiles.poll(fetchTimeout, TimeUnit.MILLISECONDS); + if (fetchedFile == null) { + // Check the latest fetch is failed + checkFetchException(); + // Or throw a timeout exception + throw new RuntimeException(new TimeoutException()); + } + resourceCloser = cacheIfPossibleAndGetCloser(fetchedFile, fetchedBytes); + // trigger fetch again for subsequent next() calls + fetchIfNeeded(fetchedBytes.get()); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + } + return new OpenedObject(fetchedFile, resourceCloser); + } + + private OpenedObject openObjectFromRemote() throws IOException + { + final OpenedObject openedObject; + final Closeable resourceCloser = getNoopCloser(); + + if (totalCachedBytes < maxCacheCapacityBytes) { + LOG.info("Caching object[%s]", objects.get(nextFetchIndex)); + try { + // Since maxFetchCapacityBytes is 0, at most one file is fetched. + fetch(); + FetchedFile fetchedFile = fetchFiles.poll(); + if (fetchedFile == null) { + throw new ISE("Cannot fetch object[%s]", objects.get(nextFetchIndex)); + } + cacheIfPossible(fetchedFile); + fetchedBytes.addAndGet(-fetchedFile.length()); + openedObject = new OpenedObject(fetchedFile, resourceCloser); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } else { + final ObjectType object = objects.get(nextFetchIndex++); + LOG.info("Reading object[%s]", object); + openedObject = new OpenedObject(object, openObjectStream(object), resourceCloser); + } + return openedObject; + } }, firehoseParser, () -> { @@ -358,26 +402,26 @@ private boolean cacheIfPossible(FetchedFile fetchedFile) } } - private Closeable cacheIfPossibleAndGetCloseable(FetchedFile fetchedFile, AtomicLong fetchedBytes) + private Closeable cacheIfPossibleAndGetCloser(FetchedFile fetchedFile, AtomicLong fetchedBytes) { final Closeable closeable; if (cacheIfPossible(fetchedFile)) { - closeable = getNoopCloseable(); + closeable = getNoopCloser(); // 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()); } else { - closeable = getFetchedFileCleanupCloseable(fetchedFile, fetchedBytes); + closeable = getFetchedFileCloser(fetchedFile, fetchedBytes); } return closeable; } - private Closeable getNoopCloseable() + private Closeable getNoopCloser() { return () -> {}; } - private Closeable getFetchedFileCleanupCloseable( + private Closeable getFetchedFileCloser( final FetchedFile fetchedFile, final AtomicLong fetchedBytes ) @@ -436,4 +480,25 @@ public void delete() file.delete(); } } + + private class OpenedObject + { + private final ObjectType object; + private final InputStream objectStream; + private final Closeable resourceCloser; + + public OpenedObject(FetchedFile fetchedFile, Closeable resourceCloser) throws IOException + { + this.object = fetchedFile.object; + this.objectStream = FileUtils.openInputStream(fetchedFile.file); + this.resourceCloser = resourceCloser; + } + + public OpenedObject(ObjectType object, InputStream objectStream, Closeable resourceCloser) + { + this.object = object; + this.objectStream = objectStream; + this.resourceCloser = resourceCloser; + } + } } diff --git a/docs/content/development/extensions-contrib/azure.md b/docs/content/development/extensions-contrib/azure.md index cb9f090cc8ee..cb4709677655 100644 --- a/docs/content/development/extensions-contrib/azure.md +++ b/docs/content/development/extensions-contrib/azure.md @@ -56,13 +56,13 @@ shardSpecs are not specified, and, in this case, caching can be useful. Prefetch |property|description|default|required?| |--------|-----------|-------|---------| -|type|This should be "static-azure-blobstore".|N/A|yes| +|type|This should be `static-azure-blobstore`.|N/A|yes| |blobs|JSON array of [Azure blobs](https://msdn.microsoft.com/en-us/library/azure/ee691964.aspx).|N/A|yes| -|maxCacheCapacityBytes|Maximum size of the cache space in bytes|1073741824|no| -|maxFetchCapacityBytes|Maximum size of the fetch space in bytes|1073741824|no| -|prefetchTriggerBytes|Threshold to trigger prefetching Azure objects|maxFetchCapacityBytes / 2|no| -|fetchTimeout|Timeout for fetching an Azure object|60000|no| -|maxFetchRetry|Maximum retry for fetching an Azure object|3|no| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache.|1073741824|no| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch.|1073741824|no| +|prefetchTriggerBytes|Threshold to trigger prefetching Azure objects.|maxFetchCapacityBytes / 2|no| +|fetchTimeout|Timeout for fetching an Azure object.|60000|no| +|maxFetchRetry|Maximum retry for fetching an Azure object.|3|no| Azure Blobs: diff --git a/docs/content/development/extensions-contrib/cloudfiles.md b/docs/content/development/extensions-contrib/cloudfiles.md index 788a5cef7a28..d11a9a0ec823 100644 --- a/docs/content/development/extensions-contrib/cloudfiles.md +++ b/docs/content/development/extensions-contrib/cloudfiles.md @@ -56,13 +56,13 @@ shardSpecs are not specified, and, in this case, caching can be useful. Prefetch |property|description|default|required?| |--------|-----------|-------|---------| -|type|This should be "static-cloudfiles".|N/A|yes| +|type|This should be `static-cloudfiles`.|N/A|yes| |blobs|JSON array of Cloud Files blobs.|N/A|yes| -|maxCacheCapacityBytes|Maximum size of the cache space in bytes|1073741824|no| -|maxFetchCapacityBytes|Maximum size of the fetch space in bytes|1073741824|no| -|prefetchTriggerBytes|Threshold to trigger prefetching Cloud Files objects|maxFetchCapacityBytes / 2|no| -|fetchTimeout|Timeout for fetching a Cloud Files object|60000|no| -|maxFetchRetry|Maximum retry for fetching a Cloud Files object|3|no| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache.|1073741824|no| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch.|1073741824|no| +|prefetchTriggerBytes|Threshold to trigger prefetching Cloud Files objects.|maxFetchCapacityBytes / 2|no| +|fetchTimeout|Timeout for fetching a Cloud Files object.|60000|no| +|maxFetchRetry|Maximum retry for fetching a Cloud Files object.|3|no| Cloud Files Blobs: diff --git a/docs/content/development/extensions-contrib/google.md b/docs/content/development/extensions-contrib/google.md index 66416ff8fa71..0852b886cebc 100644 --- a/docs/content/development/extensions-contrib/google.md +++ b/docs/content/development/extensions-contrib/google.md @@ -46,13 +46,13 @@ shardSpecs are not specified, and, in this case, caching can be useful. Prefetch |property|description|default|required?| |--------|-----------|-------|---------| -|type|This should be "static-google-blobstore".|N/A|yes| +|type|This should be `static-google-blobstore`.|N/A|yes| |blobs|JSON array of Google Blobs.|N/A|yes| -|maxCacheCapacityBytes|Maximum size of the cache space in bytes|1073741824|no| -|maxFetchCapacityBytes|Maximum size of the fetch space in bytes|1073741824|no| -|prefetchTriggerBytes|Threshold to trigger prefetching Google blobs|maxFetchCapacityBytes / 2|no| -|fetchTimeout|Timeout for fetching a Google blob|60000|no| -|maxFetchRetry|Maximum retry for fetching a Google blob|3|no| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache.|1073741824|no| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch.|1073741824|no| +|prefetchTriggerBytes|Threshold to trigger prefetching Google Blobs.|maxFetchCapacityBytes / 2|no| +|fetchTimeout|Timeout for fetching a Google Blob.|60000|no| +|maxFetchRetry|Maximum retry for fetching a Google Blob.|3|no| Google Blobs: diff --git a/docs/content/development/extensions-core/s3.md b/docs/content/development/extensions-core/s3.md index 8804cf3cb577..6a804f959975 100644 --- a/docs/content/development/extensions-core/s3.md +++ b/docs/content/development/extensions-core/s3.md @@ -37,11 +37,11 @@ shardSpecs are not specified, and, in this case, caching can be useful. Prefetch |property|description|default|required?| |--------|-----------|-------|---------| -|type|This should be "static-s3"|N/A|yes| +|type|This should be `static-s3`.|N/A|yes| |uris|JSON array of URIs where s3 files to be ingested are located.|N/A|`uris` or `prefixes` must be set| -|prefixes|JSON array of URI prefixes for the locations of s3 files to be ingested|N/A|`uris` or `prefixes` must be set| -|maxCacheCapacityBytes|Maximum size of the cache space in bytes|1073741824|no| -|maxFetchCapacityBytes|Maximum size of the fetch space in bytes|1073741824|no| -|prefetchTriggerBytes|Threshold to trigger prefetching s3 objects|maxFetchCapacityBytes / 2|no| -|fetchTimeout|Timeout for fetching an s3 object|60000|no| -|maxFetchRetry|Maximum retry for fetching an s3 object|3|no| \ No newline at end of file +|prefixes|JSON array of URI prefixes for the locations of s3 files to be ingested.|N/A|`uris` or `prefixes` must be set| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache.|1073741824|no| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch.|1073741824|no| +|prefetchTriggerBytes|Threshold to trigger prefetching s3 objects.|maxFetchCapacityBytes / 2|no| +|fetchTimeout|Timeout for fetching an s3 object.|60000|no| +|maxFetchRetry|Maximum retry for fetching an s3 object.|3|no| \ No newline at end of file From 4611652eabd746c8ae8417e2c52c19ccce88022d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 11 May 2017 17:16:19 +0900 Subject: [PATCH 20/26] Fix race condition --- .../PrefetchableTextFilesFirehoseFactory.java | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 35e42c2b67ad..8c1e192fbf8c 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -170,10 +170,10 @@ public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirec private int cacheIterateIndex; // nextFetchIndex indicates which object should be downloaded when fetch is triggered. - // This variable is only updated by the fetchExecutor thread, but read by the main thread (in hasNext()) - // and the fetchExecutor thread (in fetch()). To guarantee that the main thread can read the most fresh value, - // this variable should be volatile. - private volatile int nextFetchIndex; + // When prefetching is enabled, this variable is updated by the fetchExecutor thread, but read by the main + // thread (in hasNext()) and the fetchExecutor thread (in fetch()). To guarantee that fetchFiles and + // nextFetchIndex are updated atomically, a lock for fetchFiles must be held before updating this variable. + private int nextFetchIndex; { cacheInitialized = totalCachedBytes > 0; @@ -214,8 +214,10 @@ private void fetch() throws Exception LOG.info("Fetching object[%s], fetchedBytes[%d]", object, fetchedBytes.get()); final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, temporaryDirectory); fetchedBytes.addAndGet(download(object, outFile, 0)); - fetchFiles.put(new FetchedFile(object, outFile)); - nextFetchIndex++; + synchronized (fetchFiles) { + fetchFiles.put(new FetchedFile(object, outFile)); + nextFetchIndex++; + } } } @@ -254,9 +256,11 @@ private long download(ObjectType object, File outFile, int tryCount) throws IOEx @Override public boolean hasNext() { - return (cacheInitialized && cacheIterateIndex < cacheFiles.size()) - || !fetchFiles.isEmpty() - || nextFetchIndex < objects.size(); + synchronized (fetchFiles) { + return (cacheInitialized && cacheIterateIndex < cacheFiles.size()) + || !fetchFiles.isEmpty() + || nextFetchIndex < objects.size(); + } } @Override From 2e172efd8d1229f8b328f103264e70366da3e984 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 11 May 2017 18:06:02 +0900 Subject: [PATCH 21/26] Add fetchLock --- .../impl/PrefetchableTextFilesFirehoseFactory.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 8c1e192fbf8c..521a85a06976 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -158,6 +158,11 @@ public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirec return new FileIteratingFirehose( new Iterator() { + // When prefetching is enabled, fetchFiles and nextFetchIndex are updated by the fetchExecutor thread, but + // read by both the main thread (in hasNext()) and the fetchExecutor thread (in fetch()). To guarantee that + // fetchFiles and nextFetchIndex are updated atomically, this lock must be held before updating + // them. + private final Object fetchLock = new Object(); private final LinkedBlockingQueue fetchFiles = new LinkedBlockingQueue<>(); // Number of bytes currently fetched files. @@ -168,11 +173,7 @@ public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirec private Future fetchFuture; private int cacheIterateIndex; - // nextFetchIndex indicates which object should be downloaded when fetch is triggered. - // When prefetching is enabled, this variable is updated by the fetchExecutor thread, but read by the main - // thread (in hasNext()) and the fetchExecutor thread (in fetch()). To guarantee that fetchFiles and - // nextFetchIndex are updated atomically, a lock for fetchFiles must be held before updating this variable. private int nextFetchIndex; { @@ -214,7 +215,7 @@ private void fetch() throws Exception LOG.info("Fetching object[%s], fetchedBytes[%d]", object, fetchedBytes.get()); final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, temporaryDirectory); fetchedBytes.addAndGet(download(object, outFile, 0)); - synchronized (fetchFiles) { + synchronized (fetchLock) { fetchFiles.put(new FetchedFile(object, outFile)); nextFetchIndex++; } @@ -256,7 +257,7 @@ private long download(ObjectType object, File outFile, int tryCount) throws IOEx @Override public boolean hasNext() { - synchronized (fetchFiles) { + synchronized (fetchLock) { return (cacheInitialized && cacheIterateIndex < cacheFiles.size()) || !fetchFiles.isEmpty() || nextFetchIndex < objects.size(); From e8fad8f7728325520c8427d36b08d464dda709f8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 12 May 2017 16:27:48 +0900 Subject: [PATCH 22/26] Remove ReplayableFirehoseFactoryTest --- .../io/druid/realtime/firehose/ReplayableFirehoseFactoryTest.java | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 server/src/test/java/io/druid/realtime/firehose/ReplayableFirehoseFactoryTest.java diff --git a/server/src/test/java/io/druid/realtime/firehose/ReplayableFirehoseFactoryTest.java b/server/src/test/java/io/druid/realtime/firehose/ReplayableFirehoseFactoryTest.java deleted file mode 100644 index e69de29bb2d1..000000000000 From ac5f528f865fd71f2bd913c8e515f21b3bbddb06 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 16 May 2017 15:36:36 +0900 Subject: [PATCH 23/26] Fix compilation error --- .../segment/realtime/firehose/LocalFirehoseFactoryTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java index 957a9725c001..99d711fdecb3 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java @@ -87,7 +87,9 @@ public void testConnect() throws IOException Lists.newArrayList() ), ",", - Arrays.asList("timestamp", "a") + Arrays.asList("timestamp", "a"), + false, + 0 ), Charsets.UTF_8.name() ), null)) { From d3ece3f504754d8930a4a769184c5abd72e3dd6d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 16 May 2017 16:31:55 +0900 Subject: [PATCH 24/26] Fix test failure --- .../main/java/io/druid/indexing/common/task/IndexTask.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 4a1a846bb7f5..a39a2238c79b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -415,6 +415,10 @@ public SegmentIdentifier allocate(DateTime timestamp, String sequenceName, Strin try { final InputRow inputRow = firehose.nextRow(); + if (inputRow == null) { + continue; + } + final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); if (!optInterval.isPresent()) { fireDepartmentMetrics.incrementThrownAway(); From 0933d1b036ce7bbdb1912867f428fe04a2f6369d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 18 May 2017 09:30:06 +0900 Subject: [PATCH 25/26] Address comments --- .../AbstractTextFilesFirehoseFactory.java | 8 +---- .../PrefetchableTextFilesFirehoseFactory.java | 5 +-- .../input/impl/FileIteratingFirehoseTest.java | 7 ++-- ...fetchableTextFilesFirehoseFactoryTest.java | 32 +++++++++++++++++++ 4 files changed, 38 insertions(+), 14 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java index a42efb4dd53b..a8428b8fd299 100644 --- a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java @@ -29,11 +29,9 @@ import org.apache.commons.io.IOUtils; import org.apache.commons.io.LineIterator; -import java.io.BufferedReader; import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -76,11 +74,7 @@ public LineIterator next() } final ObjectType object = iterator.next(); try { - return IOUtils.lineIterator( - new BufferedReader( - new InputStreamReader(wrapObjectStream(object, openObjectStream(object)), Charsets.UTF_8) - ) - ); + return IOUtils.lineIterator(wrapObjectStream(object, openObjectStream(object)), Charsets.UTF_8); } catch (Exception e) { LOG.error( diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java index 521a85a06976..4baacbcf2462 100644 --- a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -32,7 +32,6 @@ import org.apache.commons.io.IOUtils; import org.apache.commons.io.LineIterator; -import java.io.BufferedReader; import java.io.Closeable; import java.io.File; import java.io.FileOutputStream; @@ -295,9 +294,7 @@ public LineIterator next() ); return new ResourceCloseableLineIterator( - new BufferedReader( - new InputStreamReader(stream, Charsets.UTF_8) - ), + new InputStreamReader(stream, Charsets.UTF_8), openedObject.resourceCloser ); } diff --git a/api/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java b/api/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java index 4e06c35956e1..f715b78c90b4 100644 --- a/api/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java +++ b/api/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java @@ -68,6 +68,8 @@ public static Collection constructorFeeder() throws IOException return args; } + private static final char[] LINE_CHARS = "\n".toCharArray(); + private final StringInputRowParser parser; private final List inputs; private final List expectedResults; @@ -133,9 +135,8 @@ public void testClose() throws IOException @Override public int read(char[] cbuf, int off, int len) throws IOException { - final char[] chs = "\n".toCharArray(); - System.arraycopy(chs, 0, cbuf, 0, chs.length); - return chs.length; + System.arraycopy(LINE_CHARS, 0, cbuf, 0, LINE_CHARS.length); + return LINE_CHARS.length; } @Override diff --git a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java index a29c2d7511a7..8fefb396ecf9 100644 --- a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java @@ -197,6 +197,38 @@ public void testWithCacheAndFetch() throws IOException assertResult(rows); } + @Test + public void testWithLargeCacheAndSmallFetch() throws IOException + { + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.with(testDir, 2048, 1024); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + + @Test + public void testWithSmallCacheAndLargeFetch() throws IOException + { + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.with(testDir, 1024, 2048); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + @Test public void testRetry() throws IOException { From 5c8aa62b2ebbbb370e09a49ae66580f6f1dea967 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 18 May 2017 14:24:44 +0900 Subject: [PATCH 26/26] Add default implementation for new method --- .../main/java/io/druid/data/input/FirehoseFactory.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/api/src/main/java/io/druid/data/input/FirehoseFactory.java b/api/src/main/java/io/druid/data/input/FirehoseFactory.java index 0600c7be189b..2494c13ea716 100644 --- a/api/src/main/java/io/druid/data/input/FirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/FirehoseFactory.java @@ -27,6 +27,11 @@ import java.io.File; import java.io.IOException; +/** + * FirehoseFactory creates a {@link Firehose} which is an interface holding onto the stream of incoming data. + * It currently provides two methods for creating a {@link Firehose} and their default implementations call each other + * for the backward compatibility. Implementations of this interface must implement one of these methods. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") public interface FirehoseFactory { @@ -60,5 +65,8 @@ default Firehose connect(T parser) throws IOException, ParseException * @param parser an input row parser * @param temporaryDirectory a directory where temporary files are stored */ - Firehose connect(T parser, File temporaryDirectory) throws IOException, ParseException; + default Firehose connect(T parser, File temporaryDirectory) throws IOException, ParseException + { + return connect(parser); + } }