mirror of https://github.com/apache/druid.git
Support retrying for PrefetchableTextFilesFirehoseFactory when prefetch is disabled (#5162)
* Add RetryingInputStream * unnecessary exception * fix PrefetchableTextFilesFirehoseFactoryTest * Fix retrying on connection reset * fix start offset * fix checkstyle * fix check connection reset * address comments * fix compile * address comments * address comments
This commit is contained in:
parent
7b8b0a96d6
commit
5d0619f5ce
|
@ -25,6 +25,7 @@ import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
|
||||||
import io.druid.guice.annotations.ExtensionPoint;
|
import io.druid.guice.annotations.ExtensionPoint;
|
||||||
import io.druid.java.util.common.parsers.ParseException;
|
import io.druid.java.util.common.parsers.ParseException;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
|
@ -67,7 +68,7 @@ public interface FirehoseFactory<T extends InputRowParser>
|
||||||
* @param parser an input row parser
|
* @param parser an input row parser
|
||||||
* @param temporaryDirectory a directory where temporary files are stored
|
* @param temporaryDirectory a directory where temporary files are stored
|
||||||
*/
|
*/
|
||||||
default Firehose connect(T parser, File temporaryDirectory) throws IOException, ParseException
|
default Firehose connect(T parser, @Nullable File temporaryDirectory) throws IOException, ParseException
|
||||||
{
|
{
|
||||||
return connect(parser);
|
return connect(parser);
|
||||||
}
|
}
|
||||||
|
|
|
@ -106,8 +106,6 @@ public abstract class AbstractTextFilesFirehoseFactory<T>
|
||||||
* @param object an object to be read
|
* @param object an object to be read
|
||||||
*
|
*
|
||||||
* @return an input stream for the object
|
* @return an input stream for the object
|
||||||
*
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
protected abstract InputStream openObjectStream(T object) throws IOException;
|
protected abstract InputStream openObjectStream(T object) throws IOException;
|
||||||
|
|
||||||
|
@ -117,8 +115,7 @@ public abstract class AbstractTextFilesFirehoseFactory<T>
|
||||||
*
|
*
|
||||||
* @param object an input object
|
* @param object an input object
|
||||||
* @param stream a stream for the object
|
* @param stream a stream for the object
|
||||||
* @return
|
* @return an wrapped input stream
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
protected abstract InputStream wrapObjectStream(T object, InputStream stream) throws IOException;
|
protected abstract InputStream wrapObjectStream(T object, InputStream stream) throws IOException;
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,11 +19,16 @@
|
||||||
|
|
||||||
package io.druid.data.input.impl.prefetch;
|
package io.druid.data.input.impl.prefetch;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import io.druid.java.util.common.ISE;
|
import io.druid.java.util.common.ISE;
|
||||||
|
import io.druid.java.util.common.RetryUtils;
|
||||||
|
import io.druid.java.util.common.StringUtils;
|
||||||
import io.druid.java.util.common.logger.Logger;
|
import io.druid.java.util.common.logger.Logger;
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileOutputStream;
|
import java.io.FileOutputStream;
|
||||||
|
@ -54,6 +59,8 @@ public class Fetcher<T> implements Iterator<OpenedObject<T>>
|
||||||
private final CacheManager<T> cacheManager;
|
private final CacheManager<T> cacheManager;
|
||||||
private final List<T> objects;
|
private final List<T> objects;
|
||||||
private final ExecutorService fetchExecutor;
|
private final ExecutorService fetchExecutor;
|
||||||
|
|
||||||
|
@Nullable
|
||||||
private final File temporaryDirectory;
|
private final File temporaryDirectory;
|
||||||
|
|
||||||
// A roughly max size of total fetched objects, but the actual fetched size can be bigger. The reason is our current
|
// A roughly max size of total fetched objects, but the actual fetched size can be bigger. The reason is our current
|
||||||
|
@ -80,6 +87,7 @@ public class Fetcher<T> implements Iterator<OpenedObject<T>>
|
||||||
private final AtomicLong fetchedBytes = new AtomicLong(0);
|
private final AtomicLong fetchedBytes = new AtomicLong(0);
|
||||||
|
|
||||||
private final ObjectOpenFunction<T> openObjectFunction;
|
private final ObjectOpenFunction<T> openObjectFunction;
|
||||||
|
private final Predicate<Throwable> retryCondition;
|
||||||
private final byte[] buffer;
|
private final byte[] buffer;
|
||||||
|
|
||||||
private Future<Void> fetchFuture;
|
private Future<Void> fetchFuture;
|
||||||
|
@ -94,12 +102,13 @@ public class Fetcher<T> implements Iterator<OpenedObject<T>>
|
||||||
CacheManager<T> cacheManager,
|
CacheManager<T> cacheManager,
|
||||||
List<T> objects,
|
List<T> objects,
|
||||||
ExecutorService fetchExecutor,
|
ExecutorService fetchExecutor,
|
||||||
File temporaryDirectory,
|
@Nullable File temporaryDirectory,
|
||||||
long maxFetchCapacityBytes,
|
long maxFetchCapacityBytes,
|
||||||
long prefetchTriggerBytes,
|
long prefetchTriggerBytes,
|
||||||
long fetchTimeout,
|
long fetchTimeout,
|
||||||
int maxFetchRetry,
|
int maxFetchRetry,
|
||||||
ObjectOpenFunction<T> openObjectFunction
|
ObjectOpenFunction<T> openObjectFunction,
|
||||||
|
Predicate<Throwable> retryCondition
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.cacheManager = cacheManager;
|
this.cacheManager = cacheManager;
|
||||||
|
@ -111,6 +120,7 @@ public class Fetcher<T> implements Iterator<OpenedObject<T>>
|
||||||
this.fetchTimeout = fetchTimeout;
|
this.fetchTimeout = fetchTimeout;
|
||||||
this.maxFetchRetry = maxFetchRetry;
|
this.maxFetchRetry = maxFetchRetry;
|
||||||
this.openObjectFunction = openObjectFunction;
|
this.openObjectFunction = openObjectFunction;
|
||||||
|
this.retryCondition = retryCondition;
|
||||||
this.buffer = new byte[BUFFER_SIZE];
|
this.buffer = new byte[BUFFER_SIZE];
|
||||||
|
|
||||||
this.prefetchEnabled = maxFetchCapacityBytes > 0;
|
this.prefetchEnabled = maxFetchCapacityBytes > 0;
|
||||||
|
@ -120,6 +130,10 @@ public class Fetcher<T> implements Iterator<OpenedObject<T>>
|
||||||
this.fetchedFiles.addAll(cacheManager.getFiles());
|
this.fetchedFiles.addAll(cacheManager.getFiles());
|
||||||
this.nextFetchIndex = fetchedFiles.size();
|
this.nextFetchIndex = fetchedFiles.size();
|
||||||
|
|
||||||
|
if (cacheManager.isEnabled() || prefetchEnabled) {
|
||||||
|
Preconditions.checkNotNull(temporaryDirectory, "temporaryDirectory");
|
||||||
|
}
|
||||||
|
|
||||||
if (prefetchEnabled) {
|
if (prefetchEnabled) {
|
||||||
fetchIfNeeded(0L);
|
fetchIfNeeded(0L);
|
||||||
}
|
}
|
||||||
|
@ -155,7 +169,7 @@ public class Fetcher<T> implements Iterator<OpenedObject<T>>
|
||||||
final T object = objects.get(nextFetchIndex);
|
final T object = objects.get(nextFetchIndex);
|
||||||
LOG.info("Fetching [%d]th object[%s], fetchedBytes[%d]", nextFetchIndex, object, fetchedBytes.get());
|
LOG.info("Fetching [%d]th object[%s], fetchedBytes[%d]", nextFetchIndex, object, fetchedBytes.get());
|
||||||
final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, temporaryDirectory);
|
final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, temporaryDirectory);
|
||||||
fetchedBytes.addAndGet(download(object, outFile, 0));
|
fetchedBytes.addAndGet(download(object, outFile));
|
||||||
fetchedFiles.put(new FetchedFile<>(object, outFile, getFileCloser(outFile, fetchedBytes)));
|
fetchedFiles.put(new FetchedFile<>(object, outFile, getFileCloser(outFile, fetchedBytes)));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -166,26 +180,27 @@ public class Fetcher<T> implements Iterator<OpenedObject<T>>
|
||||||
*
|
*
|
||||||
* @param object an object to be downloaded
|
* @param object an object to be downloaded
|
||||||
* @param outFile a file which the object data is stored
|
* @param outFile a file which the object data is stored
|
||||||
* @param tryCount current retry count
|
|
||||||
*
|
*
|
||||||
* @return number of downloaded bytes
|
* @return number of downloaded bytes
|
||||||
*/
|
*/
|
||||||
private long download(T object, File outFile, int tryCount) throws IOException
|
private long download(T object, File outFile) throws IOException
|
||||||
{
|
{
|
||||||
|
try {
|
||||||
|
return RetryUtils.retry(
|
||||||
|
() -> {
|
||||||
try (final InputStream is = openObjectFunction.open(object);
|
try (final InputStream is = openObjectFunction.open(object);
|
||||||
final OutputStream os = new FileOutputStream(outFile)) {
|
final OutputStream os = new FileOutputStream(outFile)) {
|
||||||
return IOUtils.copyLarge(is, os, buffer);
|
return IOUtils.copyLarge(is, os, buffer);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
},
|
||||||
final int nextTry = tryCount + 1;
|
retryCondition,
|
||||||
if (!Thread.currentThread().isInterrupted() && nextTry < maxFetchRetry) {
|
outFile::delete,
|
||||||
LOG.error(e, "Failed to download object[%s], retrying (%d of %d)", object, nextTry, maxFetchRetry);
|
maxFetchRetry + 1,
|
||||||
outFile.delete();
|
StringUtils.format("Failed to download object[%s]", object)
|
||||||
return download(object, outFile, nextTry);
|
);
|
||||||
} else {
|
|
||||||
LOG.error(e, "Failed to download object[%s], retries exhausted, aborting", object);
|
|
||||||
throw e;
|
|
||||||
}
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
throw new IOException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -289,7 +304,11 @@ public class Fetcher<T> implements Iterator<OpenedObject<T>>
|
||||||
final T object = objects.get(nextFetchIndex);
|
final T object = objects.get(nextFetchIndex);
|
||||||
LOG.info("Reading [%d]th object[%s]", nextFetchIndex, object);
|
LOG.info("Reading [%d]th object[%s]", nextFetchIndex, object);
|
||||||
nextFetchIndex++;
|
nextFetchIndex++;
|
||||||
return new OpenedObject<>(object, openObjectFunction.open(object), getNoopCloser());
|
return new OpenedObject<>(
|
||||||
|
object,
|
||||||
|
new RetryingInputStream<>(object, openObjectFunction, retryCondition, maxFetchRetry),
|
||||||
|
getNoopCloser()
|
||||||
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -25,4 +25,6 @@ import java.io.InputStream;
|
||||||
interface ObjectOpenFunction<T>
|
interface ObjectOpenFunction<T>
|
||||||
{
|
{
|
||||||
InputStream open(T object) throws IOException;
|
InputStream open(T object) throws IOException;
|
||||||
|
|
||||||
|
InputStream open(T object, long start) throws IOException;
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,6 +22,7 @@ package io.druid.data.input.impl.prefetch;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import io.druid.data.input.Firehose;
|
import io.druid.data.input.Firehose;
|
||||||
import io.druid.data.input.impl.AbstractTextFilesFirehoseFactory;
|
import io.druid.data.input.impl.AbstractTextFilesFirehoseFactory;
|
||||||
|
@ -32,6 +33,7 @@ import io.druid.java.util.common.concurrent.Execs;
|
||||||
import io.druid.java.util.common.logger.Logger;
|
import io.druid.java.util.common.logger.Logger;
|
||||||
import org.apache.commons.io.LineIterator;
|
import org.apache.commons.io.LineIterator;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -159,22 +161,25 @@ public abstract class PrefetchableTextFilesFirehoseFactory<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirectory) throws IOException
|
public Firehose connect(StringInputRowParser firehoseParser, @Nullable File temporaryDirectory) throws IOException
|
||||||
{
|
{
|
||||||
if (!cacheManager.isEnabled() && maxFetchCapacityBytes == 0) {
|
|
||||||
return super.connect(firehoseParser, temporaryDirectory);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (objects == null) {
|
if (objects == null) {
|
||||||
objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "objects"));
|
objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "objects"));
|
||||||
}
|
}
|
||||||
|
|
||||||
Preconditions.checkState(temporaryDirectory.exists(), "temporaryDirectory[%s] does not exist", temporaryDirectory);
|
if (cacheManager.isEnabled() || maxFetchCapacityBytes > 0) {
|
||||||
Preconditions.checkState(
|
Preconditions.checkNotNull(temporaryDirectory, "temporaryDirectory");
|
||||||
|
Preconditions.checkArgument(
|
||||||
|
temporaryDirectory.exists(),
|
||||||
|
"temporaryDirectory[%s] does not exist",
|
||||||
|
temporaryDirectory
|
||||||
|
);
|
||||||
|
Preconditions.checkArgument(
|
||||||
temporaryDirectory.isDirectory(),
|
temporaryDirectory.isDirectory(),
|
||||||
"temporaryDirectory[%s] is not a directory",
|
"temporaryDirectory[%s] is not a directory",
|
||||||
temporaryDirectory
|
temporaryDirectory
|
||||||
);
|
);
|
||||||
|
}
|
||||||
|
|
||||||
LOG.info("Create a new firehose for [%d] objects", objects.size());
|
LOG.info("Create a new firehose for [%d] objects", objects.size());
|
||||||
|
|
||||||
|
@ -189,7 +194,21 @@ public abstract class PrefetchableTextFilesFirehoseFactory<T>
|
||||||
prefetchTriggerBytes,
|
prefetchTriggerBytes,
|
||||||
fetchTimeout,
|
fetchTimeout,
|
||||||
maxFetchRetry,
|
maxFetchRetry,
|
||||||
this::openObjectStream
|
new ObjectOpenFunction<T>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public InputStream open(T object) throws IOException
|
||||||
|
{
|
||||||
|
return openObjectStream(object);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public InputStream open(T object, long start) throws IOException
|
||||||
|
{
|
||||||
|
return openObjectStream(object, start);
|
||||||
|
}
|
||||||
|
},
|
||||||
|
getRetryCondition()
|
||||||
);
|
);
|
||||||
|
|
||||||
return new FileIteratingFirehose(
|
return new FileIteratingFirehose(
|
||||||
|
@ -240,6 +259,23 @@ public abstract class PrefetchableTextFilesFirehoseFactory<T>
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a predicate describing retry conditions. {@link Fetcher} and {@link RetryingInputStream} will retry on the
|
||||||
|
* errors satisfying this condition.
|
||||||
|
*/
|
||||||
|
protected abstract Predicate<Throwable> getRetryCondition();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Open an input stream from the given object. If the object is compressed, this method should return a byte stream
|
||||||
|
* as it is compressed. The object compression should be handled in {@link #wrapObjectStream(Object, InputStream)}.
|
||||||
|
*
|
||||||
|
* @param object an object to be read
|
||||||
|
* @param start start offset
|
||||||
|
*
|
||||||
|
* @return an input stream for the object
|
||||||
|
*/
|
||||||
|
protected abstract InputStream openObjectStream(T object, long start) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class calls the {@link Closeable#close()} method of the resourceCloser when it is closed.
|
* This class calls the {@link Closeable#close()} method of the resourceCloser when it is closed.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -0,0 +1,184 @@
|
||||||
|
/*
|
||||||
|
* 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.prefetch;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
|
import com.google.common.base.Throwables;
|
||||||
|
import com.google.common.io.CountingInputStream;
|
||||||
|
import io.druid.java.util.common.RetryUtils;
|
||||||
|
import io.druid.java.util.common.logger.Logger;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.net.SocketException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class is used by {@link Fetcher} when prefetch is disabled. It's responsible for re-opening the underlying input
|
||||||
|
* stream for the input object on the socket connection reset as well as the given {@link #retryCondition}.
|
||||||
|
*
|
||||||
|
* @param <T> object type
|
||||||
|
*/
|
||||||
|
class RetryingInputStream<T> extends InputStream
|
||||||
|
{
|
||||||
|
private static final Logger log = new Logger(RetryingInputStream.class);
|
||||||
|
|
||||||
|
private final T object;
|
||||||
|
private final ObjectOpenFunction<T> objectOpenFunction;
|
||||||
|
private final Predicate<Throwable> retryCondition;
|
||||||
|
private final int maxRetry;
|
||||||
|
|
||||||
|
private CountingInputStream delegate;
|
||||||
|
private long startOffset;
|
||||||
|
|
||||||
|
RetryingInputStream(
|
||||||
|
T object,
|
||||||
|
ObjectOpenFunction<T> objectOpenFunction,
|
||||||
|
Predicate<Throwable> retryCondition,
|
||||||
|
int maxRetry
|
||||||
|
) throws IOException
|
||||||
|
{
|
||||||
|
this.object = object;
|
||||||
|
this.objectOpenFunction = objectOpenFunction;
|
||||||
|
this.retryCondition = retryCondition;
|
||||||
|
this.maxRetry = maxRetry;
|
||||||
|
this.delegate = new CountingInputStream(objectOpenFunction.open(object));
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean isConnectionReset(Throwable t)
|
||||||
|
{
|
||||||
|
return (t instanceof SocketException && (t.getMessage() != null && t.getMessage().contains("Connection reset"))) ||
|
||||||
|
(t.getCause() != null && isConnectionReset(t.getCause()));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void waitOrThrow(Throwable t, int nTry) throws IOException
|
||||||
|
{
|
||||||
|
final boolean isConnectionReset = isConnectionReset(t);
|
||||||
|
if (isConnectionReset || retryCondition.apply(t)) {
|
||||||
|
if (isConnectionReset) {
|
||||||
|
// Re-open the input stream on connection reset
|
||||||
|
startOffset += delegate.getCount();
|
||||||
|
try {
|
||||||
|
delegate.close();
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
// ignore this exception
|
||||||
|
log.warn(e, "Error while closing the delegate input stream");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
// Wait for the next try
|
||||||
|
RetryUtils.awaitNextRetry(t, null, nTry + 1, maxRetry, false);
|
||||||
|
|
||||||
|
if (isConnectionReset) {
|
||||||
|
log.info("retrying from offset[%d]", startOffset);
|
||||||
|
delegate = new CountingInputStream(objectOpenFunction.open(object, startOffset));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (InterruptedException | IOException e) {
|
||||||
|
t.addSuppressed(e);
|
||||||
|
throwAsIOException(t);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
throwAsIOException(t);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void throwAsIOException(Throwable t) throws IOException
|
||||||
|
{
|
||||||
|
Throwables.propagateIfInstanceOf(t, IOException.class);
|
||||||
|
throw new IOException(t);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int read() throws IOException
|
||||||
|
{
|
||||||
|
for (int nTry = 0; nTry < maxRetry; nTry++) {
|
||||||
|
try {
|
||||||
|
return delegate.read();
|
||||||
|
}
|
||||||
|
catch (Throwable t) {
|
||||||
|
waitOrThrow(t, nTry);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return delegate.read();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int read(byte b[]) throws IOException
|
||||||
|
{
|
||||||
|
for (int nTry = 0; nTry < maxRetry; nTry++) {
|
||||||
|
try {
|
||||||
|
return delegate.read(b);
|
||||||
|
}
|
||||||
|
catch (Throwable t) {
|
||||||
|
waitOrThrow(t, nTry);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return delegate.read(b);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int read(byte b[], int off, int len) throws IOException
|
||||||
|
{
|
||||||
|
for (int nTry = 0; nTry < maxRetry; nTry++) {
|
||||||
|
try {
|
||||||
|
return delegate.read(b, off, len);
|
||||||
|
}
|
||||||
|
catch (Throwable t) {
|
||||||
|
waitOrThrow(t, nTry);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return delegate.read(b, off, len);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long skip(long n) throws IOException
|
||||||
|
{
|
||||||
|
for (int nTry = 0; nTry < maxRetry; nTry++) {
|
||||||
|
try {
|
||||||
|
return delegate.skip(n);
|
||||||
|
}
|
||||||
|
catch (Throwable t) {
|
||||||
|
waitOrThrow(t, nTry);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return delegate.skip(n);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int available() throws IOException
|
||||||
|
{
|
||||||
|
for (int nTry = 0; nTry < maxRetry; nTry++) {
|
||||||
|
try {
|
||||||
|
return delegate.available();
|
||||||
|
}
|
||||||
|
catch (Throwable t) {
|
||||||
|
waitOrThrow(t, nTry);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return delegate.available();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
delegate.close();
|
||||||
|
}
|
||||||
|
}
|
|
@ -21,6 +21,7 @@ package io.druid.data.input.impl.prefetch;
|
||||||
|
|
||||||
import com.google.common.base.Charsets;
|
import com.google.common.base.Charsets;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.io.CountingOutputStream;
|
import com.google.common.io.CountingOutputStream;
|
||||||
import io.druid.data.input.Firehose;
|
import io.druid.data.input.Firehose;
|
||||||
|
@ -30,6 +31,7 @@ import io.druid.data.input.impl.DimensionsSpec;
|
||||||
import io.druid.data.input.impl.StringInputRowParser;
|
import io.druid.data.input.impl.StringInputRowParser;
|
||||||
import io.druid.data.input.impl.TimestampSpec;
|
import io.druid.data.input.impl.TimestampSpec;
|
||||||
import io.druid.java.util.common.DateTimes;
|
import io.druid.java.util.common.DateTimes;
|
||||||
|
import io.druid.java.util.common.RetryUtils;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
import org.apache.commons.io.filefilter.TrueFileFilter;
|
import org.apache.commons.io.filefilter.TrueFileFilter;
|
||||||
|
@ -47,6 +49,7 @@ import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.io.OutputStreamWriter;
|
import java.io.OutputStreamWriter;
|
||||||
import java.io.Writer;
|
import java.io.Writer;
|
||||||
|
import java.net.SocketException;
|
||||||
import java.nio.charset.StandardCharsets;
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.nio.file.Files;
|
import java.nio.file.Files;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -187,6 +190,25 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
||||||
assertNumRemainingCacheFiles(firehoseTmpDir, 0);
|
assertNumRemainingCacheFiles(firehoseTmpDir, 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testWithoutCacheAndFetchAgainstConnectionReset() throws IOException
|
||||||
|
{
|
||||||
|
final TestPrefetchableTextFilesFirehoseFactory factory =
|
||||||
|
TestPrefetchableTextFilesFirehoseFactory.withConnectionResets(TEST_DIR, 0, 0, 2);
|
||||||
|
|
||||||
|
final List<Row> rows = new ArrayList<>();
|
||||||
|
final File firehoseTmpDir = createFirehoseTmpDir("testWithoutCacheAndFetch");
|
||||||
|
try (Firehose firehose = factory.connect(parser, firehoseTmpDir)) {
|
||||||
|
while (firehose.hasMore()) {
|
||||||
|
rows.add(firehose.nextRow());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Assert.assertEquals(0, factory.getCacheManager().getTotalCachedBytes());
|
||||||
|
assertResult(rows);
|
||||||
|
assertNumRemainingCacheFiles(firehoseTmpDir, 0);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testWithoutCache() throws IOException
|
public void testWithoutCache() throws IOException
|
||||||
{
|
{
|
||||||
|
@ -377,10 +399,10 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
||||||
|
|
||||||
static class TestPrefetchableTextFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<File>
|
static class TestPrefetchableTextFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<File>
|
||||||
{
|
{
|
||||||
private static final long defaultTimeout = 1000;
|
|
||||||
private final long sleepMillis;
|
private final long sleepMillis;
|
||||||
private final File baseDir;
|
private final File baseDir;
|
||||||
private int openExceptionCount;
|
private int numOpenExceptions;
|
||||||
|
private int maxConnectionResets;
|
||||||
|
|
||||||
static TestPrefetchableTextFilesFirehoseFactory with(File baseDir, long cacheCapacity, long fetchCapacity)
|
static TestPrefetchableTextFilesFirehoseFactory with(File baseDir, long cacheCapacity, long fetchCapacity)
|
||||||
{
|
{
|
||||||
|
@ -389,9 +411,9 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
||||||
1024,
|
1024,
|
||||||
cacheCapacity,
|
cacheCapacity,
|
||||||
fetchCapacity,
|
fetchCapacity,
|
||||||
defaultTimeout,
|
|
||||||
3,
|
3,
|
||||||
0,
|
0,
|
||||||
|
0,
|
||||||
0
|
0
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -403,9 +425,9 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
||||||
1024,
|
1024,
|
||||||
2048,
|
2048,
|
||||||
2048,
|
2048,
|
||||||
defaultTimeout,
|
|
||||||
3,
|
3,
|
||||||
0,
|
0,
|
||||||
|
0,
|
||||||
0
|
0
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -417,9 +439,28 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
||||||
1024,
|
1024,
|
||||||
2048,
|
2048,
|
||||||
2048,
|
2048,
|
||||||
defaultTimeout,
|
|
||||||
3,
|
3,
|
||||||
count,
|
count,
|
||||||
|
0,
|
||||||
|
0
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
static TestPrefetchableTextFilesFirehoseFactory withConnectionResets(
|
||||||
|
File baseDir,
|
||||||
|
long cacheCapacity,
|
||||||
|
long fetchCapacity,
|
||||||
|
int numConnectionResets
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return new TestPrefetchableTextFilesFirehoseFactory(
|
||||||
|
baseDir,
|
||||||
|
fetchCapacity / 2,
|
||||||
|
cacheCapacity,
|
||||||
|
fetchCapacity,
|
||||||
|
3,
|
||||||
|
0,
|
||||||
|
numConnectionResets,
|
||||||
0
|
0
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -434,18 +475,54 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
||||||
100,
|
100,
|
||||||
3,
|
3,
|
||||||
0,
|
0,
|
||||||
|
0,
|
||||||
ms
|
ms
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
public TestPrefetchableTextFilesFirehoseFactory(
|
private static long computeTimeout(int maxRetry)
|
||||||
|
{
|
||||||
|
// See RetryUtils.nextRetrySleepMillis()
|
||||||
|
final double maxFuzzyMultiplier = 2.;
|
||||||
|
return (long) Math.min(
|
||||||
|
RetryUtils.MAX_SLEEP_MILLIS,
|
||||||
|
RetryUtils.BASE_SLEEP_MILLIS * Math.pow(2, maxRetry - 1) * maxFuzzyMultiplier
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
TestPrefetchableTextFilesFirehoseFactory(
|
||||||
|
File baseDir,
|
||||||
|
long prefetchTriggerThreshold,
|
||||||
|
long maxCacheCapacityBytes,
|
||||||
|
long maxFetchCapacityBytes,
|
||||||
|
int maxRetry,
|
||||||
|
int numOpenExceptions,
|
||||||
|
int numConnectionResets,
|
||||||
|
long sleepMillis
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this(
|
||||||
|
baseDir,
|
||||||
|
prefetchTriggerThreshold,
|
||||||
|
maxCacheCapacityBytes,
|
||||||
|
maxFetchCapacityBytes,
|
||||||
|
computeTimeout(maxRetry),
|
||||||
|
maxRetry,
|
||||||
|
numOpenExceptions,
|
||||||
|
numConnectionResets,
|
||||||
|
sleepMillis
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
TestPrefetchableTextFilesFirehoseFactory(
|
||||||
File baseDir,
|
File baseDir,
|
||||||
long prefetchTriggerThreshold,
|
long prefetchTriggerThreshold,
|
||||||
long maxCacheCapacityBytes,
|
long maxCacheCapacityBytes,
|
||||||
long maxFetchCapacityBytes,
|
long maxFetchCapacityBytes,
|
||||||
long timeout,
|
long timeout,
|
||||||
int maxRetry,
|
int maxRetry,
|
||||||
int openExceptionCount,
|
int numOpenExceptions,
|
||||||
|
int maxConnectionResets,
|
||||||
long sleepMillis
|
long sleepMillis
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -456,7 +533,8 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
||||||
timeout,
|
timeout,
|
||||||
maxRetry
|
maxRetry
|
||||||
);
|
);
|
||||||
this.openExceptionCount = openExceptionCount;
|
this.numOpenExceptions = numOpenExceptions;
|
||||||
|
this.maxConnectionResets = maxConnectionResets;
|
||||||
this.sleepMillis = sleepMillis;
|
this.sleepMillis = sleepMillis;
|
||||||
this.baseDir = baseDir;
|
this.baseDir = baseDir;
|
||||||
}
|
}
|
||||||
|
@ -474,8 +552,8 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
||||||
@Override
|
@Override
|
||||||
protected InputStream openObjectStream(File object) throws IOException
|
protected InputStream openObjectStream(File object) throws IOException
|
||||||
{
|
{
|
||||||
if (openExceptionCount > 0) {
|
if (numOpenExceptions > 0) {
|
||||||
openExceptionCount--;
|
numOpenExceptions--;
|
||||||
throw new IOException("Exception for retry test");
|
throw new IOException("Exception for retry test");
|
||||||
}
|
}
|
||||||
if (sleepMillis > 0) {
|
if (sleepMillis > 0) {
|
||||||
|
@ -486,7 +564,9 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return FileUtils.openInputStream(object);
|
return maxConnectionResets > 0 ?
|
||||||
|
new TestInputStream(FileUtils.openInputStream(object), maxConnectionResets) :
|
||||||
|
FileUtils.openInputStream(object);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -494,5 +574,76 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
||||||
{
|
{
|
||||||
return stream;
|
return stream;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Predicate<Throwable> getRetryCondition()
|
||||||
|
{
|
||||||
|
return e -> e instanceof IOException;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected InputStream openObjectStream(File object, long start) throws IOException
|
||||||
|
{
|
||||||
|
if (numOpenExceptions > 0) {
|
||||||
|
numOpenExceptions--;
|
||||||
|
throw new IOException("Exception for retry test");
|
||||||
|
}
|
||||||
|
if (sleepMillis > 0) {
|
||||||
|
try {
|
||||||
|
Thread.sleep(sleepMillis);
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
final InputStream in = FileUtils.openInputStream(object);
|
||||||
|
in.skip(start);
|
||||||
|
|
||||||
|
return maxConnectionResets > 0 ? new TestInputStream(in, maxConnectionResets) : in;
|
||||||
|
}
|
||||||
|
|
||||||
|
private int readCount;
|
||||||
|
private int numConnectionResets;
|
||||||
|
|
||||||
|
private class TestInputStream extends InputStream
|
||||||
|
{
|
||||||
|
private static final int NUM_READ_COUNTS_BEFORE_ERROR = 10;
|
||||||
|
private final InputStream delegate;
|
||||||
|
private final int maxConnectionResets;
|
||||||
|
|
||||||
|
TestInputStream(
|
||||||
|
InputStream delegate,
|
||||||
|
int maxConnectionResets
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.delegate = delegate;
|
||||||
|
this.maxConnectionResets = maxConnectionResets;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int read() throws IOException
|
||||||
|
{
|
||||||
|
if (readCount++ % NUM_READ_COUNTS_BEFORE_ERROR == 0) {
|
||||||
|
if (numConnectionResets++ < maxConnectionResets) {
|
||||||
|
// Simulate connection reset
|
||||||
|
throw new SocketException("Test Connection reset");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return delegate.read();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int read(byte b[], int off, int len) throws IOException
|
||||||
|
{
|
||||||
|
if (readCount++ % NUM_READ_COUNTS_BEFORE_ERROR == 0) {
|
||||||
|
if (numConnectionResets++ < maxConnectionResets) {
|
||||||
|
// Simulate connection reset
|
||||||
|
throw new SocketException("Test Connection reset");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return delegate.read(b, off, len);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,146 @@
|
||||||
|
/*
|
||||||
|
* 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.prefetch;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.apache.commons.io.FileUtils;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
|
||||||
|
import java.io.DataInputStream;
|
||||||
|
import java.io.DataOutputStream;
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.FileInputStream;
|
||||||
|
import java.io.FileOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.net.SocketException;
|
||||||
|
import java.util.zip.GZIPInputStream;
|
||||||
|
import java.util.zip.GZIPOutputStream;
|
||||||
|
|
||||||
|
public class RetryingInputStreamTest
|
||||||
|
{
|
||||||
|
private static final int MAX_RETRY = 5;
|
||||||
|
private static final int MAX_ERROR = 4;
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||||
|
|
||||||
|
private File testFile;
|
||||||
|
private DataInputStream inputStream;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setup() throws IOException
|
||||||
|
{
|
||||||
|
testFile = temporaryFolder.newFile();
|
||||||
|
|
||||||
|
try (FileOutputStream fis = new FileOutputStream(testFile);
|
||||||
|
GZIPOutputStream gis = new GZIPOutputStream(fis);
|
||||||
|
DataOutputStream dis = new DataOutputStream(gis)) {
|
||||||
|
for (int i = 0; i < 10000; i++) {
|
||||||
|
dis.writeInt(i);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
throwError = false;
|
||||||
|
|
||||||
|
final InputStream retryingInputStream = new RetryingInputStream<>(
|
||||||
|
testFile,
|
||||||
|
new ObjectOpenFunction<File>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public InputStream open(File object) throws IOException
|
||||||
|
{
|
||||||
|
return new TestInputStream(new FileInputStream(object));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public InputStream open(File object, long start) throws IOException
|
||||||
|
{
|
||||||
|
final FileInputStream fis = new FileInputStream(object);
|
||||||
|
Preconditions.checkState(fis.skip(start) == start);
|
||||||
|
return new TestInputStream(fis);
|
||||||
|
}
|
||||||
|
},
|
||||||
|
e -> e instanceof IOException,
|
||||||
|
MAX_RETRY
|
||||||
|
);
|
||||||
|
|
||||||
|
inputStream = new DataInputStream(new GZIPInputStream(retryingInputStream));
|
||||||
|
|
||||||
|
throwError = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void teardown() throws IOException
|
||||||
|
{
|
||||||
|
inputStream.close();
|
||||||
|
FileUtils.forceDelete(testFile);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testReadRetry() throws IOException
|
||||||
|
{
|
||||||
|
for (int i = 0; i < 10000; i++) {
|
||||||
|
Assert.assertEquals(i, inputStream.readInt());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean throwError = true;
|
||||||
|
private int errorCount = 0;
|
||||||
|
|
||||||
|
private class TestInputStream extends InputStream
|
||||||
|
{
|
||||||
|
private final InputStream delegate;
|
||||||
|
|
||||||
|
TestInputStream(InputStream delegate)
|
||||||
|
{
|
||||||
|
this.delegate = delegate;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int read() throws IOException
|
||||||
|
{
|
||||||
|
return delegate.read();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int read(byte b[], int off, int len) throws IOException
|
||||||
|
{
|
||||||
|
if (throwError) {
|
||||||
|
throwError = false;
|
||||||
|
errorCount++;
|
||||||
|
if (errorCount % 2 == 0) {
|
||||||
|
throw new IOException("test retry");
|
||||||
|
} else {
|
||||||
|
delegate.close();
|
||||||
|
throw new SocketException("Test Connection reset");
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
throwError = errorCount < MAX_ERROR;
|
||||||
|
return delegate.read(b, off, len);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -22,10 +22,13 @@ package io.druid.firehose.azure;
|
||||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
|
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
|
||||||
import io.druid.java.util.common.CompressionUtils;
|
import io.druid.java.util.common.CompressionUtils;
|
||||||
import io.druid.storage.azure.AzureByteSource;
|
import io.druid.storage.azure.AzureByteSource;
|
||||||
import io.druid.storage.azure.AzureStorage;
|
import io.druid.storage.azure.AzureStorage;
|
||||||
|
import io.druid.storage.azure.AzureUtils;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
|
@ -75,6 +78,16 @@ public class StaticAzureBlobStoreFirehoseFactory extends PrefetchableTextFilesFi
|
||||||
return makeByteSource(azureStorage, object).openStream();
|
return makeByteSource(azureStorage, object).openStream();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected InputStream openObjectStream(AzureBlob object, long start) throws IOException
|
||||||
|
{
|
||||||
|
// BlobInputStream.skip() moves the next read offset instead of skipping first 'start' bytes.
|
||||||
|
final InputStream in = openObjectStream(object);
|
||||||
|
final long skip = in.skip(start);
|
||||||
|
Preconditions.checkState(skip == start, "start offset was [%s] but [%s] bytes were skipped", start, skip);
|
||||||
|
return in;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected InputStream wrapObjectStream(AzureBlob object, InputStream stream) throws IOException
|
protected InputStream wrapObjectStream(AzureBlob object, InputStream stream) throws IOException
|
||||||
{
|
{
|
||||||
|
@ -124,4 +137,10 @@ public class StaticAzureBlobStoreFirehoseFactory extends PrefetchableTextFilesFi
|
||||||
getMaxFetchRetry()
|
getMaxFetchRetry()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Predicate<Throwable> getRetryCondition()
|
||||||
|
{
|
||||||
|
return AzureUtils.AZURE_RETRY;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.microsoft.azure.storage.StorageException;
|
import com.microsoft.azure.storage.StorageException;
|
||||||
|
|
||||||
import io.druid.java.util.common.CompressionUtils;
|
import io.druid.java.util.common.CompressionUtils;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
import io.druid.java.util.common.logger.Logger;
|
import io.druid.java.util.common.logger.Logger;
|
||||||
|
@ -40,7 +39,6 @@ import java.net.URI;
|
||||||
import java.net.URISyntaxException;
|
import java.net.URISyntaxException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
public class AzureDataSegmentPusher implements DataSegmentPusher
|
public class AzureDataSegmentPusher implements DataSegmentPusher
|
||||||
{
|
{
|
||||||
|
@ -149,14 +147,7 @@ public class AzureDataSegmentPusher implements DataSegmentPusher
|
||||||
final Map<String, String> azurePaths = getAzurePaths(segment);
|
final Map<String, String> azurePaths = getAzurePaths(segment);
|
||||||
|
|
||||||
return AzureUtils.retryAzureOperation(
|
return AzureUtils.retryAzureOperation(
|
||||||
new Callable<DataSegment>()
|
() -> uploadDataSegment(segment, version, size, outFile, descFile, azurePaths, replaceExisting),
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public DataSegment call() throws Exception
|
|
||||||
{
|
|
||||||
return uploadDataSegment(segment, version, size, outFile, descFile, azurePaths, replaceExisting);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
config.getMaxTries()
|
config.getMaxTries()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,7 +33,6 @@ import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.net.URISyntaxException;
|
import java.net.URISyntaxException;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
public class AzureTaskLogs implements TaskLogs
|
public class AzureTaskLogs implements TaskLogs
|
||||||
{
|
{
|
||||||
|
@ -58,7 +57,7 @@ public class AzureTaskLogs implements TaskLogs
|
||||||
|
|
||||||
try {
|
try {
|
||||||
AzureUtils.retryAzureOperation(
|
AzureUtils.retryAzureOperation(
|
||||||
(Callable<Void>) () -> {
|
() -> {
|
||||||
azureStorage.uploadBlob(logFile, config.getContainer(), taskKey, true);
|
azureStorage.uploadBlob(logFile, config.getContainer(), taskKey, true);
|
||||||
return null;
|
return null;
|
||||||
},
|
},
|
||||||
|
|
|
@ -21,12 +21,11 @@ package io.druid.storage.azure;
|
||||||
|
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.microsoft.azure.storage.StorageException;
|
import com.microsoft.azure.storage.StorageException;
|
||||||
|
|
||||||
import io.druid.java.util.common.RetryUtils;
|
import io.druid.java.util.common.RetryUtils;
|
||||||
|
import io.druid.java.util.common.RetryUtils.Task;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.URISyntaxException;
|
import java.net.URISyntaxException;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
public class AzureUtils
|
public class AzureUtils
|
||||||
{
|
{
|
||||||
|
@ -53,7 +52,7 @@ public class AzureUtils
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
public static <T> T retryAzureOperation(Callable<T> f, int maxTries) throws Exception
|
public static <T> T retryAzureOperation(Task<T> f, int maxTries) throws Exception
|
||||||
{
|
{
|
||||||
return RetryUtils.retry(f, AZURE_RETRY, maxTries);
|
return RetryUtils.retry(f, AZURE_RETRY, maxTries);
|
||||||
}
|
}
|
||||||
|
|
|
@ -35,7 +35,6 @@ import java.io.File;
|
||||||
import java.io.FileOutputStream;
|
import java.io.FileOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cassandra Segment Puller
|
* Cassandra Segment Puller
|
||||||
|
@ -77,11 +76,7 @@ public class CassandraDataSegmentPuller extends CassandraStorage implements Data
|
||||||
final FileUtils.FileCopyResult localResult;
|
final FileUtils.FileCopyResult localResult;
|
||||||
try {
|
try {
|
||||||
localResult = RetryUtils.retry(
|
localResult = RetryUtils.retry(
|
||||||
new Callable<FileUtils.FileCopyResult>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public FileUtils.FileCopyResult call() throws Exception
|
|
||||||
{
|
|
||||||
try (OutputStream os = new FileOutputStream(tmpFile)) {
|
try (OutputStream os = new FileOutputStream(tmpFile)) {
|
||||||
ChunkedStorage
|
ChunkedStorage
|
||||||
.newReader(indexStorage, key, os)
|
.newReader(indexStorage, key, os)
|
||||||
|
@ -90,7 +85,6 @@ public class CassandraDataSegmentPuller extends CassandraStorage implements Data
|
||||||
.call();
|
.call();
|
||||||
}
|
}
|
||||||
return new FileUtils.FileCopyResult(tmpFile);
|
return new FileUtils.FileCopyResult(tmpFile);
|
||||||
}
|
|
||||||
},
|
},
|
||||||
Predicates.<Throwable>alwaysTrue(),
|
Predicates.<Throwable>alwaysTrue(),
|
||||||
10
|
10
|
||||||
|
|
|
@ -22,11 +22,13 @@ package io.druid.firehose.cloudfiles;
|
||||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
|
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
|
||||||
import io.druid.java.util.common.CompressionUtils;
|
import io.druid.java.util.common.CompressionUtils;
|
||||||
import io.druid.java.util.common.logger.Logger;
|
import io.druid.java.util.common.logger.Logger;
|
||||||
import io.druid.storage.cloudfiles.CloudFilesByteSource;
|
import io.druid.storage.cloudfiles.CloudFilesByteSource;
|
||||||
import io.druid.storage.cloudfiles.CloudFilesObjectApiProxy;
|
import io.druid.storage.cloudfiles.CloudFilesObjectApiProxy;
|
||||||
|
import io.druid.storage.cloudfiles.CloudFilesUtils;
|
||||||
import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
|
import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -72,6 +74,17 @@ public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFireho
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected InputStream openObjectStream(CloudFilesBlob object) throws IOException
|
protected InputStream openObjectStream(CloudFilesBlob object) throws IOException
|
||||||
|
{
|
||||||
|
return openObjectStream(object, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected InputStream openObjectStream(CloudFilesBlob object, long start) throws IOException
|
||||||
|
{
|
||||||
|
return createCloudFilesByteSource(object).openStream(start);
|
||||||
|
}
|
||||||
|
|
||||||
|
private CloudFilesByteSource createCloudFilesByteSource(CloudFilesBlob object)
|
||||||
{
|
{
|
||||||
final String region = object.getRegion();
|
final String region = object.getRegion();
|
||||||
final String container = object.getContainer();
|
final String container = object.getContainer();
|
||||||
|
@ -82,9 +95,7 @@ public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFireho
|
||||||
);
|
);
|
||||||
CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy(
|
CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy(
|
||||||
cloudFilesApi, region, container);
|
cloudFilesApi, region, container);
|
||||||
final CloudFilesByteSource byteSource = new CloudFilesByteSource(objectApi, path);
|
return new CloudFilesByteSource(objectApi, path);
|
||||||
|
|
||||||
return byteSource.openStream();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -125,4 +136,10 @@ public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFireho
|
||||||
getMaxFetchRetry()
|
getMaxFetchRetry()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Predicate<Throwable> getRetryCondition()
|
||||||
|
{
|
||||||
|
return CloudFilesUtils.CLOUDFILESRETRY;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -51,7 +51,12 @@ public class CloudFilesByteSource extends ByteSource
|
||||||
@Override
|
@Override
|
||||||
public InputStream openStream() throws IOException
|
public InputStream openStream() throws IOException
|
||||||
{
|
{
|
||||||
payload = (payload == null) ? objectApi.get(path).getPayload() : payload;
|
return openStream(0);
|
||||||
|
}
|
||||||
|
|
||||||
|
public InputStream openStream(long start) throws IOException
|
||||||
|
{
|
||||||
|
payload = (payload == null) ? objectApi.get(path, start).getPayload() : payload;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
return payload.openStream();
|
return payload.openStream();
|
||||||
|
|
|
@ -35,7 +35,6 @@ import java.io.IOException;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.nio.file.Files;
|
import java.nio.file.Files;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
public class CloudFilesDataSegmentPusher implements DataSegmentPusher
|
public class CloudFilesDataSegmentPusher implements DataSegmentPusher
|
||||||
{
|
{
|
||||||
|
@ -90,11 +89,7 @@ public class CloudFilesDataSegmentPusher implements DataSegmentPusher
|
||||||
|
|
||||||
log.info("Copying segment[%s] to CloudFiles at location[%s]", inSegment.getIdentifier(), segmentPath);
|
log.info("Copying segment[%s] to CloudFiles at location[%s]", inSegment.getIdentifier(), segmentPath);
|
||||||
return CloudFilesUtils.retryCloudFilesOperation(
|
return CloudFilesUtils.retryCloudFilesOperation(
|
||||||
new Callable<DataSegment>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public DataSegment call() throws Exception
|
|
||||||
{
|
|
||||||
CloudFilesObject segmentData = new CloudFilesObject(
|
CloudFilesObject segmentData = new CloudFilesObject(
|
||||||
segmentPath, outFile, objectApi.getRegion(),
|
segmentPath, outFile, objectApi.getRegion(),
|
||||||
objectApi.getContainer()
|
objectApi.getContainer()
|
||||||
|
@ -123,8 +118,8 @@ public class CloudFilesDataSegmentPusher implements DataSegmentPusher
|
||||||
.withBinaryVersion(SegmentUtils.getVersionFromDir(indexFilesDir));
|
.withBinaryVersion(SegmentUtils.getVersionFromDir(indexFilesDir));
|
||||||
|
|
||||||
return outSegment;
|
return outSegment;
|
||||||
}
|
},
|
||||||
}, this.config.getOperationMaxRetries()
|
this.config.getOperationMaxRetries()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package io.druid.storage.cloudfiles;
|
package io.druid.storage.cloudfiles;
|
||||||
|
|
||||||
|
import org.jclouds.http.options.GetOptions;
|
||||||
import org.jclouds.io.Payload;
|
import org.jclouds.io.Payload;
|
||||||
import org.jclouds.openstack.swift.v1.domain.SwiftObject;
|
import org.jclouds.openstack.swift.v1.domain.SwiftObject;
|
||||||
import org.jclouds.openstack.swift.v1.features.ObjectApi;
|
import org.jclouds.openstack.swift.v1.features.ObjectApi;
|
||||||
|
@ -52,9 +53,14 @@ public class CloudFilesObjectApiProxy
|
||||||
return objectApi.put(cloudFilesObject.getPath(), cloudFilesObject.getPayload());
|
return objectApi.put(cloudFilesObject.getPath(), cloudFilesObject.getPayload());
|
||||||
}
|
}
|
||||||
|
|
||||||
public CloudFilesObject get(String path)
|
public CloudFilesObject get(String path, long start)
|
||||||
{
|
{
|
||||||
SwiftObject swiftObject = objectApi.get(path);
|
final SwiftObject swiftObject;
|
||||||
|
if (start == 0) {
|
||||||
|
swiftObject = objectApi.get(path);
|
||||||
|
} else {
|
||||||
|
swiftObject = objectApi.get(path, new GetOptions().startAt(start));
|
||||||
|
}
|
||||||
Payload payload = swiftObject.getPayload();
|
Payload payload = swiftObject.getPayload();
|
||||||
return new CloudFilesObject(payload, this.region, this.container, path);
|
return new CloudFilesObject(payload, this.region, this.container, path);
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,11 +20,10 @@
|
||||||
package io.druid.storage.cloudfiles;
|
package io.druid.storage.cloudfiles;
|
||||||
|
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
|
|
||||||
import io.druid.java.util.common.RetryUtils;
|
import io.druid.java.util.common.RetryUtils;
|
||||||
|
import io.druid.java.util.common.RetryUtils.Task;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
|
@ -50,7 +49,7 @@ public class CloudFilesUtils
|
||||||
/**
|
/**
|
||||||
* Retries CloudFiles operations that fail due to io-related exceptions.
|
* Retries CloudFiles operations that fail due to io-related exceptions.
|
||||||
*/
|
*/
|
||||||
public static <T> T retryCloudFilesOperation(Callable<T> f, final int maxTries) throws Exception
|
public static <T> T retryCloudFilesOperation(Task<T> f, final int maxTries) throws Exception
|
||||||
{
|
{
|
||||||
return RetryUtils.retry(f, CLOUDFILESRETRY, maxTries);
|
return RetryUtils.retry(f, CLOUDFILESRETRY, maxTries);
|
||||||
}
|
}
|
||||||
|
|
|
@ -42,7 +42,7 @@ public class CloudFilesByteSourceTest extends EasyMockSupport
|
||||||
Payload payload = createMock(Payload.class);
|
Payload payload = createMock(Payload.class);
|
||||||
InputStream stream = createMock(InputStream.class);
|
InputStream stream = createMock(InputStream.class);
|
||||||
|
|
||||||
expect(objectApi.get(path)).andReturn(cloudFilesObject);
|
expect(objectApi.get(path, 0)).andReturn(cloudFilesObject);
|
||||||
expect(cloudFilesObject.getPayload()).andReturn(payload);
|
expect(cloudFilesObject.getPayload()).andReturn(payload);
|
||||||
expect(payload.openStream()).andReturn(stream);
|
expect(payload.openStream()).andReturn(stream);
|
||||||
payload.close();
|
payload.close();
|
||||||
|
@ -66,7 +66,7 @@ public class CloudFilesByteSourceTest extends EasyMockSupport
|
||||||
Payload payload = createMock(Payload.class);
|
Payload payload = createMock(Payload.class);
|
||||||
InputStream stream = createMock(InputStream.class);
|
InputStream stream = createMock(InputStream.class);
|
||||||
|
|
||||||
expect(objectApi.get(path)).andReturn(cloudFilesObject);
|
expect(objectApi.get(path, 0)).andReturn(cloudFilesObject);
|
||||||
expect(cloudFilesObject.getPayload()).andReturn(payload);
|
expect(cloudFilesObject.getPayload()).andReturn(payload);
|
||||||
expect(payload.openStream()).andThrow(new IOException()).andReturn(stream);
|
expect(payload.openStream()).andThrow(new IOException()).andReturn(stream);
|
||||||
payload.close();
|
payload.close();
|
||||||
|
|
|
@ -51,7 +51,7 @@ public class CloudFilesObjectApiProxyTest extends EasyMockSupport
|
||||||
replayAll();
|
replayAll();
|
||||||
|
|
||||||
CloudFilesObjectApiProxy cfoApiProxy = new CloudFilesObjectApiProxy(cloudFilesApi, region, container);
|
CloudFilesObjectApiProxy cfoApiProxy = new CloudFilesObjectApiProxy(cloudFilesApi, region, container);
|
||||||
CloudFilesObject cloudFilesObject = cfoApiProxy.get(path);
|
CloudFilesObject cloudFilesObject = cfoApiProxy.get(path, 0);
|
||||||
|
|
||||||
assertEquals(cloudFilesObject.getPayload(), payload);
|
assertEquals(cloudFilesObject.getPayload(), payload);
|
||||||
assertEquals(cloudFilesObject.getRegion(), region);
|
assertEquals(cloudFilesObject.getRegion(), region);
|
||||||
|
|
|
@ -22,10 +22,12 @@ package io.druid.firehose.google;
|
||||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
|
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
|
||||||
import io.druid.java.util.common.CompressionUtils;
|
import io.druid.java.util.common.CompressionUtils;
|
||||||
import io.druid.storage.google.GoogleByteSource;
|
import io.druid.storage.google.GoogleByteSource;
|
||||||
import io.druid.storage.google.GoogleStorage;
|
import io.druid.storage.google.GoogleStorage;
|
||||||
|
import io.druid.storage.google.GoogleUtils;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
|
@ -68,13 +70,24 @@ public class StaticGoogleBlobStoreFirehoseFactory extends PrefetchableTextFilesF
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected InputStream openObjectStream(GoogleBlob object) throws IOException
|
protected InputStream openObjectStream(GoogleBlob object) throws IOException
|
||||||
|
{
|
||||||
|
return openObjectStream(object, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected InputStream openObjectStream(GoogleBlob object, long start) throws IOException
|
||||||
|
{
|
||||||
|
return createGoogleByteSource(object).openStream(start);
|
||||||
|
}
|
||||||
|
|
||||||
|
private GoogleByteSource createGoogleByteSource(GoogleBlob object)
|
||||||
{
|
{
|
||||||
final String bucket = object.getBucket();
|
final String bucket = object.getBucket();
|
||||||
final String path = object.getPath().startsWith("/")
|
final String path = object.getPath().startsWith("/")
|
||||||
? object.getPath().substring(1)
|
? object.getPath().substring(1)
|
||||||
: object.getPath();
|
: object.getPath();
|
||||||
|
|
||||||
return new GoogleByteSource(storage, bucket, path).openStream();
|
return new GoogleByteSource(storage, bucket, path);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -115,5 +128,11 @@ public class StaticGoogleBlobStoreFirehoseFactory extends PrefetchableTextFilesF
|
||||||
getMaxFetchRetry()
|
getMaxFetchRetry()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Predicate<Throwable> getRetryCondition()
|
||||||
|
{
|
||||||
|
return GoogleUtils.GOOGLE_RETRY;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -42,4 +42,9 @@ public class GoogleByteSource extends ByteSource
|
||||||
{
|
{
|
||||||
return storage.get(bucket, path);
|
return storage.get(bucket, path);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public InputStream openStream(long start) throws IOException
|
||||||
|
{
|
||||||
|
return storage.get(bucket, path, start);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.storage.google;
|
||||||
|
|
||||||
import com.google.api.client.http.AbstractInputStreamContent;
|
import com.google.api.client.http.AbstractInputStreamContent;
|
||||||
import com.google.api.services.storage.Storage;
|
import com.google.api.services.storage.Storage;
|
||||||
|
import com.google.api.services.storage.Storage.Objects.Get;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
|
@ -44,9 +45,17 @@ public class GoogleStorage
|
||||||
|
|
||||||
public InputStream get(final String bucket, final String path) throws IOException
|
public InputStream get(final String bucket, final String path) throws IOException
|
||||||
{
|
{
|
||||||
Storage.Objects.Get getObject = storage.objects().get(bucket, path);
|
return get(bucket, path, 0);
|
||||||
getObject.getMediaHttpDownloader().setDirectDownloadEnabled(false);
|
}
|
||||||
return getObject.executeMediaAsInputStream();
|
|
||||||
|
public InputStream get(final String bucket, final String path, long start) throws IOException
|
||||||
|
{
|
||||||
|
final Get get = storage.objects().get(bucket, path);
|
||||||
|
if (start > 0) {
|
||||||
|
get.getMediaHttpDownloader().setBytesDownloaded(start);
|
||||||
|
}
|
||||||
|
get.getMediaHttpDownloader().setDirectDownloadEnabled(false);
|
||||||
|
return get.executeMediaAsInputStream();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void delete(final String bucket, final String path) throws IOException
|
public void delete(final String bucket, final String path) throws IOException
|
||||||
|
|
|
@ -48,7 +48,6 @@ import java.io.OutputStream;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
import java.io.Writer;
|
import java.io.Writer;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -183,11 +182,7 @@ public class HdfsDataSegmentPuller implements DataSegmentPuller, URIDataPuller
|
||||||
|
|
||||||
try {
|
try {
|
||||||
return RetryUtils.retry(
|
return RetryUtils.retry(
|
||||||
new Callable<FileUtils.FileCopyResult>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public FileUtils.FileCopyResult call() throws Exception
|
|
||||||
{
|
|
||||||
if (!fs.exists(path)) {
|
if (!fs.exists(path)) {
|
||||||
throw new SegmentLoadingException("No files found at [%s]", path.toString());
|
throw new SegmentLoadingException("No files found at [%s]", path.toString());
|
||||||
}
|
}
|
||||||
|
@ -215,8 +210,6 @@ public class HdfsDataSegmentPuller implements DataSegmentPuller, URIDataPuller
|
||||||
outDir.getAbsolutePath()
|
outDir.getAbsolutePath()
|
||||||
);
|
);
|
||||||
return result;
|
return result;
|
||||||
}
|
|
||||||
|
|
||||||
},
|
},
|
||||||
shouldRetryPredicate(),
|
shouldRetryPredicate(),
|
||||||
DEFAULT_RETRY_COUNT
|
DEFAULT_RETRY_COUNT
|
||||||
|
|
|
@ -21,10 +21,8 @@ package io.druid.storage.hdfs;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
|
||||||
import io.druid.data.SearchableVersionedDataFinder;
|
import io.druid.data.SearchableVersionedDataFinder;
|
||||||
import io.druid.java.util.common.RetryUtils;
|
import io.druid.java.util.common.RetryUtils;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
@ -34,7 +32,6 @@ import org.apache.hadoop.fs.PathFilter;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -89,17 +86,12 @@ public class HdfsFileTimestampVersionFinder extends HdfsDataSegmentPuller implem
|
||||||
final Path path = new Path(uri);
|
final Path path = new Path(uri);
|
||||||
try {
|
try {
|
||||||
return RetryUtils.retry(
|
return RetryUtils.retry(
|
||||||
new Callable<URI>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public URI call() throws Exception
|
|
||||||
{
|
|
||||||
final FileSystem fs = path.getFileSystem(config);
|
final FileSystem fs = path.getFileSystem(config);
|
||||||
if (!fs.exists(path)) {
|
if (!fs.exists(path)) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
return mostRecentInDir(fs.isDirectory(path) ? path : path.getParent(), pattern);
|
return mostRecentInDir(fs.isDirectory(path) ? path : path.getParent(), pattern);
|
||||||
}
|
|
||||||
},
|
},
|
||||||
shouldRetryPredicate(),
|
shouldRetryPredicate(),
|
||||||
DEFAULT_RETRY_COUNT
|
DEFAULT_RETRY_COUNT
|
||||||
|
|
|
@ -23,10 +23,12 @@ import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
|
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
|
||||||
import io.druid.java.util.common.CompressionUtils;
|
import io.druid.java.util.common.CompressionUtils;
|
||||||
import io.druid.java.util.common.IAE;
|
import io.druid.java.util.common.IAE;
|
||||||
import io.druid.java.util.common.logger.Logger;
|
import io.druid.java.util.common.logger.Logger;
|
||||||
|
import io.druid.storage.s3.S3Utils;
|
||||||
import org.jets3t.service.S3ServiceException;
|
import org.jets3t.service.S3ServiceException;
|
||||||
import org.jets3t.service.ServiceException;
|
import org.jets3t.service.ServiceException;
|
||||||
import org.jets3t.service.StorageObjectsChunk;
|
import org.jets3t.service.StorageObjectsChunk;
|
||||||
|
@ -188,6 +190,28 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected InputStream openObjectStream(S3Object object, long start) throws IOException
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
final S3Object result = s3Client.getObject(
|
||||||
|
object.getBucketName(),
|
||||||
|
object.getKey(),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
start,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
return result.getDataInputStream();
|
||||||
|
}
|
||||||
|
catch (ServiceException e) {
|
||||||
|
throw new IOException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected InputStream wrapObjectStream(S3Object object, InputStream stream) throws IOException
|
protected InputStream wrapObjectStream(S3Object object, InputStream stream) throws IOException
|
||||||
{
|
{
|
||||||
|
@ -228,4 +252,10 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
|
||||||
getMaxFetchRetry()
|
getMaxFetchRetry()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Predicate<Throwable> getRetryCondition()
|
||||||
|
{
|
||||||
|
return S3Utils.S3RETRY;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -41,7 +41,6 @@ import org.jets3t.service.model.S3Object;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
public class S3DataSegmentMover implements DataSegmentMover
|
public class S3DataSegmentMover implements DataSegmentMover
|
||||||
{
|
{
|
||||||
|
@ -118,7 +117,7 @@ public class S3DataSegmentMover implements DataSegmentMover
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
S3Utils.retryS3Operation(
|
S3Utils.retryS3Operation(
|
||||||
(Callable<Void>) () -> {
|
() -> {
|
||||||
final String copyMsg = StringUtils.format(
|
final String copyMsg = StringUtils.format(
|
||||||
"[s3://%s/%s] to [s3://%s/%s]",
|
"[s3://%s/%s] to [s3://%s/%s]",
|
||||||
s3Bucket,
|
s3Bucket,
|
||||||
|
@ -228,7 +227,7 @@ public class S3DataSegmentMover implements DataSegmentMover
|
||||||
private void deleteWithRetries(final String s3Bucket, final String s3Path) throws Exception
|
private void deleteWithRetries(final String s3Bucket, final String s3Path) throws Exception
|
||||||
{
|
{
|
||||||
RetryUtils.retry(
|
RetryUtils.retry(
|
||||||
(Callable<Void>) () -> {
|
() -> {
|
||||||
try {
|
try {
|
||||||
s3Client.deleteObject(s3Bucket, s3Path);
|
s3Client.deleteObject(s3Bucket, s3Path);
|
||||||
return null;
|
return null;
|
||||||
|
|
|
@ -52,7 +52,6 @@ import java.io.Reader;
|
||||||
import java.io.Writer;
|
import java.io.Writer;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A data segment puller that also hanldes URI data pulls.
|
* A data segment puller that also hanldes URI data pulls.
|
||||||
|
@ -310,14 +309,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller, URIDataPuller
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
return S3Utils.retryS3Operation(
|
return S3Utils.retryS3Operation(
|
||||||
new Callable<Boolean>()
|
() -> S3Utils.isObjectInBucket(s3Client, coords.bucket, coords.path)
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Boolean call() throws Exception
|
|
||||||
{
|
|
||||||
return S3Utils.isObjectInBucket(s3Client, coords.bucket, coords.path);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (S3ServiceException | IOException e) {
|
catch (S3ServiceException | IOException e) {
|
||||||
|
|
|
@ -41,7 +41,6 @@ import java.net.URI;
|
||||||
import java.nio.file.Files;
|
import java.nio.file.Files;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
public class S3DataSegmentPusher implements DataSegmentPusher
|
public class S3DataSegmentPusher implements DataSegmentPusher
|
||||||
{
|
{
|
||||||
|
@ -100,11 +99,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher
|
||||||
|
|
||||||
try {
|
try {
|
||||||
return S3Utils.retryS3Operation(
|
return S3Utils.retryS3Operation(
|
||||||
new Callable<DataSegment>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public DataSegment call() throws Exception
|
|
||||||
{
|
|
||||||
S3Object toPush = new S3Object(zipOutFile);
|
S3Object toPush = new S3Object(zipOutFile);
|
||||||
putObject(config.getBucket(), s3Path, toPush, replaceExisting);
|
putObject(config.getBucket(), s3Path, toPush, replaceExisting);
|
||||||
|
|
||||||
|
@ -133,7 +128,6 @@ public class S3DataSegmentPusher implements DataSegmentPusher
|
||||||
|
|
||||||
return outSegment;
|
return outSegment;
|
||||||
}
|
}
|
||||||
}
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (ServiceException e) {
|
catch (ServiceException e) {
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.io.ByteSource;
|
import com.google.common.io.ByteSource;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
|
||||||
import io.druid.java.util.common.IOE;
|
import io.druid.java.util.common.IOE;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
import io.druid.java.util.common.logger.Logger;
|
import io.druid.java.util.common.logger.Logger;
|
||||||
|
@ -36,7 +35,6 @@ import org.jets3t.service.model.StorageObject;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Provides task logs archived on S3.
|
* Provides task logs archived on S3.
|
||||||
|
@ -118,17 +116,12 @@ public class S3TaskLogs implements TaskLogs
|
||||||
|
|
||||||
try {
|
try {
|
||||||
S3Utils.retryS3Operation(
|
S3Utils.retryS3Operation(
|
||||||
new Callable<Void>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Void call() throws Exception
|
|
||||||
{
|
|
||||||
final StorageObject object = new StorageObject(logFile);
|
final StorageObject object = new StorageObject(logFile);
|
||||||
object.setKey(taskKey);
|
object.setKey(taskKey);
|
||||||
service.putObject(config.getS3Bucket(), object);
|
service.putObject(config.getS3Bucket(), object);
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
|
|
|
@ -21,17 +21,14 @@ package io.druid.storage.s3;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
|
||||||
import io.druid.data.SearchableVersionedDataFinder;
|
import io.druid.data.SearchableVersionedDataFinder;
|
||||||
import io.druid.java.util.common.RetryUtils;
|
import io.druid.java.util.common.RetryUtils;
|
||||||
|
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||||
import org.jets3t.service.model.S3Object;
|
import org.jets3t.service.model.S3Object;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -64,11 +61,7 @@ public class S3TimestampVersionedDataFinder extends S3DataSegmentPuller implemen
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
return RetryUtils.retry(
|
return RetryUtils.retry(
|
||||||
new Callable<URI>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public URI call() throws Exception
|
|
||||||
{
|
|
||||||
final S3Coords coords = new S3Coords(checkURI(uri));
|
final S3Coords coords = new S3Coords(checkURI(uri));
|
||||||
long mostRecent = Long.MIN_VALUE;
|
long mostRecent = Long.MIN_VALUE;
|
||||||
URI latest = null;
|
URI latest = null;
|
||||||
|
@ -92,7 +85,6 @@ public class S3TimestampVersionedDataFinder extends S3DataSegmentPuller implemen
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return latest;
|
return latest;
|
||||||
}
|
|
||||||
},
|
},
|
||||||
shouldRetryPredicate(),
|
shouldRetryPredicate(),
|
||||||
DEFAULT_RETRY_COUNT
|
DEFAULT_RETRY_COUNT
|
||||||
|
|
|
@ -22,8 +22,8 @@ package io.druid.storage.s3;
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
|
|
||||||
import io.druid.java.util.common.RetryUtils;
|
import io.druid.java.util.common.RetryUtils;
|
||||||
|
import io.druid.java.util.common.RetryUtils.Task;
|
||||||
import org.jets3t.service.ServiceException;
|
import org.jets3t.service.ServiceException;
|
||||||
import org.jets3t.service.StorageObjectsChunk;
|
import org.jets3t.service.StorageObjectsChunk;
|
||||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||||
|
@ -32,7 +32,6 @@ import org.jets3t.service.model.StorageObject;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
|
@ -83,7 +82,7 @@ public class S3Utils
|
||||||
* Retries S3 operations that fail due to io-related exceptions. Service-level exceptions (access denied, file not
|
* Retries S3 operations that fail due to io-related exceptions. Service-level exceptions (access denied, file not
|
||||||
* found, etc) are not retried.
|
* found, etc) are not retried.
|
||||||
*/
|
*/
|
||||||
public static <T> T retryS3Operation(Callable<T> f) throws Exception
|
public static <T> T retryS3Operation(Task<T> f) throws Exception
|
||||||
{
|
{
|
||||||
final int maxTries = 10;
|
final int maxTries = 10;
|
||||||
return RetryUtils.retry(f, S3RETRY, maxTries);
|
return RetryUtils.retry(f, S3RETRY, maxTries);
|
||||||
|
@ -147,15 +146,7 @@ public class S3Utils
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
return retryS3Operation(
|
return retryS3Operation(
|
||||||
new Callable<StorageObjectsChunk>()
|
() -> s3Client.listObjectsChunked(bucket, prefix, null, maxListingLength, priorLastKey)
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public StorageObjectsChunk call() throws Exception
|
|
||||||
{
|
|
||||||
return s3Client.listObjectsChunked(
|
|
||||||
bucket, prefix, null, maxListingLength, priorLastKey);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
|
|
|
@ -64,7 +64,6 @@ import java.net.URISyntaxException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
@ -160,18 +159,13 @@ public class JobHelper
|
||||||
if (jarFile.getName().endsWith(".jar")) {
|
if (jarFile.getName().endsWith(".jar")) {
|
||||||
try {
|
try {
|
||||||
RetryUtils.retry(
|
RetryUtils.retry(
|
||||||
new Callable<Boolean>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Boolean call() throws Exception
|
|
||||||
{
|
|
||||||
if (isSnapshot(jarFile)) {
|
if (isSnapshot(jarFile)) {
|
||||||
addSnapshotJarToClassPath(jarFile, intermediateClassPath, fs, job);
|
addSnapshotJarToClassPath(jarFile, intermediateClassPath, fs, job);
|
||||||
} else {
|
} else {
|
||||||
addJarToClassPath(jarFile, distributedClassPath, intermediateClassPath, fs, job);
|
addJarToClassPath(jarFile, distributedClassPath, intermediateClassPath, fs, job);
|
||||||
}
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
|
||||||
},
|
},
|
||||||
shouldRetryPredicate(),
|
shouldRetryPredicate(),
|
||||||
NUM_RETRIES
|
NUM_RETRIES
|
||||||
|
@ -607,11 +601,7 @@ public class JobHelper
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
return RetryUtils.retry(
|
return RetryUtils.retry(
|
||||||
new Callable<Boolean>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Boolean call() throws Exception
|
|
||||||
{
|
|
||||||
final boolean needRename;
|
final boolean needRename;
|
||||||
|
|
||||||
if (outputFS.exists(finalIndexZipFilePath)) {
|
if (outputFS.exists(finalIndexZipFilePath)) {
|
||||||
|
@ -651,7 +641,6 @@ public class JobHelper
|
||||||
} else {
|
} else {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
|
||||||
},
|
},
|
||||||
FileUtils.IS_EXCEPTION,
|
FileUtils.IS_EXCEPTION,
|
||||||
NUM_RETRIES
|
NUM_RETRIES
|
||||||
|
@ -821,14 +810,7 @@ public class JobHelper
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
return RetryUtils.retry(
|
return RetryUtils.retry(
|
||||||
new Callable<Boolean>()
|
() -> fs.delete(path, recursive),
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Boolean call() throws Exception
|
|
||||||
{
|
|
||||||
return fs.delete(path, recursive);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
shouldRetryPredicate(),
|
shouldRetryPredicate(),
|
||||||
NUM_RETRIES
|
NUM_RETRIES
|
||||||
);
|
);
|
||||||
|
|
|
@ -80,11 +80,7 @@ public class OverlordResourceTestClient
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
return RetryUtils.retry(
|
return RetryUtils.retry(
|
||||||
new Callable<String>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String call() throws Exception
|
|
||||||
{
|
|
||||||
StatusResponseHolder response = httpClient.go(
|
StatusResponseHolder response = httpClient.go(
|
||||||
new Request(HttpMethod.POST, new URL(getIndexerURL() + "task"))
|
new Request(HttpMethod.POST, new URL(getIndexerURL() + "task"))
|
||||||
.setContent(
|
.setContent(
|
||||||
|
@ -106,7 +102,6 @@ public class OverlordResourceTestClient
|
||||||
String taskID = responseData.get("task");
|
String taskID = responseData.get("task");
|
||||||
LOG.info("Submitted task with TaskID[%s]", taskID);
|
LOG.info("Submitted task with TaskID[%s]", taskID);
|
||||||
return taskID;
|
return taskID;
|
||||||
}
|
|
||||||
},
|
},
|
||||||
Predicates.<Throwable>alwaysTrue(),
|
Predicates.<Throwable>alwaysTrue(),
|
||||||
5
|
5
|
||||||
|
|
|
@ -36,7 +36,6 @@ import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.util.Enumeration;
|
import java.util.Enumeration;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import java.util.zip.GZIPInputStream;
|
import java.util.zip.GZIPInputStream;
|
||||||
import java.util.zip.GZIPOutputStream;
|
import java.util.zip.GZIPOutputStream;
|
||||||
import java.util.zip.ZipEntry;
|
import java.util.zip.ZipEntry;
|
||||||
|
@ -156,14 +155,7 @@ public class CompressionUtils
|
||||||
if (!cacheLocally) {
|
if (!cacheLocally) {
|
||||||
try {
|
try {
|
||||||
return RetryUtils.retry(
|
return RetryUtils.retry(
|
||||||
new Callable<FileUtils.FileCopyResult>()
|
() -> unzip(byteSource.openStream(), outDir),
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public FileUtils.FileCopyResult call() throws Exception
|
|
||||||
{
|
|
||||||
return unzip(byteSource.openStream(), outDir);
|
|
||||||
}
|
|
||||||
},
|
|
||||||
shouldRetry,
|
shouldRetry,
|
||||||
DEFAULT_RETRY_COUNT
|
DEFAULT_RETRY_COUNT
|
||||||
);
|
);
|
||||||
|
|
|
@ -24,12 +24,31 @@ import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import io.druid.java.util.common.logger.Logger;
|
import io.druid.java.util.common.logger.Logger;
|
||||||
|
|
||||||
import java.util.concurrent.Callable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.concurrent.ThreadLocalRandom;
|
import java.util.concurrent.ThreadLocalRandom;
|
||||||
|
|
||||||
public class RetryUtils
|
public class RetryUtils
|
||||||
{
|
{
|
||||||
public static final Logger log = new Logger(RetryUtils.class);
|
public static final Logger log = new Logger(RetryUtils.class);
|
||||||
|
public static final long MAX_SLEEP_MILLIS = 60000;
|
||||||
|
public static final long BASE_SLEEP_MILLIS = 1000;
|
||||||
|
|
||||||
|
public interface Task<T>
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* This method is tried up to maxTries times unless it succeeds.
|
||||||
|
*/
|
||||||
|
T perform() throws Exception;
|
||||||
|
}
|
||||||
|
|
||||||
|
public interface CleanupAfterFailure
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* This is called once {@link Task#perform()} fails. Retrying is stopped once this method throws an exception,
|
||||||
|
* so errors inside this method should be ignored if you don't want to stop retrying.
|
||||||
|
*/
|
||||||
|
void cleanup() throws Exception;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Retry an operation using fuzzy exponentially increasing backoff. The wait time after the nth failed attempt is
|
* Retry an operation using fuzzy exponentially increasing backoff. The wait time after the nth failed attempt is
|
||||||
|
@ -49,22 +68,29 @@ public class RetryUtils
|
||||||
* @throws Exception if maxTries is exhausted, or shouldRetry returns false
|
* @throws Exception if maxTries is exhausted, or shouldRetry returns false
|
||||||
*/
|
*/
|
||||||
public static <T> T retry(
|
public static <T> T retry(
|
||||||
final Callable<T> f,
|
final Task<T> f,
|
||||||
Predicate<Throwable> shouldRetry,
|
final Predicate<Throwable> shouldRetry,
|
||||||
final int quietTries,
|
final int quietTries,
|
||||||
final int maxTries
|
final int maxTries,
|
||||||
|
@Nullable final CleanupAfterFailure cleanupAfterFailure,
|
||||||
|
@Nullable final String messageOnRetry
|
||||||
) throws Exception
|
) throws Exception
|
||||||
{
|
{
|
||||||
Preconditions.checkArgument(maxTries > 0, "maxTries > 0");
|
Preconditions.checkArgument(maxTries > 0, "maxTries > 0");
|
||||||
|
Preconditions.checkArgument(quietTries >= 0, "quietTries >= 0");
|
||||||
int nTry = 0;
|
int nTry = 0;
|
||||||
|
final int maxRetries = maxTries - 1;
|
||||||
while (true) {
|
while (true) {
|
||||||
try {
|
try {
|
||||||
nTry++;
|
nTry++;
|
||||||
return f.call();
|
return f.perform();
|
||||||
}
|
}
|
||||||
catch (Throwable e) {
|
catch (Throwable e) {
|
||||||
|
if (cleanupAfterFailure != null) {
|
||||||
|
cleanupAfterFailure.cleanup();
|
||||||
|
}
|
||||||
if (nTry < maxTries && shouldRetry.apply(e)) {
|
if (nTry < maxTries && shouldRetry.apply(e)) {
|
||||||
awaitNextRetry(e, nTry, nTry <= quietTries);
|
awaitNextRetry(e, messageOnRetry, nTry, maxRetries, nTry <= quietTries);
|
||||||
} else {
|
} else {
|
||||||
Throwables.propagateIfInstanceOf(e, Exception.class);
|
Throwables.propagateIfInstanceOf(e, Exception.class);
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
|
@ -73,23 +99,69 @@ public class RetryUtils
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
public static <T> T retry(final Task<T> f, Predicate<Throwable> shouldRetry, final int maxTries) throws Exception
|
||||||
* Same as {@link #retry(Callable, Predicate, int, int)} with quietTries = 0.
|
|
||||||
*/
|
|
||||||
public static <T> T retry(final Callable<T> f, Predicate<Throwable> shouldRetry, final int maxTries) throws Exception
|
|
||||||
{
|
{
|
||||||
return retry(f, shouldRetry, 0, maxTries);
|
return retry(f, shouldRetry, 0, maxTries);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void awaitNextRetry(final Throwable e, final int nTry, final boolean quiet) throws InterruptedException
|
public static <T> T retry(
|
||||||
|
final Task<T> f,
|
||||||
|
final Predicate<Throwable> shouldRetry,
|
||||||
|
final int quietTries,
|
||||||
|
final int maxTries
|
||||||
|
) throws Exception
|
||||||
{
|
{
|
||||||
|
return retry(f, shouldRetry, quietTries, maxTries, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static <T> T retry(
|
||||||
|
final Task<T> f,
|
||||||
|
final Predicate<Throwable> shouldRetry,
|
||||||
|
final int maxTries,
|
||||||
|
final String messageOnRetry
|
||||||
|
) throws Exception
|
||||||
|
{
|
||||||
|
return retry(f, shouldRetry, 0, maxTries, null, messageOnRetry);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static <T> T retry(
|
||||||
|
final Task<T> f,
|
||||||
|
final Predicate<Throwable> shouldRetry,
|
||||||
|
final CleanupAfterFailure onEachFailure,
|
||||||
|
final int maxTries,
|
||||||
|
final String messageOnRetry
|
||||||
|
) throws Exception
|
||||||
|
{
|
||||||
|
return retry(f, shouldRetry, 0, maxTries, onEachFailure, messageOnRetry);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void awaitNextRetry(
|
||||||
|
final Throwable e,
|
||||||
|
@Nullable final String messageOnRetry,
|
||||||
|
final int nTry,
|
||||||
|
final int maxRetries,
|
||||||
|
final boolean quiet
|
||||||
|
) throws InterruptedException
|
||||||
|
{
|
||||||
final long sleepMillis = nextRetrySleepMillis(nTry);
|
final long sleepMillis = nextRetrySleepMillis(nTry);
|
||||||
|
final String fullMessage;
|
||||||
|
|
||||||
|
if (messageOnRetry == null) {
|
||||||
|
fullMessage = StringUtils.format("Retrying (%d of %d) in %,dms.", nTry, maxRetries, sleepMillis);
|
||||||
|
} else {
|
||||||
|
fullMessage = StringUtils.format(
|
||||||
|
"%s, retrying (%d of %d) in %,dms.",
|
||||||
|
messageOnRetry,
|
||||||
|
nTry,
|
||||||
|
maxRetries,
|
||||||
|
sleepMillis
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
if (quiet) {
|
if (quiet) {
|
||||||
log.debug(e, "Failed on try %d, retrying in %,dms.", nTry, sleepMillis);
|
log.debug(e, fullMessage);
|
||||||
} else {
|
} else {
|
||||||
log.warn(e, "Failed on try %d, retrying in %,dms.", nTry, sleepMillis);
|
log.warn(e, fullMessage);
|
||||||
}
|
}
|
||||||
|
|
||||||
Thread.sleep(sleepMillis);
|
Thread.sleep(sleepMillis);
|
||||||
|
@ -97,10 +169,8 @@ public class RetryUtils
|
||||||
|
|
||||||
public static long nextRetrySleepMillis(final int nTry)
|
public static long nextRetrySleepMillis(final int nTry)
|
||||||
{
|
{
|
||||||
final long baseSleepMillis = 1000;
|
|
||||||
final long maxSleepMillis = 60000;
|
|
||||||
final double fuzzyMultiplier = Math.min(Math.max(1 + 0.2 * ThreadLocalRandom.current().nextGaussian(), 0), 2);
|
final double fuzzyMultiplier = Math.min(Math.max(1 + 0.2 * ThreadLocalRandom.current().nextGaussian(), 0), 2);
|
||||||
final long sleepMillis = (long) (Math.min(maxSleepMillis, baseSleepMillis * Math.pow(2, nTry - 1))
|
final long sleepMillis = (long) (Math.min(MAX_SLEEP_MILLIS, BASE_SLEEP_MILLIS * Math.pow(2, nTry - 1))
|
||||||
* fuzzyMultiplier);
|
* fuzzyMultiplier);
|
||||||
return sleepMillis;
|
return sleepMillis;
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,7 +28,6 @@ import com.google.common.io.ByteStreams;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -76,11 +75,7 @@ public class StreamUtils
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
return RetryUtils.retry(
|
return RetryUtils.retry(
|
||||||
new Callable<Long>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Long call() throws Exception
|
|
||||||
{
|
|
||||||
try (InputStream inputStream = byteSource.openStream()) {
|
try (InputStream inputStream = byteSource.openStream()) {
|
||||||
try (OutputStream outputStream = byteSink.openStream()) {
|
try (OutputStream outputStream = byteSink.openStream()) {
|
||||||
final long retval = ByteStreams.copy(inputStream, outputStream);
|
final long retval = ByteStreams.copy(inputStream, outputStream);
|
||||||
|
@ -89,7 +84,6 @@ public class StreamUtils
|
||||||
return retval;
|
return retval;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
},
|
},
|
||||||
shouldRetry,
|
shouldRetry,
|
||||||
maxAttempts
|
maxAttempts
|
||||||
|
|
|
@ -24,7 +24,6 @@ import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
public class RetryUtilsTest
|
public class RetryUtilsTest
|
||||||
|
@ -43,14 +42,9 @@ public class RetryUtilsTest
|
||||||
{
|
{
|
||||||
final AtomicInteger count = new AtomicInteger();
|
final AtomicInteger count = new AtomicInteger();
|
||||||
final String result = RetryUtils.retry(
|
final String result = RetryUtils.retry(
|
||||||
new Callable<String>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String call() throws Exception
|
|
||||||
{
|
|
||||||
count.incrementAndGet();
|
count.incrementAndGet();
|
||||||
return "hey";
|
return "hey";
|
||||||
}
|
|
||||||
},
|
},
|
||||||
isTransient,
|
isTransient,
|
||||||
2
|
2
|
||||||
|
@ -66,14 +60,9 @@ public class RetryUtilsTest
|
||||||
boolean threwExpectedException = false;
|
boolean threwExpectedException = false;
|
||||||
try {
|
try {
|
||||||
RetryUtils.retry(
|
RetryUtils.retry(
|
||||||
new Callable<String>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String call() throws Exception
|
|
||||||
{
|
|
||||||
count.incrementAndGet();
|
count.incrementAndGet();
|
||||||
throw new IOException("what");
|
throw new IOException("what");
|
||||||
}
|
|
||||||
},
|
},
|
||||||
isTransient,
|
isTransient,
|
||||||
2
|
2
|
||||||
|
@ -91,17 +80,12 @@ public class RetryUtilsTest
|
||||||
{
|
{
|
||||||
final AtomicInteger count = new AtomicInteger();
|
final AtomicInteger count = new AtomicInteger();
|
||||||
final String result = RetryUtils.retry(
|
final String result = RetryUtils.retry(
|
||||||
new Callable<String>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String call() throws Exception
|
|
||||||
{
|
|
||||||
if (count.incrementAndGet() >= 2) {
|
if (count.incrementAndGet() >= 2) {
|
||||||
return "hey";
|
return "hey";
|
||||||
} else {
|
} else {
|
||||||
throw new IOException("what");
|
throw new IOException("what");
|
||||||
}
|
}
|
||||||
}
|
|
||||||
},
|
},
|
||||||
isTransient,
|
isTransient,
|
||||||
3
|
3
|
||||||
|
@ -117,17 +101,12 @@ public class RetryUtilsTest
|
||||||
boolean threwExpectedException = false;
|
boolean threwExpectedException = false;
|
||||||
try {
|
try {
|
||||||
RetryUtils.retry(
|
RetryUtils.retry(
|
||||||
new Callable<String>()
|
() -> {
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String call() throws Exception
|
|
||||||
{
|
|
||||||
if (count.incrementAndGet() >= 2) {
|
if (count.incrementAndGet() >= 2) {
|
||||||
return "hey";
|
return "hey";
|
||||||
} else {
|
} else {
|
||||||
throw new IOException("uhh");
|
throw new IOException("uhh");
|
||||||
}
|
}
|
||||||
}
|
|
||||||
},
|
},
|
||||||
isTransient,
|
isTransient,
|
||||||
3
|
3
|
||||||
|
@ -139,5 +118,4 @@ public class RetryUtilsTest
|
||||||
Assert.assertTrue("threw expected exception", threwExpectedException);
|
Assert.assertTrue("threw expected exception", threwExpectedException);
|
||||||
Assert.assertEquals("count", 1, count.get());
|
Assert.assertEquals("count", 1, count.get());
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -47,7 +47,6 @@ import java.sql.SQLTransientException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
|
|
||||||
public abstract class SQLMetadataConnector implements MetadataStorageConnector
|
public abstract class SQLMetadataConnector implements MetadataStorageConnector
|
||||||
{
|
{
|
||||||
|
@ -127,16 +126,8 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector
|
||||||
final Predicate<Throwable> myShouldRetry
|
final Predicate<Throwable> myShouldRetry
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final Callable<T> call = new Callable<T>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public T call() throws Exception
|
|
||||||
{
|
|
||||||
return getDBI().withHandle(callback);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
try {
|
try {
|
||||||
return RetryUtils.retry(call, myShouldRetry, DEFAULT_MAX_TRIES);
|
return RetryUtils.retry(() -> getDBI().withHandle(callback), myShouldRetry, DEFAULT_MAX_TRIES);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
|
@ -150,16 +141,8 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector
|
||||||
|
|
||||||
public <T> T retryTransaction(final TransactionCallback<T> callback, final int quietTries, final int maxTries)
|
public <T> T retryTransaction(final TransactionCallback<T> callback, final int quietTries, final int maxTries)
|
||||||
{
|
{
|
||||||
final Callable<T> call = new Callable<T>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public T call() throws Exception
|
|
||||||
{
|
|
||||||
return getDBI().inTransaction(callback);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
try {
|
try {
|
||||||
return RetryUtils.retry(call, shouldRetry, quietTries, maxTries);
|
return RetryUtils.retry(() -> getDBI().inTransaction(callback), shouldRetry, quietTries, maxTries);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package io.druid.segment.loading;
|
package io.druid.segment.loading;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
|
|
||||||
import io.druid.data.SearchableVersionedDataFinder;
|
import io.druid.data.SearchableVersionedDataFinder;
|
||||||
import io.druid.java.util.common.RetryUtils;
|
import io.druid.java.util.common.RetryUtils;
|
||||||
|
|
||||||
|
@ -31,7 +30,6 @@ import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -81,17 +79,10 @@ public class LocalFileTimestampVersionFinder extends LocalDataSegmentPuller
|
||||||
final File file = new File(uri);
|
final File file = new File(uri);
|
||||||
try {
|
try {
|
||||||
return RetryUtils.retry(
|
return RetryUtils.retry(
|
||||||
new Callable<URI>()
|
() -> mostRecentInDir(
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public URI call() throws Exception
|
|
||||||
{
|
|
||||||
return mostRecentInDir(
|
|
||||||
file.isDirectory() ? file.toPath() : file.getParentFile().toPath(),
|
file.isDirectory() ? file.toPath() : file.getParentFile().toPath(),
|
||||||
pattern
|
pattern
|
||||||
);
|
),
|
||||||
}
|
|
||||||
},
|
|
||||||
shouldRetryPredicate(),
|
shouldRetryPredicate(),
|
||||||
DEFAULT_RETRY_COUNT
|
DEFAULT_RETRY_COUNT
|
||||||
);
|
);
|
||||||
|
|
|
@ -21,19 +21,27 @@ package io.druid.segment.realtime.firehose;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
|
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
|
||||||
import io.druid.java.util.common.CompressionUtils;
|
import io.druid.java.util.common.CompressionUtils;
|
||||||
|
import io.druid.java.util.common.StringUtils;
|
||||||
|
import io.druid.java.util.common.logger.Logger;
|
||||||
|
import org.apache.http.HttpHeaders;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
|
import java.net.URLConnection;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
|
||||||
public class HttpFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<URI>
|
public class HttpFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<URI>
|
||||||
{
|
{
|
||||||
|
private static final Logger log = new Logger(HttpFirehoseFactory.class);
|
||||||
private final List<URI> uris;
|
private final List<URI> uris;
|
||||||
|
private final boolean supportContentRange;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public HttpFirehoseFactory(
|
public HttpFirehoseFactory(
|
||||||
|
@ -43,10 +51,15 @@ public class HttpFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<UR
|
||||||
@JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
|
@JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
|
||||||
@JsonProperty("fetchTimeout") Long fetchTimeout,
|
@JsonProperty("fetchTimeout") Long fetchTimeout,
|
||||||
@JsonProperty("maxFetchRetry") Integer maxFetchRetry
|
@JsonProperty("maxFetchRetry") Integer maxFetchRetry
|
||||||
)
|
) throws IOException
|
||||||
{
|
{
|
||||||
super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
|
super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
|
||||||
this.uris = uris;
|
this.uris = uris;
|
||||||
|
|
||||||
|
Preconditions.checkArgument(uris.size() > 0, "Empty URIs");
|
||||||
|
final URLConnection connection = uris.get(0).toURL().openConnection();
|
||||||
|
final String acceptRanges = connection.getHeaderField(HttpHeaders.ACCEPT_RANGES);
|
||||||
|
this.supportContentRange = acceptRanges != null && acceptRanges.equalsIgnoreCase("bytes");
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -67,6 +80,28 @@ public class HttpFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<UR
|
||||||
return object.toURL().openConnection().getInputStream();
|
return object.toURL().openConnection().getInputStream();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected InputStream openObjectStream(URI object, long start) throws IOException
|
||||||
|
{
|
||||||
|
if (supportContentRange) {
|
||||||
|
final URLConnection connection = object.toURL().openConnection();
|
||||||
|
// Set header for range request.
|
||||||
|
// Since we need to set only the start offset, the header is "bytes=<range-start>-".
|
||||||
|
// See https://tools.ietf.org/html/rfc7233#section-2.1
|
||||||
|
connection.addRequestProperty(HttpHeaders.RANGE, StringUtils.format("bytes=%d-", start));
|
||||||
|
return connection.getInputStream();
|
||||||
|
} else {
|
||||||
|
log.warn(
|
||||||
|
"Since the input source doesn't support range requests, the object input stream is opened from the start and "
|
||||||
|
+ "then skipped. This may make the ingestion speed slower. Consider enabling prefetch if you see this message"
|
||||||
|
+ " a lot."
|
||||||
|
);
|
||||||
|
final InputStream in = openObjectStream(object);
|
||||||
|
in.skip(start);
|
||||||
|
return in;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException
|
protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException
|
||||||
{
|
{
|
||||||
|
@ -105,4 +140,10 @@ public class HttpFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<UR
|
||||||
getMaxFetchRetry()
|
getMaxFetchRetry()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Predicate<Throwable> getRetryCondition()
|
||||||
|
{
|
||||||
|
return e -> e instanceof IOException;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue