-
Notifications
You must be signed in to change notification settings - Fork 8.9k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Integrate Analytics Accelerator Library for Amazon S3
This commits is the initial integration of Analytics Accelerator Library for Amazon S3 to S3A. It performs integration by introducing a new S3ASeekableStream and modifying S3AFileSystem. Use of the Analytics Accelerator Library is controlled by a configration and it is off by default.
- Loading branch information
Showing
5 changed files
with
222 additions
and
1 deletion.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
68 changes: 68 additions & 0 deletions
68
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,68 @@ | ||
package org.apache.hadoop.fs.s3a; | ||
|
||
import java.io.IOException; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import org.apache.hadoop.fs.FSInputStream; | ||
|
||
import software.amazon.s3.analyticsaccelerator.S3SeekableInputStream; | ||
import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory; | ||
import software.amazon.s3.analyticsaccelerator.util.S3URI; | ||
|
||
public class S3ASeekableStream extends FSInputStream { | ||
|
||
private S3SeekableInputStream inputStream; | ||
private final String key; | ||
|
||
public static final Logger LOG = LoggerFactory.getLogger(S3ASeekableStream.class); | ||
|
||
|
||
public S3ASeekableStream(String bucket, String key, S3SeekableInputStreamFactory s3SeekableInputStreamFactory) | ||
throws IOException { | ||
this.inputStream = s3SeekableInputStreamFactory.createStream(S3URI.of(bucket, key)); | ||
this.key = key; | ||
} | ||
|
||
@Override | ||
public int read() throws IOException { | ||
return inputStream.read(); | ||
} | ||
|
||
@Override | ||
public void seek(long pos) throws IOException { | ||
inputStream.seek(pos); | ||
} | ||
|
||
@Override | ||
public long getPos() throws IOException { | ||
return inputStream.getPos(); | ||
} | ||
|
||
@Override | ||
public void close() throws IOException { | ||
if (inputStream != null) { | ||
inputStream.close(); | ||
inputStream = null; | ||
super.close(); | ||
} | ||
} | ||
|
||
|
||
public void readTail(byte[] buf, int off, int n) throws IOException { | ||
inputStream.readTail(buf, off, n); | ||
} | ||
|
||
@Override | ||
public int read(byte[] buf, int off, int len) throws IOException { | ||
return inputStream.read(buf, off, len); | ||
} | ||
|
||
|
||
@Override | ||
public boolean seekToNewSource(long l) throws IOException { | ||
return false; | ||
} | ||
|
||
} |
78 changes: 78 additions & 0 deletions
78
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,78 @@ | ||
package org.apache.hadoop.fs.s3a; | ||
|
||
import java.io.IOException; | ||
|
||
import org.junit.Test; | ||
|
||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.FSDataInputStream; | ||
import org.apache.hadoop.fs.FileSystem; | ||
import org.apache.hadoop.fs.Path; | ||
|
||
import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX; | ||
import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_ENABLED_KEY; | ||
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; | ||
|
||
import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamConfiguration; | ||
import software.amazon.s3.analyticsaccelerator.common.ConnectorConfiguration; | ||
import software.amazon.s3.analyticsaccelerator.util.PrefetchMode; | ||
|
||
public class ITestS3AS3SeekableStream extends AbstractS3ATestBase { | ||
|
||
final String PHYSICAL_IO_PREFIX = "physicalio"; | ||
final String LOGICAL_IO_PREFIX = "logicalio"; | ||
|
||
@Test | ||
public void testConnectorFrameWorkIntegration() throws IOException { | ||
describe("Verify S3 connector framework integration"); | ||
|
||
Configuration conf = getConfiguration(); | ||
removeBaseAndBucketOverrides(conf, ANALYTICS_ACCELERATOR_ENABLED_KEY); | ||
conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, true); | ||
|
||
String testFile = "s3a://noaa-cors-pds/raw/2023/017/ohfh/OHFH017d.23_.gz"; | ||
S3AFileSystem s3AFileSystem = (S3AFileSystem) FileSystem.newInstance(new Path(testFile).toUri(), conf); | ||
byte[] buffer = new byte[500]; | ||
|
||
try (FSDataInputStream inputStream = s3AFileSystem.open(new Path(testFile))) { | ||
inputStream.seek(5); | ||
inputStream.read(buffer, 0, 500); | ||
} | ||
|
||
} | ||
|
||
|
||
@Test | ||
public void testConnectorFrameworkConfigurable() { | ||
describe("Verify S3 connector framework reads configuration"); | ||
|
||
Configuration conf = getConfiguration(); | ||
removeBaseAndBucketOverrides(conf); | ||
|
||
//Disable Predictive Prefetching | ||
conf.set(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + LOGICAL_IO_PREFIX + ".prefetching.mode", "all"); | ||
|
||
//Set Blobstore Capacity | ||
conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", 1); | ||
|
||
ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); | ||
|
||
S3SeekableInputStreamConfiguration configuration = S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration); | ||
assertEquals(configuration.getLogicalIOConfiguration().getPrefetchingMode(), PrefetchMode.ALL); | ||
assert configuration.getPhysicalIOConfiguration().getBlobStoreCapacity() == 1; | ||
} | ||
|
||
@Test | ||
public void testInvalidConfigurationThrows() { | ||
describe("Verify S3 connector framework throws with invalid configuration"); | ||
|
||
Configuration conf = getConfiguration(); | ||
removeBaseAndBucketOverrides(conf); | ||
//Disable Sequential Prefetching | ||
conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", -1); | ||
|
||
ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); | ||
assertThrows(IllegalArgumentException.class, () -> | ||
S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration)); | ||
} | ||
} |