You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2018/05/15 11:59:12 UTC

[GitHub] sijie closed pull request #1768: InputStream to read from S3

sijie closed pull request #1768: InputStream to read from S3
URL: https://github.com/apache/incubator-pulsar/pull/1768
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/S3BackedInputStream.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/S3BackedInputStream.java
new file mode 100644
index 0000000000..28e68550f6
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/S3BackedInputStream.java
@@ -0,0 +1,27 @@
+/**
+ * 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.pulsar.broker.s3offload;
+
+import java.io.InputStream;
+import java.io.IOException;
+
+public abstract class S3BackedInputStream extends InputStream {
+    public abstract void seek(long position);
+    public abstract void seekForward(long position) throws IOException;
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/S3BackedInputStreamImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/S3BackedInputStreamImpl.java
new file mode 100644
index 0000000000..0c5e3df497
--- /dev/null
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/S3BackedInputStreamImpl.java
@@ -0,0 +1,133 @@
+/**
+ * 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.pulsar.broker.s3offload.impl;
+
+import com.amazonaws.AmazonClientException;
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.model.GetObjectRequest;
+import com.amazonaws.services.s3.model.S3Object;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.PooledByteBufAllocator;
+
+import java.io.InputStream;
+import java.io.IOException;
+
+import org.apache.pulsar.broker.s3offload.S3BackedInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class S3BackedInputStreamImpl extends S3BackedInputStream {
+    private static final Logger log = LoggerFactory.getLogger(S3BackedInputStreamImpl.class);
+
+    private final AmazonS3 s3client;
+    private final String bucket;
+    private final String key;
+    private final ByteBuf buffer;
+    private final long objectLen;
+    private final int bufferSize;
+
+    private long cursor;
+
+    public S3BackedInputStreamImpl(AmazonS3 s3client, String bucket, String key,
+                                   long objectLen, int bufferSize) {
+        this.s3client = s3client;
+        this.bucket = bucket;
+        this.key = key;
+        this.buffer = PooledByteBufAllocator.DEFAULT.buffer(bufferSize, bufferSize);
+        this.objectLen = objectLen;
+        this.bufferSize = bufferSize;
+        this.cursor = 0;
+    }
+
+    /**
+     * Refill the buffered input if it is empty.
+     * @return true if there are bytes to read, false otherwise
+     */
+    private boolean refillBufferIfNeeded() throws IOException {
+        if (buffer.readableBytes() == 0) {
+            if (cursor >= objectLen) {
+                return false;
+            }
+            long startRange = cursor;
+            long endRange = Math.min(cursor + bufferSize - 1,
+                                     objectLen - 1);
+            GetObjectRequest req = new GetObjectRequest(bucket, key)
+                .withRange(startRange, endRange);
+            log.debug("Reading range {}-{} from {}/{}", startRange, endRange, bucket, key);
+            try (S3Object obj = s3client.getObject(req)) {
+                Long[] range = obj.getObjectMetadata().getContentRange();
+                long bytesRead = range[1] - range[0] + 1;
+
+                buffer.clear();
+                InputStream s = obj.getObjectContent();
+                int bytesToCopy = (int)bytesRead;
+                while (bytesToCopy > 0) {
+                    bytesToCopy -= buffer.writeBytes(s, bytesToCopy);
+                }
+                cursor += buffer.readableBytes();
+            } catch (AmazonClientException e) {
+                throw new IOException("Error reading from S3", e);
+            }
+        }
+        return true;
+    }
+
+    @Override
+    public int read() throws IOException {
+        if (refillBufferIfNeeded()) {
+            return buffer.readUnsignedByte();
+        } else {
+            return -1;
+        }
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        if (refillBufferIfNeeded()) {
+            int bytesToRead = Math.min(len, buffer.readableBytes());
+            buffer.readBytes(b, off, bytesToRead);
+            return bytesToRead;
+        } else {
+            return -1;
+        }
+    }
+
+    @Override
+    public void seek(long position) {
+        log.debug("Seeking to {} on {}/{}, current position {}", position, bucket, key, cursor);
+        this.cursor = position;
+        buffer.clear();
+    }
+
+    @Override
+    public void seekForward(long position) throws IOException {
+        if (position >= cursor) {
+            seek(position);
+        } else {
+            throw new IOException(String.format("Error seeking, new position %d < current position %d",
+                                                position, cursor));
+        }
+    }
+
+    @Override
+    public void close() {
+        buffer.release();
+    }
+}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/s3offload/S3BackedInputStreamTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/s3offload/S3BackedInputStreamTest.java
new file mode 100644
index 0000000000..1155c2be9d
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/s3offload/S3BackedInputStreamTest.java
@@ -0,0 +1,177 @@
+/**
+ * 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.pulsar.broker.s3offload;
+
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.model.ObjectMetadata;
+
+import java.io.InputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.pulsar.broker.s3offload.impl.S3BackedInputStreamImpl;
+
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j
+class S3BackedInputStreamTest extends S3TestBase {
+    class RandomInputStream extends InputStream {
+        final Random r;
+        int bytesRemaining;
+
+        RandomInputStream(int seed, int bytesRemaining) {
+            this.r = new Random(seed);
+            this.bytesRemaining = bytesRemaining;
+        }
+
+        @Override
+        public int read() {
+            if (bytesRemaining-- > 0) {
+                return r.nextInt() & 0xFF;
+            } else {
+                return -1;
+            }
+        }
+    }
+
+    private void assertStreamsMatch(InputStream a, InputStream b) throws Exception {
+        int ret = 0;
+        while (ret >= 0) {
+            ret = a.read();
+            Assert.assertEquals(ret, b.read());
+        }
+        Assert.assertEquals(-1, a.read());
+        Assert.assertEquals(-1, b.read());
+    }
+
+    private void assertStreamsMatchByBytes(InputStream a, InputStream b) throws Exception {
+        byte[] bytesA = new byte[100];
+        byte[] bytesB = new byte[100];
+
+        int retA = 0;
+        while (retA >= 0) {
+            retA = a.read(bytesA, 0, 100);
+            int retB = b.read(bytesB, 0, 100);
+            Assert.assertEquals(retA, retB);
+            Assert.assertEquals(bytesA, bytesB);
+        }
+    }
+
+    @Test
+    public void testReadingFullObject() throws Exception {
+        String objectKey = "foobar";
+        int objectSize = 12345;
+        RandomInputStream toWrite = new RandomInputStream(0, objectSize);
+        RandomInputStream toCompare = new RandomInputStream(0, objectSize);
+
+        ObjectMetadata metadata = new ObjectMetadata();
+        metadata.setContentLength(objectSize);
+        s3client.putObject(BUCKET, objectKey, toWrite, metadata);
+
+        S3BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, objectKey, objectSize, 1000);
+        assertStreamsMatch(toTest, toCompare);
+    }
+
+    @Test
+    public void testReadingFullObjectByBytes() throws Exception {
+        String objectKey = "foobar";
+        int objectSize = 12345;
+        RandomInputStream toWrite = new RandomInputStream(0, objectSize);
+        RandomInputStream toCompare = new RandomInputStream(0, objectSize);
+
+        ObjectMetadata metadata = new ObjectMetadata();
+        metadata.setContentLength(objectSize);
+        s3client.putObject(BUCKET, objectKey, toWrite, metadata);
+
+        S3BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, objectKey, objectSize, 1000);
+        assertStreamsMatchByBytes(toTest, toCompare);
+    }
+
+    @Test(expectedExceptions = IOException.class)
+    public void testErrorOnS3Read() throws Exception {
+        S3BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, "doesn't exist", 1234, 1000);
+        toTest.read();
+    }
+
+
+    @Test
+    public void testSeek() throws Exception {
+        String objectKey = "foobar";
+        int objectSize = 12345;
+        RandomInputStream toWrite = new RandomInputStream(0, objectSize);
+
+        Map<Integer, InputStream> seeks = new HashMap<>();
+        Random r = new Random(12345);
+        for (int i = 0; i < 20; i++) {
+            int seek = r.nextInt(objectSize+1);
+            RandomInputStream stream = new RandomInputStream(0, objectSize);
+            stream.skip(seek);
+            seeks.put(seek, stream);
+        }
+
+        ObjectMetadata metadata = new ObjectMetadata();
+        metadata.setContentLength(objectSize);
+        s3client.putObject(BUCKET, objectKey, toWrite, metadata);
+
+        S3BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, objectKey, objectSize, 1000);
+        for (Map.Entry<Integer, InputStream> e : seeks.entrySet()) {
+            toTest.seek(e.getKey());
+            assertStreamsMatch(toTest, e.getValue());
+        }
+    }
+
+    @Test
+    public void testSeekForward() throws Exception {
+        String objectKey = "foobar";
+        int objectSize = 12345;
+        RandomInputStream toWrite = new RandomInputStream(0, objectSize);
+
+        ObjectMetadata metadata = new ObjectMetadata();
+        metadata.setContentLength(objectSize);
+        s3client.putObject(BUCKET, objectKey, toWrite, metadata);
+
+        S3BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, objectKey, objectSize, 1000);
+
+        // seek forward to middle
+        long middle = objectSize/2;
+        toTest.seekForward(middle);
+
+        try {
+            long before = middle - objectSize/4;
+            toTest.seekForward(before);
+            Assert.fail("Shound't be able to seek backwards");
+        } catch (IOException ioe) {
+            // correct
+        }
+
+        long after = middle + objectSize/4;
+        RandomInputStream toCompare = new RandomInputStream(0, objectSize);
+        toCompare.skip(after);
+
+        toTest.seekForward(after);
+        assertStreamsMatch(toTest, toCompare);
+    }
+}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/s3offload/S3TestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/s3offload/S3TestBase.java
index e3b994bfd4..2c65c0cd50 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/s3offload/S3TestBase.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/s3offload/S3TestBase.java
@@ -18,6 +18,8 @@
  */
 package org.apache.pulsar.broker.s3offload;
 
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
 import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration;
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.AmazonS3ClientBuilder;
@@ -46,6 +48,7 @@ public void start() throws Exception {
         } else {
             s3client = AmazonS3ClientBuilder.standard()
                 .withEndpointConfiguration(new EndpointConfiguration(s3endpoint, "foobar"))
+                .withCredentials(new AWSStaticCredentialsProvider(new AnonymousAWSCredentials()))
                 .withPathStyleAccessEnabled(true).build();
         }
 


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services