Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HADOOP-19348. Integrate analytics accelerator into S3A. #7334

Draft
wants to merge 5 commits into
base: trunk
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions hadoop-tools/hadoop-aws/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -472,6 +472,12 @@
<artifactId>amazon-s3-encryption-client-java</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>software.amazon.s3.analyticsaccelerator</groupId>
<artifactId>analyticsaccelerator-s3</artifactId>
<version>0.0.2</version>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.assertj</groupId>
<artifactId>assertj-core</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1827,4 +1827,12 @@ private Constants() {
* Value: {@value}.
*/
public static final String S3A_IO_RATE_LIMIT = "fs.s3a.io.rate.limit";


/**
* Prefix to configure Analytics Accelerator Library.
*/
public static final String ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX =
"fs.s3a.analytics.accelerator";

}
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
import java.util.concurrent.atomic.AtomicBoolean;
import javax.annotation.Nullable;

import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType;
import software.amazon.awssdk.core.exception.SdkException;
import software.amazon.awssdk.services.s3.S3Client;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
Expand Down Expand Up @@ -643,8 +644,6 @@ public void initialize(URI name, Configuration originalConf)
dirOperationsPurgeUploads = conf.getBoolean(DIRECTORY_OPERATIONS_PURGE_UPLOADS,
s3ExpressStore);

this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED,
DEFAULT_MULTIPART_UPLOAD_ENABLED);
// multipart copy and upload are the same; this just makes it explicit
this.isMultipartCopyEnabled = isMultipartUploadEnabled;

