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

GCS offload support(1): rename s3offload related classes to be reuse-able #2064

Merged
merged 1 commit into from
Jul 2, 2018
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@
import org.apache.pulsar.broker.loadbalance.LoadSheddingTask;
import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared;
import org.apache.pulsar.broker.namespace.NamespaceService;
import org.apache.pulsar.broker.s3offload.S3ManagedLedgerOffloader;
import org.apache.pulsar.broker.offload.impl.S3ManagedLedgerOffloader;
import org.apache.pulsar.broker.service.BrokerService;
import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.broker.service.schema.SchemaRegistryService;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,12 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload;
package org.apache.pulsar.broker.offload;

import java.io.InputStream;
import java.io.IOException;

public abstract class S3BackedInputStream extends InputStream {
public abstract class BackedInputStream extends InputStream {
public abstract void seek(long position);
public abstract void seekForward(long position) throws IOException;
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,8 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload;
package org.apache.pulsar.broker.offload;

import java.io.IOException;
import java.io.InputStream;
import org.apache.bookkeeper.client.api.ReadHandle;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,8 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload;
package org.apache.pulsar.broker.offload;

import java.io.IOException;
import java.io.InputStream;
import org.apache.bookkeeper.common.annotation.InterfaceStability.Unstable;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload;
package org.apache.pulsar.broker.offload;

import java.io.Closeable;
import java.io.FilterInputStream;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,14 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload;
package org.apache.pulsar.broker.offload;

import java.io.IOException;
import java.io.InputStream;
import org.apache.bookkeeper.client.api.LedgerMetadata;
import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate;
import org.apache.bookkeeper.common.annotation.InterfaceStability.Unstable;
import org.apache.pulsar.broker.s3offload.impl.OffloadIndexBlockBuilderImpl;
import org.apache.pulsar.broker.offload.impl.OffloadIndexBlockBuilderImpl;

/**
* Interface for builder of index block used for offload a ledger to long term storage.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload;
package org.apache.pulsar.broker.offload;

import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate;
import org.apache.bookkeeper.common.annotation.InterfaceStability.Unstable;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload.impl;
package org.apache.pulsar.broker.offload.impl;

import static com.google.common.base.Preconditions.checkState;

Expand All @@ -32,7 +32,7 @@
import org.apache.bookkeeper.client.api.LedgerEntries;
import org.apache.bookkeeper.client.api.LedgerEntry;
import org.apache.bookkeeper.client.api.ReadHandle;
import org.apache.pulsar.broker.s3offload.BlockAwareSegmentInputStream;
import org.apache.pulsar.broker.offload.BlockAwareSegmentInputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload.impl;
package org.apache.pulsar.broker.offload.impl;

import com.google.common.io.CountingInputStream;

Expand All @@ -27,7 +27,7 @@
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import org.apache.pulsar.broker.s3offload.DataBlockHeader;
import org.apache.pulsar.broker.offload.DataBlockHeader;

/**
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload.impl;
package org.apache.pulsar.broker.offload.impl;

import static com.google.common.base.Preconditions.checkState;

Expand All @@ -25,8 +25,8 @@
import java.io.InputStream;
import java.util.List;
import org.apache.bookkeeper.client.api.LedgerMetadata;
import org.apache.pulsar.broker.s3offload.OffloadIndexBlock;
import org.apache.pulsar.broker.s3offload.OffloadIndexBlockBuilder;
import org.apache.pulsar.broker.offload.OffloadIndexBlock;
import org.apache.pulsar.broker.offload.OffloadIndexBlockBuilder;

/**
* Interface for builder of index block used for offload a ledger to long term storage.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload.impl;
package org.apache.pulsar.broker.offload.impl;

import static com.google.common.base.Preconditions.checkState;

Expand All @@ -31,7 +31,6 @@

import java.io.DataInputStream;
import java.io.IOException;
import java.io.FilterInputStream;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.List;
Expand All @@ -47,8 +46,8 @@
import org.apache.bookkeeper.proto.DataFormats;
import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat;
import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.State;
import org.apache.pulsar.broker.s3offload.OffloadIndexBlock;
import org.apache.pulsar.broker.s3offload.OffloadIndexEntry;
import org.apache.pulsar.broker.offload.OffloadIndexBlock;
import org.apache.pulsar.broker.offload.OffloadIndexEntry;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,9 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload.impl;
package org.apache.pulsar.broker.offload.impl;

import org.apache.pulsar.broker.s3offload.OffloadIndexEntry;
import org.apache.pulsar.broker.offload.OffloadIndexEntry;

/**
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload.impl;
package org.apache.pulsar.broker.offload.impl;

import com.amazonaws.AmazonClientException;
import com.amazonaws.services.s3.AmazonS3;
Expand All @@ -29,13 +29,13 @@
import java.io.InputStream;
import java.io.IOException;

import org.apache.pulsar.broker.s3offload.S3BackedInputStream;
import org.apache.pulsar.broker.s3offload.S3ManagedLedgerOffloader.VersionCheck;
import org.apache.pulsar.broker.offload.BackedInputStream;
import org.apache.pulsar.broker.offload.impl.S3ManagedLedgerOffloader.VersionCheck;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class S3BackedInputStreamImpl extends S3BackedInputStream {
public class S3BackedInputStreamImpl extends BackedInputStream {
private static final Logger log = LoggerFactory.getLogger(S3BackedInputStreamImpl.class);

private final AmazonS3 s3client;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,11 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload.impl;
package org.apache.pulsar.broker.offload.impl;

import com.amazonaws.AmazonClientException;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.ObjectMetadata;
import com.amazonaws.services.s3.model.S3Object;

import io.netty.buffer.ByteBuf;
Expand All @@ -44,11 +43,11 @@
import org.apache.bookkeeper.client.impl.LedgerEntriesImpl;
import org.apache.bookkeeper.client.impl.LedgerEntryImpl;

import org.apache.pulsar.broker.s3offload.OffloadIndexBlock;
import org.apache.pulsar.broker.s3offload.OffloadIndexBlockBuilder;
import org.apache.pulsar.broker.s3offload.OffloadIndexEntry;
import org.apache.pulsar.broker.s3offload.S3BackedInputStream;
import org.apache.pulsar.broker.s3offload.S3ManagedLedgerOffloader.VersionCheck;
import org.apache.pulsar.broker.offload.OffloadIndexBlock;
import org.apache.pulsar.broker.offload.OffloadIndexBlockBuilder;
import org.apache.pulsar.broker.offload.OffloadIndexEntry;
import org.apache.pulsar.broker.offload.BackedInputStream;
import org.apache.pulsar.broker.offload.impl.S3ManagedLedgerOffloader.VersionCheck;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -58,12 +57,12 @@ public class S3BackedReadHandleImpl implements ReadHandle {

private final long ledgerId;
private final OffloadIndexBlock index;
private final S3BackedInputStream inputStream;
private final BackedInputStream inputStream;
private final DataInputStream dataStream;
private final ExecutorService executor;

private S3BackedReadHandleImpl(long ledgerId, OffloadIndexBlock index,
S3BackedInputStream inputStream,
BackedInputStream inputStream,
ExecutorService executor) {
this.ledgerId = ledgerId;
this.index = index;
Expand Down Expand Up @@ -201,7 +200,7 @@ public static ReadHandle open(ScheduledExecutorService executor,
OffloadIndexBlockBuilder indexBuilder = OffloadIndexBlockBuilder.create();
OffloadIndexBlock index = indexBuilder.fromStream(obj.getObjectContent());

S3BackedInputStream inputStream = new S3BackedInputStreamImpl(s3client, bucket, key,
BackedInputStream inputStream = new S3BackedInputStreamImpl(s3client, bucket, key,
versionCheck,
index.getDataObjectLength(),
readBufferSize);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload;
package org.apache.pulsar.broker.offload.impl;

import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration;
import com.amazonaws.services.s3.AmazonS3;
Expand All @@ -32,7 +32,6 @@
import com.amazonaws.services.s3.model.UploadPartRequest;
import com.amazonaws.services.s3.model.UploadPartResult;
import com.google.common.base.Strings;
import java.io.InputStream;
import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
Expand All @@ -44,8 +43,9 @@
import org.apache.bookkeeper.mledger.LedgerOffloader;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.s3offload.impl.BlockAwareSegmentInputStreamImpl;
import org.apache.pulsar.broker.s3offload.impl.S3BackedReadHandleImpl;
import org.apache.pulsar.broker.offload.BlockAwareSegmentInputStream;
import org.apache.pulsar.broker.offload.OffloadIndexBlock;
import org.apache.pulsar.broker.offload.OffloadIndexBlockBuilder;
import org.apache.pulsar.utils.PulsarBrokerVersionStringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload;
package org.apache.pulsar.broker.offload;

import static org.mockito.Matchers.anyObject;
import static org.mockito.Mockito.spy;
Expand All @@ -34,11 +34,9 @@

import lombok.extern.slf4j.Slf4j;

import org.apache.pulsar.broker.s3offload.impl.S3BackedInputStreamImpl;
import org.apache.pulsar.broker.offload.impl.S3BackedInputStreamImpl;

import org.testng.Assert;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;

@Slf4j
Expand Down Expand Up @@ -96,7 +94,7 @@ public void testReadingFullObject() throws Exception {
metadata.setContentLength(objectSize);
s3client.putObject(BUCKET, objectKey, toWrite, metadata);

S3BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, objectKey,
BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, objectKey,
(key, md) -> {},
objectSize, 1000);
assertStreamsMatch(toTest, toCompare);
Expand All @@ -113,15 +111,15 @@ public void testReadingFullObjectByBytes() throws Exception {
metadata.setContentLength(objectSize);
s3client.putObject(BUCKET, objectKey, toWrite, metadata);

S3BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, objectKey,
BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, objectKey,
(key, md) -> {},
objectSize, 1000);
assertStreamsMatchByBytes(toTest, toCompare);
}

@Test(expectedExceptions = IOException.class)
public void testErrorOnS3Read() throws Exception {
S3BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, "doesn't exist",
BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, "doesn't exist",
(key, md) -> {},
1234, 1000);
toTest.read();
Expand All @@ -147,7 +145,7 @@ public void testSeek() throws Exception {
metadata.setContentLength(objectSize);
s3client.putObject(BUCKET, objectKey, toWrite, metadata);

S3BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, objectKey,
BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, objectKey,
(key, md) -> {},
objectSize, 1000);
for (Map.Entry<Integer, InputStream> e : seeks.entrySet()) {
Expand All @@ -167,7 +165,7 @@ public void testSeekWithinCurrent() throws Exception {
s3client.putObject(BUCKET, objectKey, toWrite, metadata);

AmazonS3 spiedClient = spy(s3client);
S3BackedInputStream toTest = new S3BackedInputStreamImpl(spiedClient, BUCKET, objectKey,
BackedInputStream toTest = new S3BackedInputStreamImpl(spiedClient, BUCKET, objectKey,
(key, md) -> {},
objectSize, 1000);

Expand Down Expand Up @@ -208,7 +206,7 @@ public void testSeekForward() throws Exception {
metadata.setContentLength(objectSize);
s3client.putObject(BUCKET, objectKey, toWrite, metadata);

S3BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, objectKey,
BackedInputStream toTest = new S3BackedInputStreamImpl(s3client, BUCKET, objectKey,
(key, md) -> {},
objectSize, 1000);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload;
package org.apache.pulsar.broker.offload;

import com.amazonaws.services.s3.AbstractAmazonS3;
import com.amazonaws.services.s3.model.AmazonS3Exception;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,18 +16,15 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.s3offload;
package org.apache.pulsar.broker.offload;

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;

import org.testng.annotations.BeforeMethod;

public class S3TestBase {
final static String BUCKET = "pulsar-unittest";
public final static String BUCKET = "pulsar-unittest";

protected AmazonS3 s3client = null;

Expand Down