Expand Down Expand Up @@ -1847,6 +1846,7 @@ private FSDataInputStream executeOpen(
(requirements.vectorSupported()
? requirements.vectoredIOContext().getVectoredActiveRangeReads()
: 0);

// create an executor which is a subset of the
// bounded thread pool.
final SemaphoredDelegatingExecutor pool = new SemaphoredDelegatingExecutor(
Expand All @@ -1863,6 +1863,7 @@ private FSDataInputStream executeOpen(
.withContext(readContext.build())
.withObjectAttributes(createObjectAttributes(path, fileStatus))
.withStreamStatistics(inputStreamStats);

return new FSDataInputStream(getStore().readObject(parameters));
}

Expand Down Expand Up @@ -4160,7 +4161,7 @@ PutObjectResponse executePut(
throws IOException {
String key = putObjectRequest.key();
ProgressableProgressListener listener =
new ProgressableProgressListener(store, key, progress);
new ProgressableProgressListener(getStore(), key, progress);
UploadInfo info = putObject(putObjectRequest, file, listener);
PutObjectResponse result = getStore().waitForUploadCompletion(key, info).response();
listener.uploadCompleted(info.getFileUpload());
Expand Down Expand Up @@ -4257,7 +4258,9 @@ public void close() throws IOException {
protected synchronized void stopAllServices() {
try {
trackDuration(getDurationTrackerFactory(), FILESYSTEM_CLOSE.getSymbol(), () -> {

closeAutocloseables(LOG, getStore());

store = null;
s3Client = null;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,8 +91,7 @@
import org.apache.hadoop.util.functional.Tuples;

import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.s3a.Constants.BUFFER_DIR;
import static org.apache.hadoop.fs.s3a.Constants.HADOOP_TMP_DIR;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3AUtils.extractException;
import static org.apache.hadoop.fs.s3a.S3AUtils.getPutRequestLength;
import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException;
Expand Down Expand Up @@ -948,7 +947,7 @@
* All stream factory initialization required after {@code Service.init()},
* after all other services have themselves been initialized.
*/
private void finishStreamFactoryInit() {
private void finishStreamFactoryInit() throws Exception {
// must be on be invoked during service initialization
Preconditions.checkState(isInState(STATE.INITED),
"Store is in wrong state: %s", getServiceState());
Expand All @@ -974,7 +973,7 @@
/**
* This operation is not implemented, as
* is this class which invokes it on the actual factory.
* @param factoryBindingParameters@throws UnsupportedOperationException always

Check failure on line 976 in hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java

View check run for this annotation

ASF Cloudbees Jenkins ci-hadoop / Apache Yetus

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java#L976

javadoc: warning: no description for @param

Check failure on line 976 in hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java

View check run for this annotation

ASF Cloudbees Jenkins ci-hadoop / Apache Yetus

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java#L976

javadoc: warning: no description for @param
*/
@Override /* ObjectInputStreamFactory */
public void bind(final FactoryBindingParameters factoryBindingParameters) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@

package org.apache.hadoop.fs.s3a.impl.streams;

import java.io.IOException;

import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.statistics.StreamStatisticNames;
import org.apache.hadoop.service.AbstractService;
Expand Down Expand Up @@ -54,7 +56,7 @@ protected AbstractObjectInputStreamFactory(final String name) {
* @param factoryBindingParameters parameters for the factory binding
*/
@Override
public void bind(final FactoryBindingParameters factoryBindingParameters) {
public void bind(final FactoryBindingParameters factoryBindingParameters) throws IOException {
// must be on be invoked during service initialization
Preconditions.checkState(isInState(STATE.INITED),
"Input Stream factory %s is in wrong state: %s",
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,200 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.hadoop.fs.s3a.impl.streams;

import java.io.EOFException;
import java.io.IOException;

import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import software.amazon.s3.analyticsaccelerator.S3SeekableInputStream;
import software.amazon.s3.analyticsaccelerator.util.S3URI;

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

java doc

public class AnalyticsStream extends ObjectInputStream implements StreamCapabilities {

private S3SeekableInputStream inputStream;
private long lastReadCurrentPos = 0;
private volatile boolean closed;

public static final Logger LOG = LoggerFactory.getLogger(AnalyticsStream.class);

public AnalyticsStream(final ObjectReadParameters parameters, final S3SeekableInputStreamFactory s3SeekableInputStreamFactory) {
super(InputStreamType.Analytics, parameters);
S3ObjectAttributes s3Attributes = parameters.getObjectAttributes();
this.inputStream = s3SeekableInputStreamFactory.createStream(S3URI.of(s3Attributes.getBucket(), s3Attributes.getKey()));
}

/**
* Indicates whether the given {@code capability} is supported by this stream.
*
* @param capability the capability to check.
* @return true if the given {@code capability} is supported by this stream, false otherwise.
*/
@Override
public boolean hasCapability(String capability) {
return false;
}

@Override
public int read() throws IOException {
throwIfClosed();
int bytesRead;
try {
bytesRead = inputStream.read();
} catch (IOException ioe) {
onReadFailure(ioe);
throw ioe;
}
return bytesRead;
}

@Override
public void seek(long pos) throws IOException {
throwIfClosed();
if (pos < 0) {
throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK
+ " " + pos);
}
inputStream.seek(pos);
}


@Override
public synchronized long getPos() {
if (!closed) {
lastReadCurrentPos = inputStream.getPos();
}
return lastReadCurrentPos;
}


/**
* Reads the last n bytes from the stream into a byte buffer. Blocks until end of stream is
* reached. Leaves the position of the stream unaltered.
*
* @param buf buffer to read data into
* @param off start position in buffer at which data is written
* @param len the number of bytes to read; the n-th byte should be the last byte of the stream.
* @return the total number of bytes read into the buffer
* @throws IOException if an I/O error occurs
*/
public int readTail(byte[] buf, int off, int len) throws IOException {
throwIfClosed();
int bytesRead;
try {
bytesRead = inputStream.readTail(buf, off, len);
} catch (IOException ioe) {
onReadFailure(ioe);
throw ioe;
}
return bytesRead;
}

@Override
public int read(byte[] buf, int off, int len) throws IOException {
throwIfClosed();
int bytesRead;
try {
bytesRead = inputStream.read(buf, off, len);
} catch (IOException ioe) {
onReadFailure(ioe);
throw ioe;
}
return bytesRead;
}


@Override
public boolean seekToNewSource(long l) throws IOException {
return false;
}

@Override
public int available() throws IOException {
throwIfClosed();
return super.available();
}

@Override
protected boolean isStreamOpen() {
return !isClosed();
}

protected boolean isClosed() {
return inputStream == null;
}

@Override
protected void abortInFinalizer() {
try {
close();
} catch (IOException ignored) {

}
}

@Override
public synchronized void close() throws IOException {
if(!closed) {
closed = true;
try {
inputStream.close();
inputStream = null;
super.close();
} catch (IOException ioe) {
LOG.debug("Failure closing stream {}: ", getKey());
throw ioe;
}
}
}

/**
* Close the stream on read failure.
* No attempt to recover from failure
*
* @param ioe exception caught.
*/
@Retries.OnceTranslated
private void onReadFailure(IOException ioe) throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Got exception while trying to read from stream {}, " +
"not trying to recover:",
getKey(), ioe);
} else {
LOG.info("Got exception while trying to read from stream {}, " +
"not trying to recover:",
getKey(), ioe);
}
this.close();
}


protected void throwIfClosed() throws IOException {
if (closed) {
throw new IOException(getKey() + ": " + FSExceptionMessages.STREAM_IS_CLOSED);
}
}
}
Loading
Loading