From 9e14bea64811d0028a43201e21be3291e14e0b5e Mon Sep 17 00:00:00 2001 From: Prashanth Sagar Date: Wed, 18 Aug 2021 00:55:54 +0530 Subject: [PATCH] Migrate gRPC channels to GCS v2 APIs (#590) --- bigquery/pom.xml | 2 + gcs/pom.xml | 2 + gcsio/pom.xml | 29 +- .../GoogleCloudStorageGrpcReadChannel.java | 51 +- .../GoogleCloudStorageGrpcWriteChannel.java | 142 +- .../cloud/hadoop/gcsio/GrpcChannelUtils.java | 35 + .../hadoop/gcsio/StorageStubProvider.java | 10 +- .../proto/google/storage/v1/storage.proto | 1986 ----------------- .../google/storage/v1/storage_resources.proto | 793 ------- ...GoogleCloudStorageGrpcReadChannelTest.java | 826 ++++--- ...oogleCloudStorageGrpcWriteChannelTest.java | 145 +- .../hadoop/gcsio/GrpcChannelUtilsTest.java | 46 + .../gcsio/ZeroCopyMessageMarshallerTest.java | 22 +- pom.xml | 24 +- 14 files changed, 751 insertions(+), 3362 deletions(-) create mode 100644 gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GrpcChannelUtils.java delete mode 100644 gcsio/src/main/proto/google/storage/v1/storage.proto delete mode 100644 gcsio/src/main/proto/google/storage/v1/storage_resources.proto create mode 100644 gcsio/src/test/java/com/google/cloud/hadoop/gcsio/GrpcChannelUtilsTest.java diff --git a/bigquery/pom.xml b/bigquery/pom.xml index 5f2f96c95e..eb8ee093b8 100644 --- a/bigquery/pom.xml +++ b/bigquery/pom.xml @@ -282,6 +282,7 @@ com.google.http-client com.google.oauth-client com.google.protobuf + com.google.storage.v2 io.grpc io.opencensus io.perfmark @@ -306,6 +307,7 @@ com.google.longrunning.** com.google.protobuf.** com.google.rpc.** + com.google.storage.** com.google.cloud.hadoop.util.AccessTokenProvider diff --git a/gcs/pom.xml b/gcs/pom.xml index 5d44aa55d4..cb8fce3f4c 100644 --- a/gcs/pom.xml +++ b/gcs/pom.xml @@ -267,6 +267,7 @@ com.google.http-client com.google.oauth-client com.google.protobuf + com.google.storage.v2 commons-codec:commons-codec io.grpc @@ -297,6 +298,7 @@ com.google.protobuf.** com.google.thirdparty.** com.google.type.** + com.google.storage.** com.google.cloud.hadoop.gcsio.authorization.AuthorizationHandler diff --git a/gcsio/pom.xml b/gcsio/pom.xml index aff45517a5..e489682426 100644 --- a/gcsio/pom.xml +++ b/gcsio/pom.xml @@ -93,6 +93,10 @@ io.grpc grpc-protobuf + + com.google.api.grpc + grpc-google-cloud-storage-v2 + io.grpc grpc-stub @@ -167,10 +171,6 @@ - - kr.motd.maven - os-maven-plugin - maven-surefire-plugin @@ -180,27 +180,6 @@ - - org.xolstice.maven.plugins - protobuf-maven-plugin - - - com.google.protobuf:protoc:${google.protobuf.version}:exe:${os.detected.classifier} - - grpc-java - - io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier} - - - - - - compile - compile-custom - - - - diff --git a/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcReadChannel.java b/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcReadChannel.java index 3a6a1436a0..df0abff44b 100644 --- a/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcReadChannel.java +++ b/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcReadChannel.java @@ -33,11 +33,11 @@ import com.google.common.base.Preconditions; import com.google.common.flogger.GoogleLogger; import com.google.common.hash.Hashing; -import com.google.google.storage.v1.GetObjectMediaRequest; -import com.google.google.storage.v1.GetObjectMediaResponse; -import com.google.google.storage.v1.StorageGrpc; -import com.google.google.storage.v1.StorageGrpc.StorageBlockingStub; import com.google.protobuf.ByteString; +import com.google.storage.v2.ReadObjectRequest; +import com.google.storage.v2.ReadObjectResponse; +import com.google.storage.v2.StorageGrpc; +import com.google.storage.v2.StorageGrpc.StorageBlockingStub; import io.grpc.Context; import io.grpc.Context.CancellableContext; import io.grpc.MethodDescriptor; @@ -60,10 +60,10 @@ public class GoogleCloudStorageGrpcReadChannel implements SeekableByteChannel { // ZeroCopy version of GetObjectMedia Method private static final ZeroCopyMessageMarshaller getObjectMediaResponseMarshaller = - new ZeroCopyMessageMarshaller(GetObjectMediaResponse.getDefaultInstance()); - private static final MethodDescriptor + new ZeroCopyMessageMarshaller(ReadObjectResponse.getDefaultInstance()); + private static final MethodDescriptor getObjectMediaMethod = - StorageGrpc.getGetObjectMediaMethod() + StorageGrpc.getReadObjectMethod() .toBuilder() .setResponseMarshaller(getObjectMediaResponseMarshaller) .build(); @@ -104,7 +104,7 @@ public class GoogleCloudStorageGrpcReadChannel implements SeekableByteChannel { @Nullable private InputStream streamForBufferedContent = null; // The streaming read operation. If null, there is not an in-flight read in progress. - @Nullable private Iterator resIterator = null; + @Nullable private Iterator resIterator = null; // Fine-grained options. private final GoogleCloudStorageReadOptions readOptions; @@ -256,20 +256,20 @@ private static ByteString getFooterContent( long footerOffset) throws IOException { try { - Iterator footerContentResponse = + Iterator footerContentResponse = stub.withDeadlineAfter(readOptions.getGrpcReadTimeoutMillis(), MILLISECONDS) - .getObjectMedia( - GetObjectMediaRequest.newBuilder() + .readObject( + ReadObjectRequest.newBuilder() .setReadOffset(footerOffset) - .setBucket(resourceId.getBucketName()) + .setBucket(GrpcChannelUtils.toV2BucketName(resourceId.getBucketName())) .setObject(resourceId.getObjectName()) .build()); ByteString footerContent = null; while (footerContentResponse.hasNext()) { - GetObjectMediaResponse objectMediaResponse = footerContentResponse.next(); - if (objectMediaResponse.hasChecksummedData()) { - ByteString content = objectMediaResponse.getChecksummedData().getContent(); + ReadObjectResponse readObjectResponse = footerContentResponse.next(); + if (readObjectResponse.hasChecksummedData()) { + ByteString content = readObjectResponse.getChecksummedData().getContent(); if (footerContent == null) { footerContent = content; } else { @@ -439,7 +439,8 @@ private boolean isByteBufferBeyondCurrentRequestRange(ByteBuffer byteBuffer) { private int readObjectContentFromGCS(ByteBuffer byteBuffer) throws IOException { int bytesRead = 0; while (moreServerContent() && byteBuffer.hasRemaining()) { - GetObjectMediaResponse res = resIterator.next(); + ReadObjectResponse res = resIterator.next(); + // When zero-copy mashaller is used, the stream that backs GetObjectMediaResponse // should be closed when the mssage is no longed needed so that all buffers in the // stream can be reclaimed. If zero-copy is not used, stream will be null. @@ -484,11 +485,11 @@ private int readObjectContentFromGCS(ByteBuffer byteBuffer) throws IOException { return bytesRead; } - private void validateChecksum(GetObjectMediaResponse res) throws IOException { + private void validateChecksum(ReadObjectResponse res) throws IOException { // TODO: Concatenate all these hashes together and compare the result at the end. int calculatedChecksum = Hashing.crc32c().hashBytes(res.getChecksummedData().getContent().toByteArray()).asInt(); - int expectedChecksum = res.getChecksummedData().getCrc32C().getValue(); + int expectedChecksum = res.getChecksummedData().getCrc32C(); if (calculatedChecksum != expectedChecksum) { throw new IOException( String.format( @@ -534,14 +535,14 @@ private int readFooterContentIntoBuffer(ByteBuffer byteBuffer) { } private void requestObjectMedia(OptionalLong bytesToRead) throws IOException { - GetObjectMediaRequest.Builder requestBuilder = - GetObjectMediaRequest.newBuilder() - .setBucket(resourceId.getBucketName()) + ReadObjectRequest.Builder requestBuilder = + ReadObjectRequest.newBuilder() + .setBucket(GrpcChannelUtils.toV2BucketName(resourceId.getBucketName())) .setObject(resourceId.getObjectName()) .setGeneration(objectGeneration) .setReadOffset(positionInGrpcStream); bytesToRead.ifPresent(requestBuilder::setReadLimit); - GetObjectMediaRequest request = requestBuilder.build(); + ReadObjectRequest request = requestBuilder.build(); try { ResilientOperation.retry( () -> { @@ -559,7 +560,7 @@ private void requestObjectMedia(OptionalLong bytesToRead) throws IOException { blockingStub.getCallOptions(), request); } else { - resIterator = blockingStub.getObjectMedia(request); + resIterator = blockingStub.readObject(request); } } finally { requestContext.detach(toReattach); @@ -641,9 +642,7 @@ public long position() throws IOException { throw new ClosedChannelException(); } // Our real position is tracked in "positionInGrpcStream," but if the user is skipping - // forwards - // a bit, we - // pretend we're at the new position already. + // forwards a bit, we pretend we're at the new position already. return positionInGrpcStream + bytesToSkipBeforeReading; } diff --git a/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcWriteChannel.java b/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcWriteChannel.java index 26046d31d8..ca56b388f1 100644 --- a/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcWriteChannel.java +++ b/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcWriteChannel.java @@ -16,7 +16,7 @@ import static com.google.cloud.hadoop.gcsio.GoogleCloudStorageImpl.encodeMetadata; import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.google.storage.v1.ServiceConstants.Values.MAX_WRITE_CHUNK_BYTES; +import static com.google.storage.v2.ServiceConstants.Values.MAX_WRITE_CHUNK_BYTES; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.stream.Collectors.toMap; @@ -30,20 +30,21 @@ import com.google.common.hash.Hashing; import com.google.common.io.BaseEncoding; import com.google.common.io.ByteStreams; -import com.google.google.storage.v1.ChecksummedData; -import com.google.google.storage.v1.InsertObjectRequest; -import com.google.google.storage.v1.InsertObjectSpec; -import com.google.google.storage.v1.Object; -import com.google.google.storage.v1.ObjectChecksums; -import com.google.google.storage.v1.QueryWriteStatusRequest; -import com.google.google.storage.v1.QueryWriteStatusResponse; -import com.google.google.storage.v1.StartResumableWriteRequest; -import com.google.google.storage.v1.StartResumableWriteResponse; -import com.google.google.storage.v1.StorageGrpc.StorageStub; import com.google.protobuf.ByteString; -import com.google.protobuf.Int64Value; -import com.google.protobuf.UInt32Value; import com.google.protobuf.util.Timestamps; +import com.google.storage.v2.ChecksummedData; +import com.google.storage.v2.CommonRequestParams; +import com.google.storage.v2.CommonRequestParams.Builder; +import com.google.storage.v2.Object; +import com.google.storage.v2.ObjectChecksums; +import com.google.storage.v2.QueryWriteStatusRequest; +import com.google.storage.v2.QueryWriteStatusResponse; +import com.google.storage.v2.StartResumableWriteRequest; +import com.google.storage.v2.StartResumableWriteResponse; +import com.google.storage.v2.StorageGrpc.StorageStub; +import com.google.storage.v2.WriteObjectRequest; +import com.google.storage.v2.WriteObjectResponse; +import com.google.storage.v2.WriteObjectSpec; import io.grpc.Status; import io.grpc.stub.ClientCallStreamObserver; import io.grpc.stub.ClientResponseObserver; @@ -61,7 +62,7 @@ /** Implements WritableByteChannel to provide write access to GCS via gRPC. */ public final class GoogleCloudStorageGrpcWriteChannel - extends BaseAbstractGoogleAsyncWriteChannel + extends BaseAbstractGoogleAsyncWriteChannel implements GoogleCloudStorageItemInfo.Provider { private static final Duration START_RESUMABLE_WRITE_TIMEOUT = Duration.ofMinutes(1); @@ -111,33 +112,39 @@ protected String getResourceString() { } @Override - public void handleResponse(Object response) { + public void handleResponse(WriteObjectResponse response) { + Object resource = response.getResource(); Map metadata = - response.getMetadataMap().entrySet().stream() + resource.getMetadataMap().entrySet().stream() .collect( toMap(Map.Entry::getKey, entry -> BaseEncoding.base64().decode(entry.getValue()))); - byte[] md5Hash = - !response.getMd5Hash().isEmpty() - ? BaseEncoding.base64().decode(response.getMd5Hash()) - : null; + byte[] md5Hash = null; + byte[] crc32c = null; - byte[] crc32c = - response.hasCrc32C() - ? ByteBuffer.allocate(4).putInt(response.getCrc32C().getValue()).array() - : null; + if (resource.hasChecksums()) { + md5Hash = + !resource.getChecksums().getMd5Hash().isEmpty() + ? resource.getChecksums().getMd5Hash().toByteArray() + : null; + + crc32c = + resource.getChecksums().hasCrc32C() + ? ByteBuffer.allocate(4).putInt(resource.getChecksums().getCrc32C()).array() + : null; + } completedItemInfo = GoogleCloudStorageItemInfo.createObject( resourceId, - Timestamps.toMillis(response.getTimeCreated()), - Timestamps.toMillis(response.getUpdated()), - response.getSize(), - response.getContentType(), - response.getContentEncoding(), + Timestamps.toMillis(resource.getCreateTime()), + Timestamps.toMillis(resource.getUpdateTime()), + resource.getSize(), + resource.getContentType(), + resource.getContentEncoding(), metadata, - response.getGeneration(), - response.getMetageneration(), + resource.getGeneration(), + resource.getMetageneration(), new VerificationAttributes(md5Hash, crc32c)); } @@ -152,7 +159,7 @@ public void startUpload(InputStream pipeSource) { } } - private class UploadOperation implements Callable { + private class UploadOperation implements Callable { // Read end of the pipe. private final BufferedInputStream pipeSource; @@ -162,8 +169,8 @@ private class UploadOperation implements Callable { private String uploadId; private long writeOffset = 0; private InsertChunkResponseObserver responseObserver; - // Holds list of most recent number of NUMBER_OF_REQUESTS_TO_RETAIN requests, so upload can be - // rewound and re-sent upon transient errors. + // Holds list of most recent number of NUMBER_OF_REQUESTS_TO_RETAIN requests, so upload can + // be rewound and re-sent upon transient errors. private final TreeMap dataChunkMap = new TreeMap<>(); UploadOperation(InputStream pipeSource) { @@ -174,7 +181,7 @@ private class UploadOperation implements Callable { } @Override - public Object call() throws IOException { + public WriteObjectResponse call() throws IOException { // Try-with-resource will close this end of the pipe so that // the writer at the other end will not hang indefinitely. // Send the initial StartResumableWrite request to get an uploadId. @@ -192,16 +199,16 @@ public Object call() throws IOException { } } - private Object doResumableUpload() throws IOException { + private WriteObjectResponse doResumableUpload() throws IOException { // Only request committed size for the first insert request. if (writeOffset > 0) { writeOffset = getCommittedWriteSize(uploadId); } responseObserver = new InsertChunkResponseObserver(uploadId, writeOffset); // TODO(b/151184800): Implement per-message timeout, in addition to stream timeout. - StreamObserver requestStreamObserver = + StreamObserver requestStreamObserver = stub.withDeadlineAfter(channelOptions.getGrpcWriteTimeout(), MILLISECONDS) - .insertObject(responseObserver); + .writeObject(responseObserver); // Wait for streaming RPC to become ready for upload. try { @@ -216,7 +223,7 @@ private Object doResumableUpload() throws IOException { boolean objectFinalized = false; while (!objectFinalized) { - InsertObjectRequest insertRequest; + WriteObjectRequest insertRequest; if (dataChunkMap.size() > 0 && dataChunkMap.lastKey() >= writeOffset) { insertRequest = buildRequestFromBufferedDataChunk(dataChunkMap, writeOffset); writeOffset += insertRequest.getChecksummedData().getContent().size(); @@ -260,10 +267,10 @@ private Object doResumableUpload() throws IOException { return responseObserver.getResponseOrThrow(); } - private InsertObjectRequest buildInsertRequest( + private WriteObjectRequest buildInsertRequest( long writeOffset, ByteString dataChunk, boolean resumeFromFailedInsert) { - InsertObjectRequest.Builder requestBuilder = - InsertObjectRequest.newBuilder().setUploadId(uploadId).setWriteOffset(writeOffset); + WriteObjectRequest.Builder requestBuilder = + WriteObjectRequest.newBuilder().setUploadId(uploadId).setWriteOffset(writeOffset); if (dataChunk.size() > 0) { ChecksummedData.Builder requestDataBuilder = @@ -272,7 +279,7 @@ private InsertObjectRequest buildInsertRequest( if (!resumeFromFailedInsert) { updateObjectHash(dataChunk); } - requestDataBuilder.setCrc32C(UInt32Value.newBuilder().setValue(getChunkHash(dataChunk))); + requestDataBuilder.setCrc32C(getChunkHash(dataChunk)); } requestBuilder.setChecksummedData(requestDataBuilder); } @@ -281,8 +288,7 @@ private InsertObjectRequest buildInsertRequest( requestBuilder.setFinishWrite(true); if (channelOptions.isGrpcChecksumsEnabled()) { requestBuilder.setObjectChecksums( - ObjectChecksums.newBuilder() - .setCrc32C(UInt32Value.newBuilder().setValue(objectHasher.hash().asInt()))); + ObjectChecksums.newBuilder().setCrc32C(objectHasher.hash().asInt())); } } @@ -306,11 +312,11 @@ private void updateObjectHash(ByteString dataChunk) { // Handles the case when a writeOffset of data read previously is being processed. // This happens if a transient failure happens while uploading, and can be resumed by // querying the current committed offset. - private InsertObjectRequest buildRequestFromBufferedDataChunk( + private WriteObjectRequest buildRequestFromBufferedDataChunk( TreeMap dataChunkMap, long writeOffset) throws IOException { // Resume will only work if the first request builder in the cache carries an offset // not greater than the current writeOffset. - InsertObjectRequest request = null; + WriteObjectRequest request = null; if (dataChunkMap.size() > 0 && dataChunkMap.firstKey() <= writeOffset) { for (Map.Entry entry : dataChunkMap.entrySet()) { if (entry.getKey() + entry.getValue().size() > writeOffset @@ -342,19 +348,19 @@ private void recreateStub(Status.Code statusCode) { /** Handler for responses from the Insert streaming RPC. */ private class InsertChunkResponseObserver - implements ClientResponseObserver { + implements ClientResponseObserver { private final long writeOffset; private final String uploadId; // The response from the server, populated at the end of a successful streaming RPC. - private Object response; + private WriteObjectResponse response; // The last transient error to occur during the streaming RPC. public Throwable transientError = null; // The last non-transient error to occur during the streaming RPC. public Throwable nonTransientError = null; - // CountDownLatch tracking completion of the streaming RPC. Set on error, or once the request - // stream is closed. + // CountDownLatch tracking completion of the streaming RPC. Set on error, or once the + // request stream is closed. final CountDownLatch done = new CountDownLatch(1); // CountDownLatch tracking readiness of the streaming RPC. final CountDownLatch ready = new CountDownLatch(1); @@ -364,7 +370,7 @@ private class InsertChunkResponseObserver this.writeOffset = writeOffset; } - public Object getResponseOrThrow() throws IOException { + public WriteObjectResponse getResponseOrThrow() throws IOException { if (hasNonTransientError()) { throw new IOException( String.format("Resumable upload failed for '%s'", resourceId), nonTransientError); @@ -381,7 +387,7 @@ boolean hasNonTransientError() { } @Override - public void onNext(Object response) { + public void onNext(WriteObjectResponse response) { this.response = response; } @@ -412,37 +418,41 @@ public void onCompleted() { @Override public void beforeStart( - ClientCallStreamObserver clientCallStreamObserver) { + ClientCallStreamObserver clientCallStreamObserver) { clientCallStreamObserver.setOnReadyHandler(ready::countDown); } } /** Send a StartResumableWriteRequest and return the uploadId of the resumable write. */ private String startResumableUpload() throws IOException { - InsertObjectSpec.Builder insertObjectSpecBuilder = - InsertObjectSpec.newBuilder() + WriteObjectSpec.Builder insertObjectSpecBuilder = + WriteObjectSpec.newBuilder() .setResource( Object.newBuilder() - .setBucket(resourceId.getBucketName()) + .setBucket(GrpcChannelUtils.toV2BucketName(resourceId.getBucketName())) .setName(resourceId.getObjectName()) .setContentType(createOptions.getContentType()) .putAllMetadata(encodeMetadata(createOptions.getMetadata())) .build()); if (writeConditions.hasContentGenerationMatch()) { - insertObjectSpecBuilder.setIfGenerationMatch( - Int64Value.newBuilder().setValue(writeConditions.getContentGenerationMatch())); + insertObjectSpecBuilder.setIfGenerationMatch(writeConditions.getContentGenerationMatch()); } if (writeConditions.hasMetaGenerationMatch()) { - insertObjectSpecBuilder.setIfMetagenerationMatch( - Int64Value.newBuilder().setValue(writeConditions.getMetaGenerationMatch())); + insertObjectSpecBuilder.setIfMetagenerationMatch(writeConditions.getMetaGenerationMatch()); } + + Builder commonRequestParamsBuilder = null; if (requesterPaysProject != null) { - insertObjectSpecBuilder.setUserProject(requesterPaysProject); + commonRequestParamsBuilder = + CommonRequestParams.newBuilder().setUserProject(requesterPaysProject); + } + + StartResumableWriteRequest.Builder startResumableWriteRequestBuilder = + StartResumableWriteRequest.newBuilder().setWriteObjectSpec(insertObjectSpecBuilder); + if (commonRequestParamsBuilder != null) { + startResumableWriteRequestBuilder.setCommonRequestParams(commonRequestParamsBuilder); } - StartResumableWriteRequest request = - StartResumableWriteRequest.newBuilder() - .setInsertObjectSpec(insertObjectSpecBuilder) - .build(); + StartResumableWriteRequest request = startResumableWriteRequestBuilder.build(); SimpleResponseObserver responseObserver = new SimpleResponseObserver<>(); diff --git a/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GrpcChannelUtils.java b/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GrpcChannelUtils.java new file mode 100644 index 0000000000..b79fc0c525 --- /dev/null +++ b/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GrpcChannelUtils.java @@ -0,0 +1,35 @@ +/* + * Copyright 2021 Google Inc. All Rights Reserved. + * + * Licensed 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 com.google.cloud.hadoop.gcsio; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; + +/** Utility methods for gRPC channel */ +final class GrpcChannelUtils { + + private GrpcChannelUtils() {} + + static final String V2_BUCKET_NAME_PREFIX = "projects/_/buckets/"; + + /** + * @param v1BucketName name of the bucket + * @return bucket name in to GCS v2 bucket name format + */ + static String toV2BucketName(String v1BucketName) { + checkArgument(!isNullOrEmpty(v1BucketName), "v1BucketName cannot be null or empty"); + return V2_BUCKET_NAME_PREFIX + v1BucketName; + } +} diff --git a/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/StorageStubProvider.java b/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/StorageStubProvider.java index 0561d65d7c..800f9f38c0 100644 --- a/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/StorageStubProvider.java +++ b/gcsio/src/main/java/com/google/cloud/hadoop/gcsio/StorageStubProvider.java @@ -13,11 +13,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.google.storage.v1.StorageGrpc; -import com.google.google.storage.v1.StorageGrpc.StorageBlockingStub; -import com.google.google.storage.v1.StorageGrpc.StorageStub; -import com.google.google.storage.v1.StorageOuterClass; import com.google.protobuf.util.Durations; +import com.google.storage.v2.StorageGrpc; +import com.google.storage.v2.StorageGrpc.StorageBlockingStub; +import com.google.storage.v2.StorageGrpc.StorageStub; +import com.google.storage.v2.StorageProto; import io.grpc.CallOptions; import io.grpc.Channel; import io.grpc.ClientCall; @@ -59,7 +59,7 @@ class StorageStubProvider { // The GCS gRPC server address. private static final String DEFAULT_GCS_GRPC_SERVER_ADDRESS = - StorageOuterClass.getDescriptor() + StorageProto.getDescriptor() .findServiceByName("Storage") .getOptions() .getExtension(ClientProto.defaultHost); diff --git a/gcsio/src/main/proto/google/storage/v1/storage.proto b/gcsio/src/main/proto/google/storage/v1/storage.proto deleted file mode 100644 index 2d0a7a0ff3..0000000000 --- a/gcsio/src/main/proto/google/storage/v1/storage.proto +++ /dev/null @@ -1,1986 +0,0 @@ -// Copyright 2019 Google LLC. -// -// Licensed 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. -// - -syntax = "proto3"; - -package google.storage.v1; - -import "google/iam/v1/iam_policy.proto"; -import "google/iam/v1/policy.proto"; -import "google/protobuf/empty.proto"; -import "google/protobuf/field_mask.proto"; -import "google/protobuf/wrappers.proto"; -import "google/storage/v1/storage_resources.proto"; -import "google/api/client.proto"; - -option go_package = "google.golang.org/genproto/googleapis/storage/v1;storage"; -option java_multiple_files = true; -option java_package = "com.google.google.storage.v1"; - -// Manages Google Cloud Storage resources. -service Storage { - option (google.api.default_host) = "storage.googleapis.com"; - option (google.api.oauth_scopes) = - "https://www.googleapis.com/auth/cloud-platform," - "https://www.googleapis.com/auth/cloud-platform.read-only," - "https://www.googleapis.com/auth/devstorage.full_control," - "https://www.googleapis.com/auth/devstorage.read_only," - "https://www.googleapis.com/auth/devstorage.read_write"; - - // Permanently deletes the ACL entry for the specified entity on the specified - // bucket. - rpc DeleteBucketAccessControl(DeleteBucketAccessControlRequest) returns (google.protobuf.Empty) { - } - - // Returns the ACL entry for the specified entity on the specified bucket. - rpc GetBucketAccessControl(GetBucketAccessControlRequest) returns (BucketAccessControl) { - } - - // Creates a new ACL entry on the specified bucket. - rpc InsertBucketAccessControl(InsertBucketAccessControlRequest) returns (BucketAccessControl) { - } - - // Retrieves ACL entries on the specified bucket. - rpc ListBucketAccessControls(ListBucketAccessControlsRequest) returns (ListBucketAccessControlsResponse) { - } - - // Updates an ACL entry on the specified bucket. Equivalent to - // PatchBucketAccessControl, but all unspecified fields will be - // reset to their default values. - rpc UpdateBucketAccessControl(UpdateBucketAccessControlRequest) returns (BucketAccessControl) { - } - - // Updates an ACL entry on the specified bucket. - rpc PatchBucketAccessControl(PatchBucketAccessControlRequest) returns (BucketAccessControl) { - } - - // Permanently deletes an empty bucket. - rpc DeleteBucket(DeleteBucketRequest) returns (google.protobuf.Empty) { - } - - // Returns metadata for the specified bucket. - rpc GetBucket(GetBucketRequest) returns (Bucket) { - } - - // Creates a new bucket. - rpc InsertBucket(InsertBucketRequest) returns (Bucket) { - } - - // Retrieves a list of buckets for a given project. - rpc ListBuckets(ListBucketsRequest) returns (ListBucketsResponse) { - } - - // Locks retention policy on a bucket. - rpc LockBucketRetentionPolicy(LockRetentionPolicyRequest) returns (Bucket) { - } - - // Gets the IAM policy for the specified bucket. - rpc GetBucketIamPolicy(google.iam.v1.GetIamPolicyRequest) returns (google.iam.v1.Policy) { - } - - // Updates an IAM policy for the specified bucket. - rpc SetBucketIamPolicy(google.iam.v1.SetIamPolicyRequest) returns (google.iam.v1.Policy) { - } - - // Tests a set of permissions on the given bucket to see which, if - // any, are held by the caller. - rpc TestBucketIamPermissions(google.iam.v1.TestIamPermissionsRequest) returns (google.iam.v1.TestIamPermissionsResponse) { - } - - // Updates a bucket. Changes to the bucket will be readable immediately after - // writing, but configuration changes may take time to propagate. - rpc PatchBucket(PatchBucketRequest) returns (Bucket) { - } - - // Updates a bucket. Equivalent to PatchBucket, but always replaces all - // mutatable fields of the bucket with new values, reverting all - // unspecified fields to their default values. - // Like PatchBucket, Changes to the bucket will be readable immediately after - // writing, but configuration changes may take time to propagate. - rpc UpdateBucket(UpdateBucketRequest) returns (Bucket) { - } - - // Halts "Object Change Notification" push messagages. - // See https://cloud.google.com/storage/docs/object-change-notification - // Note: this is not related to the newer "Notifications" resource, which - // are stopped using DeleteNotification. - rpc StopChannel(StopChannelRequest) returns (google.protobuf.Empty) { - } - - // Permanently deletes the default object ACL entry for the specified entity - // on the specified bucket. - rpc DeleteDefaultObjectAccessControl(DeleteDefaultObjectAccessControlRequest) returns (google.protobuf.Empty) { - } - - // Returns the default object ACL entry for the specified entity on the - // specified bucket. - rpc GetDefaultObjectAccessControl(GetDefaultObjectAccessControlRequest) returns (ObjectAccessControl) { - } - - // Creates a new default object ACL entry on the specified bucket. - rpc InsertDefaultObjectAccessControl(InsertDefaultObjectAccessControlRequest) returns (ObjectAccessControl) { - } - - // Retrieves default object ACL entries on the specified bucket. - rpc ListDefaultObjectAccessControls(ListDefaultObjectAccessControlsRequest) returns (ListObjectAccessControlsResponse) { - } - - // Updates a default object ACL entry on the specified bucket. - rpc PatchDefaultObjectAccessControl(PatchDefaultObjectAccessControlRequest) returns (ObjectAccessControl) { - } - - // Updates a default object ACL entry on the specified bucket. Equivalent to - // PatchDefaultObjectAccessControl, but modifies all unspecified fields to - // their default values. - rpc UpdateDefaultObjectAccessControl(UpdateDefaultObjectAccessControlRequest) returns (ObjectAccessControl) { - } - - // Permanently deletes a notification subscription. - // Note: Older, "Object Change Notification" push subscriptions should be - // deleted using StopChannel instead. - rpc DeleteNotification(DeleteNotificationRequest) returns (google.protobuf.Empty) { - } - - // View a notification configuration. - rpc GetNotification(GetNotificationRequest) returns (Notification) { - } - - // Creates a notification subscription for a given bucket. - // These notifications, when triggered, publish messages to the specified - // Cloud Pub/Sub topics. - // See https://cloud.google.com/storage/docs/pubsub-notifications. - rpc InsertNotification(InsertNotificationRequest) returns (Notification) { - } - - // Retrieves a list of notification subscriptions for a given bucket. - rpc ListNotifications(ListNotificationsRequest) returns (ListNotificationsResponse) { - } - - // Permanently deletes the ACL entry for the specified entity on the specified - // object. - rpc DeleteObjectAccessControl(DeleteObjectAccessControlRequest) returns (google.protobuf.Empty) { - } - - // Returns the ACL entry for the specified entity on the specified object. - rpc GetObjectAccessControl(GetObjectAccessControlRequest) returns (ObjectAccessControl) { - } - - // Creates a new ACL entry on the specified object. - rpc InsertObjectAccessControl(InsertObjectAccessControlRequest) returns (ObjectAccessControl) { - } - - // Retrieves ACL entries on the specified object. - rpc ListObjectAccessControls(ListObjectAccessControlsRequest) returns (ListObjectAccessControlsResponse) { - } - - // Updates an ACL entry on the specified object. - rpc UpdateObjectAccessControl(UpdateObjectAccessControlRequest) returns (ObjectAccessControl) { - } - - // Concatenates a list of existing objects into a new object in the same - // bucket. - rpc ComposeObject(ComposeObjectRequest) returns (Object) { - } - - // Copies a source object to a destination object. Optionally overrides - // metadata. - rpc CopyObject(CopyObjectRequest) returns (Object) { - } - - // Deletes an object and its metadata. Deletions are permanent if versioning - // is not enabled for the bucket, or if the generation parameter - // is used. - rpc DeleteObject(DeleteObjectRequest) returns (google.protobuf.Empty) { - } - - // Retrieves an object's metadata. - rpc GetObject(GetObjectRequest) returns (Object) { - } - - // Reads an object's data. - rpc GetObjectMedia(GetObjectMediaRequest) returns (stream GetObjectMediaResponse) { - } - - // Stores a new object and metadata. - // - // An object can be written either in a single message stream or in a - // resumable sequence of message streams. To write using a single stream, - // the client should include in the first message of the stream an - // `InsertObjectSpec` describing the destination bucket, object, and any - // preconditions. Additionally, the final message must set 'finish_write' to - // true, or else it is an error. - // - // For a resumable write, the client should instead call - // `StartResumableWrite()` and provide that method an `InsertObjectSpec.` - // They should then attach the returned `upload_id` to the first message of - // each following call to `Insert`. If there is an error or the connection is - // broken during the resumable `Insert()`, the client should check the status - // of the `Insert()` by calling `QueryWriteStatus()` and continue writing from - // the returned `committed_size`. This may be less than the amount of data the - // client previously sent. - // - // The service will not view the object as complete until the client has - // sent an `Insert` with `finish_write` set to `true`. Sending any - // requests on a stream after sending a request with `finish_write` set to - // `true` will cause an error. The client **should** check the - // `Object` it receives to determine how much data the service was - // able to commit and whether the service views the object as complete. - rpc InsertObject(stream InsertObjectRequest) returns (Object) { - } - - // Retrieves a list of objects matching the criteria. - rpc ListObjects(ListObjectsRequest) returns (ListObjectsResponse) { - } - - // Rewrites a source object to a destination object. Optionally overrides - // metadata. - rpc RewriteObject(RewriteObjectRequest) returns (RewriteResponse) { - } - - // Starts a resumable write. How long the write operation remains valid, and - // what happens when the write operation becomes invalid, are - // service-dependent. - rpc StartResumableWrite(StartResumableWriteRequest) returns (StartResumableWriteResponse) { - } - - // Determines the `committed_size` for an object that is being written, which - // can then be used as the `write_offset` for the next `Write()` call. - // - // If the object does not exist (i.e., the object has been deleted, or the - // first `Write()` has not yet reached the service), this method returns the - // error `NOT_FOUND`. - // - // The client **may** call `QueryWriteStatus()` at any time to determine how - // much data has been processed for this object. This is useful if the - // client is buffering data and needs to know which data can be safely - // evicted. For any sequence of `QueryWriteStatus()` calls for a given - // object name, the sequence of returned `committed_size` values will be - // non-decreasing. - rpc QueryWriteStatus(QueryWriteStatusRequest) returns (QueryWriteStatusResponse) { - } - - // Updates an object's metadata. - rpc PatchObject(PatchObjectRequest) returns (Object) { - } - - // Updates an object's metadata. Equivalent to PatchObject, but always - // replaces all mutatable fields of the bucket with new values, reverting all - // unspecified fields to their default values. - rpc UpdateObject(UpdateObjectRequest) returns (Object) { - } - - // Gets the IAM policy for the specified object. - rpc GetObjectIamPolicy(google.iam.v1.GetIamPolicyRequest) returns (google.iam.v1.Policy) { - } - - // Updates an IAM policy for the specified object. - rpc SetObjectIamPolicy(google.iam.v1.SetIamPolicyRequest) returns (google.iam.v1.Policy) { - } - - // Tests a set of permissions on the given object to see which, if - // any, are held by the caller. - rpc TestObjectIamPermissions(google.iam.v1.TestIamPermissionsRequest) returns (google.iam.v1.TestIamPermissionsResponse) { - } - - // Watch for changes on all objects in a bucket. - rpc WatchAllObjects(WatchAllObjectsRequest) returns (Channel) { - } - - // Retrieves the name of a project's Google Cloud Storage service account. - rpc GetServiceAccount(GetProjectServiceAccountRequest) returns (ServiceAccount) { - } -} - -// Request message for DeleteBucketAccessControl. -message DeleteBucketAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The entity holding the permission. Can be - // user-userId, - // user-emailAddress, - // group-groupId, - // group-emailAddress, allUsers, or - // allAuthenticatedUsers. - // Required. - string entity = 2; - - // The project to be billed for this request. Required for Requester Pays - // buckets. - string user_project = 3; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 4; -} - -// Request message for GetBucketAccessControl. -message GetBucketAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The entity holding the permission. Can be - // user-userId, - // user-emailAddress, - // group-groupId, - // group-emailAddress, allUsers, or - // allAuthenticatedUsers. - // Required. - string entity = 2; - - // The project to be billed for this request. Required for Requester Pays - // buckets. - string user_project = 3; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 4; -} - -// Request message for InsertBucketAccessControl. -message InsertBucketAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The project to be billed for this request. Required for Requester Pays - // buckets. - string user_project = 2; - - // Properties of the new bucket access control being inserted. - BucketAccessControl bucket_access_control = 3; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 4; -} - -// Request message for ListBucketAccessControl. -message ListBucketAccessControlsRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The project to be billed for this request. Required for Requester Pays - // buckets. - string user_project = 2; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 3; -} - -// Request for PatchBucketAccessControl. -message PatchBucketAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The entity holding the permission. Can be - // user-userId, - // user-emailAddress, - // group-groupId, - // group-emailAddress, allUsers, or - // allAuthenticatedUsers. - // Required. - string entity = 2; - - // The project to be billed for this request. Required for Requester Pays - // buckets. - string user_project = 3; - - // The BucketAccessControl for updating. - BucketAccessControl bucket_access_control = 4; - - // List of fields to be updated. - // - // To specify ALL fields, equivalent to the JSON API's "update" function, - // specify a single field with the value `*`. - // - google.protobuf.FieldMask update_mask = 5; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 6; -} - -// Request for UpdateBucketAccessControl. -message UpdateBucketAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The entity holding the permission. Can be - // user-userId, - // user-emailAddress, - // group-groupId, - // group-emailAddress, allUsers, or - // allAuthenticatedUsers. - // Required. - string entity = 2; - - // The project to be billed for this request. Required for Requester Pays - // buckets. - string user_project = 3; - - // The BucketAccessControl for updating. - BucketAccessControl bucket_access_control = 4; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 5; -} - -// Request message for DeleteBucket. -message DeleteBucketRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // If set, only deletes the bucket if its metageneration matches this value. - google.protobuf.Int64Value if_metageneration_match = 2; - - // If set, only deletes the bucket if its metageneration does not match this - // value. - google.protobuf.Int64Value if_metageneration_not_match = 3; - - // The project to be billed for this request. Required for Requester Pays - // buckets. - string user_project = 4; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 5; -} - -// Request message for GetBucket. -message GetBucketRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // Makes the return of the bucket metadata conditional on whether the bucket's - // current metageneration matches the given value. - google.protobuf.Int64Value if_metageneration_match = 2; - - // Makes the return of the bucket metadata conditional on whether the bucket's - // current metageneration does not match the given value. - google.protobuf.Int64Value if_metageneration_not_match = 3; - - // Set of properties to return. Defaults to NO_ACL. - CommonEnums.Projection projection = 4; - - // The project to be billed for this request. Required for Requester Pays - // buckets. - string user_project = 5; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 6; -} - -// Request message for InsertBucket. -message InsertBucketRequest { - // Apply a predefined set of access controls to this bucket. - CommonEnums.PredefinedBucketAcl predefined_acl = 1; - - // Apply a predefined set of default object access controls to this bucket. - CommonEnums.PredefinedObjectAcl predefined_default_object_acl = 2; - - // A valid API project identifier. - // Required. - string project = 3; - - // Set of properties to return. Defaults to NO_ACL, unless the - // bucket resource specifies acl or defaultObjectAcl - // properties, when it defaults to FULL. - CommonEnums.Projection projection = 4; - - // The project to be billed for this request. - string user_project = 5; - - // Properties of the new bucket being inserted, including its name. - Bucket bucket = 6; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 7; -} - -// Request message for ListBuckets. -message ListBucketsRequest { - // Maximum number of buckets to return in a single response. The service will - // use this parameter or 1,000 items, whichever is smaller. - int32 max_results = 1; - - // A previously-returned page token representing part of the larger set of - // results to view. - string page_token = 2; - - // Filter results to buckets whose names begin with this prefix. - string prefix = 3; - - // A valid API project identifier. - // Required. - string project = 4; - - // Set of properties to return. Defaults to NO_ACL. - CommonEnums.Projection projection = 5; - - // The project to be billed for this request. - string user_project = 6; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 7; -} - -// Request message for LockRetentionPolicy. -message LockRetentionPolicyRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // Makes the operation conditional on whether bucket's current metageneration - // matches the given value. Must be positive. - int64 if_metageneration_match = 2; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 3; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 4; -} - -// Request for PatchBucket method. -message PatchBucketRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // Makes the return of the bucket metadata conditional on whether the bucket's - // current metageneration matches the given value. - google.protobuf.Int64Value if_metageneration_match = 2; - - // Makes the return of the bucket metadata conditional on whether the bucket's - // current metageneration does not match the given value. - google.protobuf.Int64Value if_metageneration_not_match = 3; - - // Apply a predefined set of access controls to this bucket. - CommonEnums.PredefinedBucketAcl predefined_acl = 4; - - // Apply a predefined set of default object access controls to this bucket. - CommonEnums.PredefinedObjectAcl predefined_default_object_acl = 5; - - // Set of properties to return. Defaults to FULL. - CommonEnums.Projection projection = 6; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 7; - - // The Bucket metadata for updating. - Bucket metadata = 8; - - // List of fields to be updated. - // - // To specify ALL fields, equivalent to the JSON API's "update" function, - // specify a single field with the value `*`. Note: not recommended. If a new - // field is introduced at a later time, an older client updating with the `*` - // may accidentally reset the new field's value. - // - // Not specifying any fields is an error. - // Not specifying a field while setting that field to a non-default value is - // an error. - google.protobuf.FieldMask update_mask = 9; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 10; -} - -// Request for UpdateBucket method. -message UpdateBucketRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // Makes the return of the bucket metadata conditional on whether the bucket's - // current metageneration matches the given value. - google.protobuf.Int64Value if_metageneration_match = 2; - - // Makes the return of the bucket metadata conditional on whether the bucket's - // current metageneration does not match the given value. - google.protobuf.Int64Value if_metageneration_not_match = 3; - - // Apply a predefined set of access controls to this bucket. - CommonEnums.PredefinedBucketAcl predefined_acl = 4; - - // Apply a predefined set of default object access controls to this bucket. - CommonEnums.PredefinedObjectAcl predefined_default_object_acl = 5; - - // Set of properties to return. Defaults to FULL. - CommonEnums.Projection projection = 6; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 7; - - // The Bucket metadata for updating. - Bucket metadata = 8; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 9; -} - -// Request message for StopChannel. -message StopChannelRequest { - // The channel to be stopped. - Channel channel = 1; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 2; -} - -// Request message for DeleteDefaultObjectAccessControl. -message DeleteDefaultObjectAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The entity holding the permission. Can be - // user-userId, - // user-emailAddress, - // group-groupId, - // group-emailAddress, allUsers, or - // allAuthenticatedUsers. - // Required. - string entity = 2; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 3; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 4; -} - -// Request message for GetDefaultObjectAccessControl. -message GetDefaultObjectAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The entity holding the permission. Can be - // user-userId, - // user-emailAddress, - // group-groupId, - // group-emailAddress, allUsers, or - // allAuthenticatedUsers. - // Required. - string entity = 2; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 3; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 4; -} - -// Request message for InsertDefaultObjectAccessControl. -message InsertDefaultObjectAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 2; - - // Properties of the object access control being inserted. - ObjectAccessControl object_access_control = 3; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 4; -} - -// Request message for ListDefaultObjectAccessControls. -message ListDefaultObjectAccessControlsRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // If present, only return default ACL listing if the bucket's current - // metageneration matches this value. - google.protobuf.Int64Value if_metageneration_match = 2; - - // If present, only return default ACL listing if the bucket's current - // metageneration does not match the given value. - google.protobuf.Int64Value if_metageneration_not_match = 3; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 4; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 5; -} - -// Request message for PatchDefaultObjectAccessControl. -message PatchDefaultObjectAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The entity holding the permission. Can be - // user-userId, - // user-emailAddress, - // group-groupId, - // group-emailAddress, allUsers, or - // allAuthenticatedUsers. - // Required. - string entity = 2; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 3; - - // The ObjectAccessControl for updating. - ObjectAccessControl object_access_control = 4; - - // List of fields to be updated. - // - // To specify ALL fields, equivalent to the JSON API's "update" function, - // specify a single field with the value `*`. Note: not recommended. If a new - // field is introduced at a later time, an older client updating with the `*` - // may accidentally reset the new field's value. - // - // Not specifying any fields is an error. - // Not specifying a field while setting that field to a non-default value is - // an error. - google.protobuf.FieldMask update_mask = 5; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 6; -} - -// Request message for UpdateDefaultObjectAccessControl. -message UpdateDefaultObjectAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The entity holding the permission. Can be - // user-userId, - // user-emailAddress, - // group-groupId, - // group-emailAddress, allUsers, or - // allAuthenticatedUsers. - // Required. - string entity = 2; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 3; - - // The ObjectAccessControl for updating. - ObjectAccessControl object_access_control = 4; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 5; -} - -// Request message for DeleteNotification. -message DeleteNotificationRequest { - // The parent bucket of the notification. - // Required. - string bucket = 1; - - // ID of the notification to delete. - // Required. - string notification = 2; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 3; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 4; -} - -// Request message for GetNotification. -message GetNotificationRequest { - // The parent bucket of the notification. - // Required. - string bucket = 1; - - // Notification ID. - // Required. - string notification = 2; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 3; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 4; -} - -// Request message for InsertNotification. -message InsertNotificationRequest { - // The parent bucket of the notification. - // Required. - string bucket = 1; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 2; - - // Properties of the notification to be inserted. - Notification notification = 3; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 4; -} - -// Request message for ListNotifications. -message ListNotificationsRequest { - // Name of a Google Cloud Storage bucket. - // Required. - string bucket = 1; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 2; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 3; -} - -// Request message for DeleteObjectAccessControl. -message DeleteObjectAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The entity holding the permission. Can be - // user-userId, - // user-emailAddress, - // group-groupId, - // group-emailAddress, allUsers, or - // allAuthenticatedUsers. - // Required. - string entity = 2; - - // Name of the object. - // Required. - string object = 3; - - // If present, selects a specific revision of this object (as opposed to the - // latest version, the default). - int64 generation = 4; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 5; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 6; -} - -// Request message for GetObjectAccessControl. -message GetObjectAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The entity holding the permission. Can be - // user-userId, - // user-emailAddress, - // group-groupId, - // group-emailAddress, allUsers, or - // allAuthenticatedUsers. - // Required. - string entity = 2; - - // Name of the object. - // Required. - string object = 3; - - // If present, selects a specific revision of this object (as opposed to the - // latest version, the default). - int64 generation = 4; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 5; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 6; -} - -// Request message for InsertObjectAccessControl. -message InsertObjectAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // Name of the object. - // Required. - string object = 2; - - // If present, selects a specific revision of this object (as opposed to the - // latest version, the default). - int64 generation = 3; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 4; - - // Properties of the object access control to be inserted. - ObjectAccessControl object_access_control = 5; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 6; -} - -// Request message for ListObjectAccessControls. -message ListObjectAccessControlsRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // Name of the object. - // Required. - string object = 2; - - // If present, selects a specific revision of this object (as opposed to the - // latest version, the default). - int64 generation = 3; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 4; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 5; -} - -// Request message for UpdateObjetAccessControl. -message UpdateObjectAccessControlRequest { - // Name of a bucket. - // Required. - string bucket = 1; - - // The entity holding the permission. Can be - // user-userId, - // user-emailAddress, - // group-groupId, - // group-emailAddress, allUsers, or - // allAuthenticatedUsers. - // Required. - string entity = 2; - - // Name of the object. - // Required. - string object = 3; - - // If present, selects a specific revision of this object (as opposed to the - // latest version, the default). - int64 generation = 4; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 5; - - // The ObjectAccessControl for updating. - ObjectAccessControl object_access_control = 6; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 7; - - // List of fields to be updated. - // - // To specify ALL fields, equivalent to the JSON API's "update" function, - // specify a single field with the value `*`. Note: not recommended. If a new - // field is introduced at a later time, an older client updating with the `*` - // may accidentally reset the new field's value. - // - // Not specifying any fields is an error. - // Not specifying a field while setting that field to a non-default value is - // an error. - google.protobuf.FieldMask update_mask = 8; -} - -// Request message for ComposeObject. -message ComposeObjectRequest { - // Description of a source object for a composition request. - message SourceObjects { - // Preconditions for a source object of a composition request. - message ObjectPreconditions { - // Only perform the composition if the generation of the source object - // that would be used matches this value. If this value and a generation - // are both specified, they must be the same value or the call will fail. - google.protobuf.Int64Value if_generation_match = 1; - } - - // The source object's name. All source objects must reside in the same - // bucket. - string name = 1; - - // The generation of this object to use as the source. - int64 generation = 2; - - // Conditions that must be met for this operation to execute. - ObjectPreconditions object_preconditions = 3; - } - - // Name of the bucket containing the source objects. The destination object is - // stored in this bucket. - // Required. - string destination_bucket = 1; - - // Name of the new object. - // Required. - string destination_object = 2; - - // Apply a predefined set of access controls to the destination object. - CommonEnums.PredefinedObjectAcl destination_predefined_acl = 3; - - // Properties of the resulting object. - Object destination = 11; - - // The list of source objects that will be concatenated into a single object. - repeated SourceObjects source_objects = 12; - - // Makes the operation conditional on whether the object's current generation - // matches the given value. Setting to 0 makes the operation succeed only if - // there are no live versions of the object. - google.protobuf.Int64Value if_generation_match = 4; - - // Makes the operation conditional on whether the object's current - // metageneration matches the given value. - google.protobuf.Int64Value if_metageneration_match = 5; - - // Resource name of the Cloud KMS key, of the form - // projects/my-project/locations/my-location/keyRings/my-kr/cryptoKeys/my-key, - // that will be used to encrypt the object. Overrides the object - // metadata's kms_key_name value, if any. - string kms_key_name = 6; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 7; - - // A set of parameters common to Storage API requests concerning an object. - CommonObjectRequestParams common_object_request_params = 9; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 10; -} - -// Request message for CopyObject. -message CopyObjectRequest { - // Name of the bucket in which to store the new object. Overrides the provided - // object - // metadata's bucket value, if any. - // Required. - string destination_bucket = 1; - - // Name of the new object. - // Required when the object metadata is not otherwise provided. Overrides the - // object metadata's name value, if any. - // Required. - string destination_object = 2; - - // Apply a predefined set of access controls to the destination object. - CommonEnums.PredefinedObjectAcl destination_predefined_acl = 3; - - // Makes the operation conditional on whether the destination object's current - // generation matches the given value. Setting to 0 makes the operation - // succeed only if there are no live versions of the object. - google.protobuf.Int64Value if_generation_match = 4; - - // Makes the operation conditional on whether the destination object's current - // generation does not match the given value. If no live object exists, the - // precondition fails. Setting to 0 makes the operation succeed only if there - // is a live version of the object. - google.protobuf.Int64Value if_generation_not_match = 5; - - // Makes the operation conditional on whether the destination object's current - // metageneration matches the given value. - google.protobuf.Int64Value if_metageneration_match = 6; - - // Makes the operation conditional on whether the destination object's current - // metageneration does not match the given value. - google.protobuf.Int64Value if_metageneration_not_match = 7; - - // Makes the operation conditional on whether the source object's current - // generation matches the given value. - google.protobuf.Int64Value if_source_generation_match = 8; - - // Makes the operation conditional on whether the source object's current - // generation does not match the given value. - google.protobuf.Int64Value if_source_generation_not_match = 9; - - // Makes the operation conditional on whether the source object's current - // metageneration matches the given value. - google.protobuf.Int64Value if_source_metageneration_match = 10; - - // Makes the operation conditional on whether the source object's current - // metageneration does not match the given value. - google.protobuf.Int64Value if_source_metageneration_not_match = 11; - - // Set of properties to return. Defaults to NO_ACL, unless the - // object resource specifies the acl property, when it defaults - // to full. - CommonEnums.Projection projection = 12; - - // Name of the bucket in which to find the source object. - // Required. - string source_bucket = 13; - - // Name of the source object. - // Required. - string source_object = 14; - - // If present, selects a specific revision of the source object (as opposed to - // the latest version, the default). - int64 source_generation = 15; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 16; - - // Properties of the resulting object. If not set, duplicate properties of - // source object. - Object destination = 17; - - // Resource name of the Cloud KMS key, of the form - // projects/my-project/locations/my-location/keyRings/my-kr/cryptoKeys/my-key, - // that will be used to encrypt the object. Overrides the object - // metadata's kms_key_name value, if any. - string destination_kms_key_name = 20; - - // A set of parameters common to Storage API requests concerning an object. - CommonObjectRequestParams common_object_request_params = 18; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 19; -} - -// Message for deleting an object. -// Either `bucket` and `object` *or* `upload_id` **must** be set (but not both). -message DeleteObjectRequest { - // Name of the bucket in which the object resides. - // Required. - string bucket = 1; - - // The name of the object to delete (when not using a resumable write). - // Required. - string object = 2; - - // The resumable upload_id of the object to delete (when using a - // resumable write). This should be copied from the `upload_id` field of - // `StartResumableWriteResponse`. - string upload_id = 3; - - // If present, permanently deletes a specific revision of this object (as - // opposed to the latest version, the default). - int64 generation = 4; - - // Makes the operation conditional on whether the object's current generation - // matches the given value. Setting to 0 makes the operation succeed only if - // there are no live versions of the object. - google.protobuf.Int64Value if_generation_match = 5; - - // Makes the operation conditional on whether the object's current generation - // does not match the given value. If no live object exists, the precondition - // fails. Setting to 0 makes the operation succeed only if there is a live - // version of the object. - google.protobuf.Int64Value if_generation_not_match = 6; - - // Makes the operation conditional on whether the object's current - // metageneration matches the given value. - google.protobuf.Int64Value if_metageneration_match = 7; - - // Makes the operation conditional on whether the object's current - // metageneration does not match the given value. - google.protobuf.Int64Value if_metageneration_not_match = 8; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 9; - - // A set of parameters common to Storage API requests concerning an object. - CommonObjectRequestParams common_object_request_params = 10; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 11; -} - -// Request message for GetObjectMedia. -message GetObjectMediaRequest { - // The name of the bucket containing the object to read. - string bucket = 1; - - // The name of the object to read. - string object = 2; - - // If present, selects a specific revision of this object (as opposed - // to the latest version, the default). - int64 generation = 3; - - // The offset for the first byte to return in the read, relative to the start - // of the object. - // - // A `read_offset` that is negative or greater than the size of the object - // will cause an `OUT_OF_RANGE` error. - int64 read_offset = 4; - - // The maximum number of `data` bytes the server is allowed to return in the - // sum of all `Object` messages. A `read_limit` of zero indicates that there - // is no limit, and a negative `read_limit` will cause an error. - // - // If the stream returns fewer bytes than allowed by the `read_limit` and no - // error occurred, the stream includes all data from the `read_offset` to the - // end of the resource. - int64 read_limit = 5; - - // Makes the operation conditional on whether the object's current generation - // matches the given value. Setting to 0 makes the operation succeed only if - // there are no live versions of the object. - google.protobuf.Int64Value if_generation_match = 6; - - // Makes the operation conditional on whether the object's current generation - // does not match the given value. If no live object exists, the precondition - // fails. Setting to 0 makes the operation succeed only if there is a live - // version of the object. - google.protobuf.Int64Value if_generation_not_match = 7; - - // Makes the operation conditional on whether the object's current - // metageneration matches the given value. - google.protobuf.Int64Value if_metageneration_match = 8; - - // Makes the operation conditional on whether the object's current - // metageneration does not match the given value. - google.protobuf.Int64Value if_metageneration_not_match = 9; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 10; - - // A set of parameters common to Storage API requests concerning an object. - CommonObjectRequestParams common_object_request_params = 11; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 12; -} - -// Request message for GetObject. -message GetObjectRequest { - // Name of the bucket in which the object resides. - // Required. - string bucket = 1; - - // Name of the object. - // Required. - string object = 2; - - // If present, selects a specific revision of this object (as opposed to the - // latest version, the default). - int64 generation = 3; - - // Makes the operation conditional on whether the object's current generation - // matches the given value. Setting to 0 makes the operation succeed only if - // there are no live versions of the object. - google.protobuf.Int64Value if_generation_match = 4; - - // Makes the operation conditional on whether the object's current generation - // does not match the given value. If no live object exists, the precondition - // fails. Setting to 0 makes the operation succeed only if there is a live - // version of the object. - google.protobuf.Int64Value if_generation_not_match = 5; - - // Makes the operation conditional on whether the object's current - // metageneration matches the given value. - google.protobuf.Int64Value if_metageneration_match = 6; - - // Makes the operation conditional on whether the object's current - // metageneration does not match the given value. - google.protobuf.Int64Value if_metageneration_not_match = 7; - - // Set of properties to return. Defaults to NO_ACL. - CommonEnums.Projection projection = 8; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 9; - - // A set of parameters common to Storage API requests concerning an object. - CommonObjectRequestParams common_object_request_params = 10; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 11; -} - -// Response message for GetObject. -message GetObjectMediaResponse { - // A portion of the data for the object. The service **may** leave `data` - // empty for any given `ReadResponse`. This enables the service to inform the - // client that the request is still live while it is running an operation to - // generate more data. - ChecksummedData checksummed_data = 1; - - // The checksums of the complete object. The client should compute one of - // these checksums over the downloaded object and compare it against the value - // provided here. - ObjectChecksums object_checksums = 2; - - // If read_offset and or read_limit was specified on the - // GetObjectMediaRequest, ContentRange will be populated on the first - // GetObjectMediaResponse message of the read stream. - ContentRange content_range = 3; -} - -// Describes an attempt to insert an object, possibly over multiple requests. -message InsertObjectSpec { - // Destination object, including its name and its metadata. - Object resource = 1; - - // Apply a predefined set of access controls to this object. - CommonEnums.PredefinedObjectAcl predefined_acl = 2; - - // Makes the operation conditional on whether the object's current - // generation matches the given value. Setting to 0 makes the operation - // succeed only if there are no live versions of the object. - google.protobuf.Int64Value if_generation_match = 3; - - // Makes the operation conditional on whether the object's current - // generation does not match the given value. If no live object exists, the - // precondition fails. Setting to 0 makes the operation succeed only if - // there is a live version of the object. - google.protobuf.Int64Value if_generation_not_match = 4; - - // Makes the operation conditional on whether the object's current - // metageneration matches the given value. - google.protobuf.Int64Value if_metageneration_match = 5; - - // Makes the operation conditional on whether the object's current - // metageneration does not match the given value. - google.protobuf.Int64Value if_metageneration_not_match = 6; - - // Set of properties to return. Defaults to NO_ACL, unless the - // object resource specifies the acl property, when it defaults - // to full. - CommonEnums.Projection projection = 7; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 8; -} - -// Message for writing an object. -message InsertObjectRequest { - // The first message of each stream should set one of the following. - oneof first_message { - // For resumable uploads. This should be the `upload_id` returned from a - // call to `StartResumableWriteResponse`. - string upload_id = 1; - - // For non-resumable uploads. Describes the overall upload, including the - // destination bucket and object name, preconditions, etc. - InsertObjectSpec insert_object_spec = 2; - } - - // The offset from the beginning of the object at which the data should be - // written. - // Required. - // - // In the first `InsertObjectRequest` of a `InsertObject()` action, it - // indicates the initial offset for the `Insert()` call. The value **must** be - // equal to the `committed_size` that a call to `QueryWriteStatus()` would - // return (0 if this is the first write to the object). - // - // On subsequent calls, this value **must** be no larger than the sum of the - // first `write_offset` and the sizes of all `data` chunks sent previously on - // this stream. - // - // An incorrect value will cause an error. - int64 write_offset = 3; - - // A portion of the data for the object. - oneof data { - // The data to insert. If a crc32c checksum is provided that doesn't match - // the checksum computed by the service, the request will fail. - ChecksummedData checksummed_data = 4; - - // A reference to an existing object. This can be used to support - // several use cases: - // - Writing a sequence of data buffers supports the basic use case of - // uploading a complete object, chunk by chunk. - // - Writing a sequence of references to existing objects allows an - // object to be composed from a collection of objects, which can be - // used to support parallel object writes. - // - Writing a single reference with a given offset and size can be used - // to create an object from a slice of an existing object. - // - Writing an object referencing a object slice (created as noted - // above) followed by a data buffer followed by another object - // slice can be used to support delta upload functionality. - GetObjectMediaRequest reference = 5; - } - - // Checksums for the complete object. If the checksums computed by the service - // don't match the specifified checksums the call will fail. May only be - // provided in the first or last request (either with first_message, or - // finish_write set). - ObjectChecksums object_checksums = 6; - - // If `true`, this indicates that the write is complete. Sending any - // `InsertObjectRequest`s subsequent to one in which `finish_write` is `true` - // will cause an error. - // For a non-resumable write (where the upload_id was not set in the first - // message), it is an error not to set this field in the final message of the - // stream. - bool finish_write = 7; - - // A set of parameters common to Storage API requests concerning an object. - CommonObjectRequestParams common_object_request_params = 8; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 9; -} - -// Request message for ListObjects. -message ListObjectsRequest { - // Name of the bucket in which to look for objects. - // Required. - string bucket = 1; - - // Returns results in a directory-like mode. items will contain - // only objects whose names, aside from the prefix, do not - // contain delimiter. Objects whose names, aside from the - // prefix, contain delimiter will have their name, - // truncated after the delimiter, returned in - // prefixes. Duplicate prefixes are omitted. - string delimiter = 2; - - // If true, objects that end in exactly one instance of delimiter - // will have their metadata included in items in addition to - // prefixes. - bool include_trailing_delimiter = 3; - - // Maximum number of items plus prefixes to return - // in a single page of responses. As duplicate prefixes are - // omitted, fewer total results may be returned than requested. The service - // will use this parameter or 1,000 items, whichever is smaller. - int32 max_results = 4; - - // A previously-returned page token representing part of the larger set of - // results to view. - string page_token = 5; - - // Filter results to objects whose names begin with this prefix. - string prefix = 6; - - // Set of properties to return. Defaults to NO_ACL. - CommonEnums.Projection projection = 7; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 8; - - // If true, lists all versions of an object as distinct results. - // The default is false. For more information, see Object Versioning. - bool versions = 9; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 10; -} - -// Request object for `ByteStream.QueryWriteStatus`. -message QueryWriteStatusRequest { - // The name of the resume token for the object whose write status is being - // requested. - // Required. - string upload_id = 1; - - // A set of parameters common to Storage API requests concerning an object. - CommonObjectRequestParams common_object_request_params = 2; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 3; -} - -// Response object for `ByteStream.QueryWriteStatus`. -message QueryWriteStatusResponse { - // The number of bytes that have been processed for the given object. - int64 committed_size = 1; - - // `complete` is `true` only if the client has sent a `InsertObjectRequest` - // with `finish_write` set to true, and the server has processed that request. - bool complete = 2; -} - -// Request message for RewriteObject. -message RewriteObjectRequest { - // Name of the bucket in which to store the new object. Overrides the provided - // object metadata's bucket value, if any. - // Required. - string destination_bucket = 1; - - // Name of the new object. - // Required when the object metadata is not otherwise provided. Overrides the - // object metadata's name value, if any. - // Required. - string destination_object = 2; - - // Resource name of the Cloud KMS key, of the form - // projects/my-project/locations/my-location/keyRings/my-kr/cryptoKeys/my-key, - // that will be used to encrypt the object. Overrides the object - // metadata's kms_key_name value, if any. - string destination_kms_key_name = 3; - - // Apply a predefined set of access controls to the destination object. - CommonEnums.PredefinedObjectAcl destination_predefined_acl = 4; - - // Makes the operation conditional on whether the object's current generation - // matches the given value. Setting to 0 makes the operation succeed only if - // there are no live versions of the object. - google.protobuf.Int64Value if_generation_match = 5; - - // Makes the operation conditional on whether the object's current generation - // does not match the given value. If no live object exists, the precondition - // fails. Setting to 0 makes the operation succeed only if there is a live - // version of the object. - google.protobuf.Int64Value if_generation_not_match = 6; - - // Makes the operation conditional on whether the destination object's current - // metageneration matches the given value. - google.protobuf.Int64Value if_metageneration_match = 7; - - // Makes the operation conditional on whether the destination object's current - // metageneration does not match the given value. - google.protobuf.Int64Value if_metageneration_not_match = 8; - - // Makes the operation conditional on whether the source object's current - // generation matches the given value. - google.protobuf.Int64Value if_source_generation_match = 9; - - // Makes the operation conditional on whether the source object's current - // generation does not match the given value. - google.protobuf.Int64Value if_source_generation_not_match = 10; - - // Makes the operation conditional on whether the source object's current - // metageneration matches the given value. - google.protobuf.Int64Value if_source_metageneration_match = 11; - - // Makes the operation conditional on whether the source object's current - // metageneration does not match the given value. - google.protobuf.Int64Value if_source_metageneration_not_match = 12; - - // The maximum number of bytes that will be rewritten per rewrite request. - // Most callers - // shouldn't need to specify this parameter - it is primarily in place to - // support testing. If specified the value must be an integral multiple of - // 1 MiB (1048576). Also, this only applies to requests where the source and - // destination span locations and/or storage classes. Finally, this value must - // not change across rewrite calls else you'll get an error that the - // rewriteToken is invalid. - int64 max_bytes_rewritten_per_call = 13; - - // Set of properties to return. Defaults to NO_ACL, unless the - // object resource specifies the acl property, when it defaults - // to full. - CommonEnums.Projection projection = 14; - - // Include this field (from the previous rewrite response) on each rewrite - // request after the first one, until the rewrite response 'done' flag is - // true. Calls that provide a rewriteToken can omit all other request fields, - // but if included those fields must match the values provided in the first - // rewrite request. - string rewrite_token = 15; - - // Name of the bucket in which to find the source object. - // Required. - string source_bucket = 16; - - // Name of the source object. - // Required. - string source_object = 17; - - // If present, selects a specific revision of the source object (as opposed to - // the latest version, the default). - int64 source_generation = 18; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 19; - - // Properties of the destination, post-rewrite object. - Object object = 20; - - // The algorithm used to encrypt the source object, if any. - string copy_source_encryption_algorithm = 21; - - // The encryption key used to encrypt the source object, if any. - string copy_source_encryption_key = 22; - - // The SHA-256 hash of the key used to encrypt the source object, if any. - string copy_source_encryption_key_sha256 = 23; - - // A set of parameters common to Storage API requests concerning an object. - CommonObjectRequestParams common_object_request_params = 24; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 25; -} - -// A rewrite response. -message RewriteResponse { - // The total bytes written so far, which can be used to provide a waiting user - // with a progress indicator. This property is always present in the response. - int64 total_bytes_rewritten = 1; - - // The total size of the object being copied in bytes. This property is always - // present in the response. - int64 object_size = 2; - - // true if the copy is finished; otherwise, false if - // the copy is in progress. This property is always present in the response. - bool done = 3; - - // A token to use in subsequent requests to continue copying data. This token - // is present in the response only when there is more data to copy. - string rewrite_token = 4; - - // A resource containing the metadata for the copied-to object. This property - // is present in the response only when copying completes. - Object resource = 5; -} - -// Request message StartResumableWrite. -message StartResumableWriteRequest { - // The destination bucket, object, and metadata, as well as any preconditions. - InsertObjectSpec insert_object_spec = 1; - - // A set of parameters common to Storage API requests concerning an object. - CommonObjectRequestParams common_object_request_params = 3; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 4; -} - -// Response object for ByteStream.StartResumableWrite. -message StartResumableWriteResponse { - // The upload_id of the newly started resumable write operation. This - // value should be copied into the `InsertObjectRequest.upload_id` field. - string upload_id = 1; -} - -// Request message for PatchObject. -message PatchObjectRequest { - // Name of the bucket in which the object resides. - // Required. - string bucket = 1; - - // Name of the object. - // Required. - string object = 2; - - // If present, selects a specific revision of this object (as opposed to the - // latest version, the default). - int64 generation = 3; - - // Makes the operation conditional on whether the object's current generation - // matches the given value. Setting to 0 makes the operation succeed only if - // there are no live versions of the object. - google.protobuf.Int64Value if_generation_match = 4; - - // Makes the operation conditional on whether the object's current generation - // does not match the given value. If no live object exists, the precondition - // fails. Setting to 0 makes the operation succeed only if there is a live - // version of the object. - google.protobuf.Int64Value if_generation_not_match = 5; - - // Makes the operation conditional on whether the object's current - // metageneration matches the given value. - google.protobuf.Int64Value if_metageneration_match = 6; - - // Makes the operation conditional on whether the object's current - // metageneration does not match the given value. - google.protobuf.Int64Value if_metageneration_not_match = 7; - - // Apply a predefined set of access controls to this object. - CommonEnums.PredefinedObjectAcl predefined_acl = 8; - - // Set of properties to return. Defaults to FULL. - CommonEnums.Projection projection = 9; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 10; - - // The Object metadata for updating. - Object metadata = 11; - - // List of fields to be updated. - // - // To specify ALL fields, equivalent to the JSON API's "update" function, - // specify a single field with the value `*`. Note: not recommended. If a new - // field is introduced at a later time, an older client updating with the `*` - // may accidentally reset the new field's value. - // - // Not specifying any fields is an error. - // Not specifying a field while setting that field to a non-default value is - // an error. - google.protobuf.FieldMask update_mask = 12; - - // A set of parameters common to Storage API requests concerning an object. - CommonObjectRequestParams common_object_request_params = 13; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 14; -} - -// Request message for UpdateObject. -message UpdateObjectRequest { - // Name of the bucket in which the object resides. - // Required. - string bucket = 1; - - // Name of the object. - // Required. - string object = 2; - - // If present, selects a specific revision of this object (as opposed to the - // latest version, the default). - int64 generation = 3; - - // Makes the operation conditional on whether the object's current generation - // matches the given value. Setting to 0 makes the operation succeed only if - // there are no live versions of the object. - google.protobuf.Int64Value if_generation_match = 4; - - // Makes the operation conditional on whether the object's current generation - // does not match the given value. If no live object exists, the precondition - // fails. Setting to 0 makes the operation succeed only if there is a live - // version of the object. - google.protobuf.Int64Value if_generation_not_match = 5; - - // Makes the operation conditional on whether the object's current - // metageneration matches the given value. - google.protobuf.Int64Value if_metageneration_match = 6; - - // Makes the operation conditional on whether the object's current - // metageneration does not match the given value. - google.protobuf.Int64Value if_metageneration_not_match = 7; - - // Apply a predefined set of access controls to this object. - CommonEnums.PredefinedObjectAcl predefined_acl = 8; - - // Set of properties to return. Defaults to FULL. - CommonEnums.Projection projection = 9; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 10; - - // The Object metadata for updating. - Object metadata = 11; - - // A set of parameters common to Storage API requests concerning an object. - CommonObjectRequestParams common_object_request_params = 12; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 13; -} - -// Request message for WatchAllObjects. -message WatchAllObjectsRequest { - // Name of the bucket in which to look for objects. - string bucket = 1; - - // If true, lists all versions of an object as distinct results. - // The default is false. For more information, see Object Versioning. - bool versions = 2; - - // Returns results in a directory-like mode. items will contain - // only objects whose names, aside from the prefix, do not - // contain delimiter. Objects whose names, aside from the - // prefix, contain delimiter will have their name, - // truncated after the delimiter, returned in - // prefixes. Duplicate prefixes are omitted. - string delimiter = 3; - - // Maximum number of items plus prefixes to return - // in a single page of responses. As duplicate prefixes are - // omitted, fewer total results may be returned than requested. The service - // will use this parameter or 1,000 items, whichever is smaller. - int32 max_results = 4; - - // Filter results to objects whose names begin with this prefix. - string prefix = 5; - - // If true, objects that end in exactly one instance of delimiter - // will have their metadata included in items in addition to - // prefixes. - bool include_trailing_delimiter = 6; - - // A previously-returned page token representing part of the larger set of - // results to view. - string page_token = 7; - - // Set of properties to return. Defaults to NO_ACL. - CommonEnums.Projection projection = 8; - - // The project to be billed for this request. - // Required for Requester Pays buckets. - string user_project = 9; - - // Properties of the channel to be inserted. - Channel channel = 10; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 11; -} - -// Request message for GetProjectServiceAccount. -message GetProjectServiceAccountRequest { - // Project ID. - // Required. - string project_id = 1; - - // The project to be billed for this request. - string user_project = 2; - - // A set of parameters common to all Storage API requests. - CommonRequestParams common_request_params = 3; -} - -// Parameters that can be passed to any object request. -message CommonObjectRequestParams { - // Encryption algorithm used with Customer-Supplied Encryption Keys feature. - string encryption_algorithm = 1; - - // Encryption key used with Customer-Supplied Encryption Keys feature. - string encryption_key = 2; - - // SHA256 hash of encryption key used with Customer-Supplied Encryption Keys - // feature. - string encryption_key_sha256 = 3; -} - -// Parameters that can be passed to any request. -message CommonRequestParams { - // Required when using buckets with Requestor Pays feature enabled. - string user_project = 1; - - // Lets you enforce per-user quotas from a server-side application even in - // cases when the user's IP address is unknown. This can occur, for example, - // with applications that run cron jobs on App Engine on a user's behalf. - // You can choose any arbitrary string that uniquely identifies a user, but it - // is limited to 40 characters. - // Overrides user_ip if both are provided. - string quota_user = 2; - - // IP address of the end user for whom the API call is being made. - // Lets you enforce per-user quotas when calling the API from a server-side - // application. - string user_ip = 3; - - // Subset of fields to include in the response. - google.protobuf.FieldMask fields = 4; -} - -// Shared constants. -message ServiceConstants { - // A collection of constant values meaningful to the Storage API. - enum Values { - option allow_alias = true; - - // Unused. Proto3 requires first enum to be 0. - SIZE_UNSPECIFIED = 0; - - // The maximum size chunk that can will be returned in a single - // ReadRequest. - // 2 MiB. - MAX_READ_CHUNK_BYTES = 2097152; - - // The maximum size chunk that can be sent in a single InsertObjectRequest. - // 2 MiB. - MAX_WRITE_CHUNK_BYTES = 2097152; - - // The maximum size of an object in MB - whether written in a single stream - // or composed from multiple other objects. - // 5 TiB. - MAX_OBJECT_SIZE_MB = 5242880; - - // The maximum length field name that can be sent in a single - // custom metadata field. - // 1 KiB. - MAX_CUSTOM_METADATA_FIELD_NAME_BYTES = 1024; - - // The maximum length field value that can be sent in a single - // custom_metadata field. - // 4 KiB. - MAX_CUSTOM_METADATA_FIELD_VALUE_BYTES = 4096; - - // The maximum total bytes that can be populated into all field names and - // values of the custom_metadata for one object. - // 8 KiB. - MAX_CUSTOM_METADATA_TOTAL_SIZE_BYTES = 8192; - - // The maximum total bytes that can be populated into all bucket metadata - // fields. - // 20 KiB. - MAX_BUCKET_METADATA_TOTAL_SIZE_BYTES = 20480; - - // The maximum number of NotificationConfigurations that can be registered - // for a given bucket. - MAX_NOTIFICATION_CONFIGS_PER_BUCKET = 100; - - // The maximum number of LifecycleRules that can be registered for a given - // bucket. - MAX_LIFECYCLE_RULES_PER_BUCKET = 100; - - // The maximum number of custom attributes per NotificationConfig. - MAX_NOTIFICATION_CUSTOM_ATTRIBUTES = 5; - - // The maximum length of a custom attribute key included in - // NotificationConfig. - MAX_NOTIFICATION_CUSTOM_ATTRIBUTE_KEY_LENGTH = 256; - - // The maximum length of a custom attribute value included in a - // NotificationConfig. - MAX_NOTIFICATION_CUSTOM_ATTRIBUTE_VALUE_LENGTH = 1024; - - // The maximum number of key/value entries per bucket label. - MAX_LABELS_ENTRIES_COUNT = 64; - - // The maximum character length of the key or value in a bucket - // label map. - MAX_LABELS_KEY_VALUE_LENGTH = 63; - - // The maximum byte size of the key or value in a bucket label - // map. - MAX_LABELS_KEY_VALUE_BYTES = 128; - - // The maximum number of object IDs that can be included in a - // DeleteObjectsRequest. - MAX_OBJECT_IDS_PER_DELETE_OBJECTS_REQUEST = 1000; - - // The maximum number of days for which a token returned by the - // GetListObjectsSplitPoints RPC is valid. - SPLIT_TOKEN_MAX_VALID_DAYS = 14; - } - - -} diff --git a/gcsio/src/main/proto/google/storage/v1/storage_resources.proto b/gcsio/src/main/proto/google/storage/v1/storage_resources.proto deleted file mode 100644 index 77a01785af..0000000000 --- a/gcsio/src/main/proto/google/storage/v1/storage_resources.proto +++ /dev/null @@ -1,793 +0,0 @@ -// Copyright 2019 Google LLC. -// -// Licensed 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. -// - -syntax = "proto3"; - -package google.storage.v1; - -import "google/protobuf/timestamp.proto"; -import "google/protobuf/wrappers.proto"; - -option go_package = "google.golang.org/genproto/googleapis/storage/v1;storage"; -option java_multiple_files = true; -option java_outer_classname = "CloudStorageResourcesProto"; -option java_package = "com.google.google.storage.v1"; - -// A bucket. -message Bucket { - // Billing properties of a bucket. - message Billing { - // When set to true, Requester Pays is enabled for this bucket. - bool requester_pays = 1; - } - - // Cross-Origin Response sharing (CORS) properties for a bucket. - // For more on GCS and CORS, see - // https://cloud.google.com/storage/docs/cross-origin. - // For more on CORS in general, see https://tools.ietf.org/html/rfc6454. - message Cors { - // The list of Origins eligible to receive CORS response headers. See - // [https://tools.ietf.org/html/rfc6454][RFC 6454] for more on origins. - // Note: "*" is permitted in the list of origins, and means "any Origin". - repeated string origin = 1; - - // The list of HTTP methods on which to include CORS response headers, - // (`GET`, `OPTIONS`, `POST`, etc) Note: "*" is permitted in the list of - // methods, and means "any method". - repeated string method = 2; - - // The list of HTTP headers other than the - // [https://www.w3.org/TR/cors/#simple-response-header][simple response - // headers] to give permission for the user-agent to share across domains. - repeated string response_header = 3; - - // The value, in seconds, to return in the - // [https://www.w3.org/TR/cors/#access-control-max-age-response-header][Access-Control-Max-Age - // header] used in preflight responses. - int32 max_age_seconds = 4; - } - - // Encryption properties of a bucket. - message Encryption { - // A Cloud KMS key that will be used to encrypt objects inserted into this - // bucket, if no encryption method is specified. - string default_kms_key_name = 1; - } - - // Lifecycle properties of a bucket. - // For more information, see https://cloud.google.com/storage/docs/lifecycle. - message Lifecycle { - // A lifecycle Rule, combining an action to take on an object and a - // condition which will trigger that action. - message Rule { - // An action to take on an object. - message Action { - // Type of the action. Currently, only `Delete` and - // `SetStorageClass` are supported. - string type = 1; - - // Target storage class. Required iff the type of the action is - // SetStorageClass. - string storage_class = 2; - } - - // A condition of an object which triggers some action. - message Condition { - // Age of an object (in days). This condition is satisfied when an - // object reaches the specified age. - int32 age = 1; - - // A date in [RFC 3339][1] format with only the date part (for - // instance, "2013-01-15"). This condition is satisfied when an - // object is created before midnight of the specified date in UTC. - // [1]: https://tools.ietf.org/html/rfc3339 - google.protobuf.Timestamp created_before = 2; - - // Relevant only for versioned objects. If the value is - // `true`, this condition matches live objects; if the value - // is `false`, it matches archived objects. - google.protobuf.BoolValue is_live = 3; - - // Relevant only for versioned objects. If the value is N, this - // condition is satisfied when there are at least N versions (including - // the live version) newer than this version of the object. - int32 num_newer_versions = 4; - - // Objects having any of the storage classes specified by this condition - // will be matched. Values include `MULTI_REGIONAL`, `REGIONAL`, - // `NEARLINE`, `COLDLINE`, `STANDARD`, and - // `DURABLE_REDUCED_AVAILABILITY`. - repeated string matches_storage_class = 5; - - // A regular expression that satisfies the RE2 syntax. This condition is - // satisfied when the name of the object matches the RE2 pattern. Note: - // This feature is currently in the "Early Access" launch stage and is - // only available to a whitelisted set of users; that means that this - // feature may be changed in backward-incompatible ways and that it is - // not guaranteed to be released. - string matches_pattern = 6; - } - - // The action to take. - Action action = 1; - - // The condition(s) under which the action will be taken. - Condition condition = 2; - } - - // A lifecycle management rule, which is made of an action to take and the - // condition(s) under which the action will be taken. - repeated Rule rule = 1; - } - - // Logging-related properties of a bucket. - message Logging { - // The destination bucket where the current bucket's logs should be placed. - string log_bucket = 1; - - // A prefix for log object names. - string log_object_prefix = 2; - } - - // Retention policy properties of a bucket. - message RetentionPolicy { - // Server-determined value that indicates the time from which policy was - // enforced and effective. This value is in - // [https://tools.ietf.org/html/rfc3339][RFC 3339] format. - google.protobuf.Timestamp effective_time = 1; - - // Once locked, an object retention policy cannot be modified. - bool is_locked = 2; - - // The duration in seconds that objects need to be retained. Retention - // duration must be greater than zero and less than 100 years. Note that - // enforcement of retention periods less than a day is not guaranteed. Such - // periods should only be used for testing purposes. - int64 retention_period = 3; - } - - // Properties of a bucket related to versioning. - // For more on GCS versioning, see - // https://cloud.google.com/storage/docs/object-versioning. - message Versioning { - // While set to true, versioning is fully enabled for this bucket. - bool enabled = 1; - } - - // Properties of a bucket related to accessing the contents as a static - // website. For more on hosting a static website via GCS, see - // https://cloud.google.com/storage/docs/hosting-static-website. - message Website { - // If the requested object path is missing, the service will ensure the path - // has a trailing '/', append this suffix, and attempt to retrieve the - // resulting object. This allows the creation of `index.html` - // objects to represent directory pages. - string main_page_suffix = 1; - - // If the requested object path is missing, and any - // `mainPageSuffix` object is missing, if applicable, the service - // will return the named object from this bucket as the content for a - // [https://tools.ietf.org/html/rfc7231#section-6.5.4][404 Not Found] - // result. - string not_found_page = 2; - } - - // Access controls on the bucket. - repeated BucketAccessControl acl = 1; - - // Default access controls to apply to new objects when no ACL is provided. - repeated ObjectAccessControl default_object_acl = 2; - - // The bucket's lifecycle configuration. See - // [https://developers.google.com/storage/docs/lifecycle]Lifecycle Management] - // for more information. - Lifecycle lifecycle = 3; - - // The creation time of the bucket in - // [https://tools.ietf.org/html/rfc3339][RFC 3339] format. - // Attempting to set this field will result in an error. - google.protobuf.Timestamp time_created = 4; - - // The ID of the bucket. For buckets, the `id` and `name` properties are the - // same. - // Attempting to update this field after the bucket is created will result in - // an error. - string id = 5; - - // The name of the bucket. - // Attempting to update this field after the bucket is created will result in - // an error. - string name = 6; - - // The project number of the project the bucket belongs to. - // Attempting to set this field will result in an error. - int64 project_number = 7; - - // The metadata generation of this bucket. - // Attempting to set this field will result in an error. - int64 metageneration = 8; - - // The bucket's [https://www.w3.org/TR/cors/][Cross-Origin Resource Sharing] - // (CORS) configuration. - repeated Cors cors = 9; - - // The location of the bucket. Object data for objects in the bucket resides - // in physical storage within this region. Defaults to `US`. See the - // [https://developers.google.com/storage/docs/concepts-techniques#specifyinglocations"][developer's - // guide] for the authoritative list. Attempting to update this field after - // the bucket is created will result in an error. - string location = 10; - - // The bucket's default storage class, used whenever no storageClass is - // specified for a newly-created object. This defines how objects in the - // bucket are stored and determines the SLA and the cost of storage. - // If this value is not specified when the bucket is created, it will default - // to `STANDARD`. For more information, see - // https://developers.google.com/storage/docs/storage-classes. - string storage_class = 11; - - // HTTP 1.1 [https://tools.ietf.org/html/rfc7232#section-2.3"]Entity tag] - // for the bucket. - // Attempting to set this field will result in an error. - string etag = 12; - - // The modification time of the bucket. - // Attempting to set this field will result in an error. - google.protobuf.Timestamp updated = 13; - - // The default value for event-based hold on newly created objects in this - // bucket. Event-based hold is a way to retain objects indefinitely until an - // event occurs, signified by the - // hold's release. After being released, such objects will be subject to - // bucket-level retention (if any). One sample use case of this flag is for - // banks to hold loan documents for at least 3 years after loan is paid in - // full. Here, bucket-level retention is 3 years and the event is loan being - // paid in full. In this example, these objects will be held intact for any - // number of years until the event has occurred (event-based hold on the - // object is released) and then 3 more years after that. That means retention - // duration of the objects begins from the moment event-based hold - // transitioned from true to false. Objects under event-based hold cannot be - // deleted, overwritten or archived until the hold is removed. - bool default_event_based_hold = 14; - - // User-provided labels, in key/value pairs. - map labels = 15; - - // The bucket's website configuration, controlling how the service behaves - // when accessing bucket contents as a web site. See the - // [https://cloud.google.com/storage/docs/static-website][Static Website - // Examples] for more information. - Website website = 16; - - // The bucket's versioning configuration. - Versioning versioning = 17; - - // The bucket's logging configuration, which defines the destination bucket - // and optional name prefix for the current bucket's logs. - Logging logging = 18; - - // The owner of the bucket. This is always the project team's owner group. - Owner owner = 19; - - // Encryption configuration for a bucket. - Encryption encryption = 20; - - // The bucket's billing configuration. - Billing billing = 21; - - // The bucket's retention policy. The retention policy enforces a minimum - // retention time for all objects contained in the bucket, based on their - // creation time. Any attempt to overwrite or delete objects younger than the - // retention period will result in a PERMISSION_DENIED error. An unlocked - // retention policy can be modified or removed from the bucket via a - // storage.buckets.update operation. A locked retention policy cannot be - // removed or shortened in duration for the lifetime of the bucket. - // Attempting to remove or decrease period of a locked retention policy will - // result in a PERMISSION_DENIED error. - RetentionPolicy retention_policy = 22; -} - -// An access-control entry. -message BucketAccessControl { - // The access permission for the entity. - string role = 1; - - // HTTP 1.1 ["https://tools.ietf.org/html/rfc7232#section-2.3][Entity tag] - // for the access-control entry. - string etag = 2; - - // The ID of the access-control entry. - string id = 3; - - // The name of the bucket. - string bucket = 4; - - // The entity holding the permission, in one of the following forms: - // * `user-{userid}` - // * `user-{email}` - // * `group-{groupid}` - // * `group-{email}` - // * `domain-{domain}` - // * `project-{team-projectid}` - // * `allUsers` - // * `allAuthenticatedUsers` - // Examples: - // * The user `liz@example.com` would be `user-liz@example.com`. - // * The group `example@googlegroups.com` would be - // `group-example@googlegroups.com` - // * All members of the Google Apps for Business domain `example.com` would be - // `domain-example.com` - string entity = 6; - - // The ID for the entity, if any. - string entity_id = 7; - - // The email address associated with the entity, if any. - string email = 8; - - // The domain associated with the entity, if any. - string domain = 9; - - // The project team associated with the entity, if any. - ProjectTeam project_team = 10; -} - -// The response to a call to BucketAccessControls.ListBucketAccessControls. -message ListBucketAccessControlsResponse { - // The list of items. - repeated BucketAccessControl items = 1; -} - -// The result of a call to Buckets.ListBuckets -message ListBucketsResponse { - // The list of items. - repeated Bucket items = 1; - - // The continuation token, used to page through large result sets. Provide - // this value in a subsequent request to return the next page of results. - string next_page_token = 2; -} - -// An notification channel used to watch for resource changes. -message Channel { - // A UUID or similar unique string that identifies this channel. - string id = 1; - - // An opaque ID that identifies the resource being watched on this channel. - // Stable across different API versions. - string resource_id = 2; - - // A version-specific identifier for the watched resource. - string resource_uri = 3; - - // An arbitrary string delivered to the target address with each notification - // delivered over this channel. Optional. - string token = 4; - - // Date and time of notification channel expiration. Optional. - google.protobuf.Timestamp expiration = 5; - - // The type of delivery mechanism used for this channel. - string type = 6; - - // The address where notifications are delivered for this channel. - string address = 7; - - // Additional parameters controlling delivery channel behavior. Optional. - map params = 8; - - // A Boolean value to indicate whether payload is wanted. Optional. - bool payload = 9; -} - -// Message used to convey content being read or written, along with its -// checksum. -message ChecksummedData { - // The data. - bytes content = 1; - - // CRC32C digest of the contents. - google.protobuf.UInt32Value crc32c = 2; -} - -// Message used for storing full (not subrange) object checksums. -message ObjectChecksums { - // CRC32C digest of the object data. Computed by the GCS service for - // all written objects, and validated by the GCS service against - // client-supplied values if present in an InsertObjectRequest. - google.protobuf.UInt32Value crc32c = 1; - - // Hex-encoded MD5 hash of the object data (hexdigest). Whether/how this - // checksum is provided and validated is service-dependent. - string md5_hash = 2; -} - -// A collection of enums used in multiple places throughout the API. -message CommonEnums { - // A set of properties to return in a response. - enum Projection { - // No specified projection. - PROJECTION_UNSPECIFIED = 0; - - // Omit `owner`, `acl`, and `defaultObjectAcl` properties. - NO_ACL = 1; - - // Include all properties. - FULL = 2; - } - - // Predefined or "canned" aliases for sets of specific bucket ACL entries. - enum PredefinedBucketAcl { - // No predefined ACL. - PREDEFINED_BUCKET_ACL_UNSPECIFIED = 0; - - // Project team owners get `OWNER` access, and - // `allAuthenticatedUsers` get `READER` access. - BUCKET_ACL_AUTHENTICATED_READ = 1; - - // Project team owners get `OWNER` access. - BUCKET_ACL_PRIVATE = 2; - - // Project team members get access according to their roles. - BUCKET_ACL_PROJECT_PRIVATE = 3; - - // Project team owners get `OWNER` access, and - // `allUsers` get `READER` access. - BUCKET_ACL_PUBLIC_READ = 4; - - // Project team owners get `OWNER` access, and - // `allUsers` get `WRITER` access. - BUCKET_ACL_PUBLIC_READ_WRITE = 5; - } - - // Predefined or "canned" aliases for sets of specific object ACL entries. - enum PredefinedObjectAcl { - // No predefined ACL. - PREDEFINED_OBJECT_ACL_UNSPECIFIED = 0; - - // Object owner gets `OWNER` access, and - // `allAuthenticatedUsers` get `READER` access. - OBJECT_ACL_AUTHENTICATED_READ = 1; - - // Object owner gets `OWNER` access, and project team owners get - // `OWNER` access. - OBJECT_ACL_BUCKET_OWNER_FULL_CONTROL = 2; - - // Object owner gets `OWNER` access, and project team owners get - // `READER` access. - OBJECT_ACL_BUCKET_OWNER_READ = 3; - - // Object owner gets `OWNER` access. - OBJECT_ACL_PRIVATE = 4; - - // Object owner gets `OWNER` access, and project team members get - // access according to their roles. - OBJECT_ACL_PROJECT_PRIVATE = 5; - - // Object owner gets `OWNER` access, and `allUsers` - // get `READER` access. - OBJECT_ACL_PUBLIC_READ = 6; - } - - -} - -// Specifies a requested range of bytes to download. -message ContentRange { - // The starting offset of the object data. - int64 start = 1; - - // The ending offset of the object data. - int64 end = 2; - - // The complete length of the object data. - int64 complete_length = 3; -} - -// A subscription to receive Google PubSub notifications. -message Notification { - // The Cloud PubSub topic to which this subscription publishes. Formatted as: - // '//pubsub.googleapis.com/projects/{project-identifier}/topics/{my-topic}' - string topic = 1; - - // If present, only send notifications about listed event types. If empty, - // sent notifications for all event types. - repeated string event_types = 2; - - // An optional list of additional attributes to attach to each Cloud PubSub - // message published for this notification subscription. - map custom_attributes = 3; - - // HTTP 1.1 [https://tools.ietf.org/html/rfc7232#section-2.3][Entity tag] - // for this subscription notification. - string etag = 4; - - // If present, only apply this notification configuration to object names that - // begin with this prefix. - string object_name_prefix = 5; - - // The desired content of the Payload. - string payload_format = 6; - - // The ID of the notification. - string id = 7; -} - -// The result of a call to Notifications.ListNotifications -message ListNotificationsResponse { - // The list of items. - repeated Notification items = 1; -} - -// An object. -message Object { - // Describes the customer-specified mechanism used to store the data at rest. - message CustomerEncryption { - // The encryption algorithm. - string encryption_algorithm = 1; - - // SHA256 hash value of the encryption key. - string key_sha256 = 2; - } - - // Content-Encoding of the object data, matching - // [https://tools.ietf.org/html/rfc7231#section-3.1.2.2][RFC 7231 §3.1.2.2] - string content_encoding = 1; - - // Content-Disposition of the object data, matching - // [https://tools.ietf.org/html/rfc6266][RFC 6266]. - string content_disposition = 2; - - // Cache-Control directive for the object data, matching - // [https://tools.ietf.org/html/rfc7234#section-5.2"][RFC 7234 §5.2]. - // If omitted, and the object is accessible to all anonymous users, the - // default will be `public, max-age=3600`. - string cache_control = 3; - - // Access controls on the object. - repeated ObjectAccessControl acl = 4; - - // Content-Language of the object data, matching - // [https://tools.ietf.org/html/rfc7231#section-3.1.3.2][RFC 7231 §3.1.3.2]. - string content_language = 5; - - // The version of the metadata for this object at this generation. Used for - // preconditions and for detecting changes in metadata. A metageneration - // number is only meaningful in the context of a particular generation of a - // particular object. - // Attempting to set this field will result in an error. - int64 metageneration = 6; - - // The deletion time of the object. Will be returned if and only if this - // version of the object has been deleted. - // Attempting to set this field will result in an error. - google.protobuf.Timestamp time_deleted = 7; - - // Content-Type of the object data, matching - // [https://tools.ietf.org/html/rfc7231#section-3.1.1.5][RFC 7231 §3.1.1.5]. - // If an object is stored without a Content-Type, it is served as - // `application/octet-stream`. - string content_type = 8; - - // Content-Length of the object data in bytes, matching - // [https://tools.ietf.org/html/rfc7230#section-3.3.2][RFC 7230 §3.3.2]. - // Attempting to set this field will result in an error. - int64 size = 9; - - // The creation time of the object. - // Attempting to set this field will result in an error. - google.protobuf.Timestamp time_created = 10; - - // CRC32c checksum. For more information about using the CRC32c - // checksum, see - // [https://cloud.google.com/storage/docs/hashes-etags#_JSONAPI][Hashes and - // ETags: Best Practices]. This is a server determined value and should not be - // supplied by the user when sending an Object. The server will ignore any - // value provided. Users should instead use the object_checksums field on the - // InsertObjectRequest when uploading an object. - google.protobuf.UInt32Value crc32c = 11; - - // Number of underlying components that make up this object. Components are - // accumulated by compose operations. - // Attempting to set this field will result in an error. - int32 component_count = 12; - - // MD5 hash of the data; encoded using base64 as per - // [https://tools.ietf.org/html/rfc4648#section-4][RFC 4648 §4]. For more - // information about using the MD5 hash, see - // [https://cloud.google.com/storage/docs/hashes-etags#_JSONAPI][Hashes and - // ETags: Best Practices]. This is a server determined value and should not be - // supplied by the user when sending an Object. The server will ignore any - // value provided. Users should instead use the object_checksums field on the - // InsertObjectRequest when uploading an object. - string md5_hash = 13; - - // HTTP 1.1 Entity tag for the object. See - // [https://tools.ietf.org/html/rfc7232#section-2.3][RFC 7232 §2.3]. - // Attempting to set this field will result in an error. - string etag = 14; - - // The modification time of the object metadata. - // Attempting to set this field will result in an error. - google.protobuf.Timestamp updated = 15; - - // Storage class of the object. - string storage_class = 16; - - // Cloud KMS Key used to encrypt this object, if the object is encrypted by - // such a key. - string kms_key_name = 17; - - // The time at which the object's storage class was last changed. When the - // object is initially created, it will be set to time_created. - // Attempting to set this field will result in an error. - google.protobuf.Timestamp time_storage_class_updated = 18; - - // Whether an object is under temporary hold. While this flag is set to true, - // the object is protected against deletion and overwrites. A common use case - // of this flag is regulatory investigations where objects need to be retained - // while the investigation is ongoing. Note that unlike event-based hold, - // temporary hold does not impact retention expiration time of an object. - bool temporary_hold = 19; - - // A server-determined value that specifies the earliest time that the - // object's retention period expires. This value is in - // [https://tools.ietf.org/html/rfc3339][RFC 3339] format. - // Note 1: This field is not provided for objects with an active event-based - // hold, since retention expiration is unknown until the hold is removed. - // Note 2: This value can be provided even when temporary hold is set (so that - // the user can reason about policy without having to first unset the - // temporary hold). - google.protobuf.Timestamp retention_expiration_time = 20; - - // User-provided metadata, in key/value pairs. - map metadata = 21; - - // Whether an object is under event-based hold. Event-based hold is a way to - // retain objects until an event occurs, which is signified by the - // hold's release (i.e. this value is set to false). After being released (set - // to false), such objects will be subject to bucket-level retention (if any). - // One sample use case of this flag is for banks to hold loan documents for at - // least 3 years after loan is paid in full. Here, bucket-level retention is 3 - // years and the event is the loan being paid in full. In this example, these - // objects will be held intact for any number of years until the event has - // occurred (event-based hold on the object is released) and then 3 more years - // after that. That means retention duration of the objects begins from the - // moment event-based hold transitioned from true to false. - google.protobuf.BoolValue event_based_hold = 29; - - // The name of the object. Required if not specified by URL parameter. - // Attempting to update this field after the object is created will result in - // an error. - string name = 23; - - // The ID of the object, including the bucket name, object name, and - // generation number. - // Attempting to update this field after the object is created will result in - // an error. - string id = 24; - - // The name of the bucket containing this object. - // Attempting to update this field after the object is created will result in - // an error. - string bucket = 25; - - // The content generation of this object. Used for object versioning. - // Attempting to set this field will result in an error. - int64 generation = 26; - - // The owner of the object. This will always be the uploader of the object. - // Attempting to set this field will result in an error. - Owner owner = 27; - - // Metadata of customer-supplied encryption key, if the object is encrypted by - // such a key. - CustomerEncryption customer_encryption = 28; -} - -// An access-control entry. -message ObjectAccessControl { - // The access permission for the entity. - string role = 1; - - // HTTP 1.1 Entity tag for the access-control entry. - // See [https://tools.ietf.org/html/rfc7232#section-2.3][RFC 7232 §2.3]. - string etag = 2; - - // The ID of the access-control entry. - string id = 3; - - // The name of the bucket. - string bucket = 4; - - // The name of the object, if applied to an object. - string object = 5; - - // The content generation of the object, if applied to an object. - int64 generation = 6; - - // The entity holding the permission, in one of the following forms: - // * `user-{userid}` - // * `user-{email}` - // * `group-{groupid}` - // * `group-{email}` - // * `domain-{domain}` - // * `project-{team-projectid}` - // * `allUsers` - // * `allAuthenticatedUsers` - // Examples: - // * The user `liz@example.com` would be `user-liz@example.com`. - // * The group `example@googlegroups.com` would be - // `group-example@googlegroups.com`. - // * All members of the Google Apps for Business domain `example.com` would be - // `domain-example.com`. - string entity = 7; - - // The ID for the entity, if any. - string entity_id = 8; - - // The email address associated with the entity, if any. - string email = 9; - - // The domain associated with the entity, if any. - string domain = 10; - - // The project team associated with the entity, if any. - ProjectTeam project_team = 11; -} - -// The result of a call to ObjectAccessControls.ListObjectAccessControls. -message ListObjectAccessControlsResponse { - // The list of items. - repeated ObjectAccessControl items = 1; -} - -// The result of a call to Objects.ListObjects -message ListObjectsResponse { - // The list of prefixes of objects matching-but-not-listed up to and including - // the requested delimiter. - repeated string prefixes = 1; - - // The list of items. - repeated Object items = 2; - - // The continuation token, used to page through large result sets. Provide - // this value in a subsequent request to return the next page of results. - string next_page_token = 3; -} - -// Represents the Viewers, Editors, or Owners of a given project. -message ProjectTeam { - // The project number. - string project_number = 1; - - // The team. - string team = 2; -} - -// A subscription to receive Google PubSub notifications. -message ServiceAccount { - // The ID of the notification. - string email_address = 1; -} - -// The owner of a specific resource. -message Owner { - // The entity, in the form `user-`*userId*. - string entity = 1; - - // The ID for the entity. - string entity_id = 2; -} diff --git a/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcReadChannelTest.java b/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcReadChannelTest.java index 5158de307f..9127287f6a 100644 --- a/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcReadChannelTest.java +++ b/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcReadChannelTest.java @@ -33,16 +33,15 @@ import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.cloud.hadoop.util.testing.MockHttpTransportHelper.ErrorResponses; import com.google.common.hash.Hashing; -import com.google.google.storage.v1.ChecksummedData; -import com.google.google.storage.v1.GetObjectMediaRequest; -import com.google.google.storage.v1.GetObjectMediaResponse; -import com.google.google.storage.v1.GetObjectRequest; -import com.google.google.storage.v1.Object; -import com.google.google.storage.v1.StorageGrpc; -import com.google.google.storage.v1.StorageGrpc.StorageBlockingStub; -import com.google.google.storage.v1.StorageGrpc.StorageImplBase; import com.google.protobuf.ByteString; -import com.google.protobuf.UInt32Value; +import com.google.storage.v2.ChecksummedData; +import com.google.storage.v2.Object; +import com.google.storage.v2.ObjectChecksums; +import com.google.storage.v2.ReadObjectRequest; +import com.google.storage.v2.ReadObjectResponse; +import com.google.storage.v2.StorageGrpc; +import com.google.storage.v2.StorageGrpc.StorageBlockingStub; +import com.google.storage.v2.StorageGrpc.StorageImplBase; import io.grpc.ManagedChannelBuilder; import io.grpc.Status; import io.grpc.Status.Code; @@ -56,7 +55,6 @@ import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import org.junit.Before; import org.junit.Rule; @@ -70,32 +68,32 @@ @RunWith(JUnit4.class) public final class GoogleCloudStorageGrpcReadChannelTest { - private static final String BUCKET_NAME = "bucket-name"; + private static final String V1_BUCKET_NAME = "bucket-name"; + private static final String BUCKET_NAME = GrpcChannelUtils.toV2BucketName(V1_BUCKET_NAME); private static final String OBJECT_NAME = "object-name"; private static final long OBJECT_GENERATION = 7; private static final int OBJECT_SIZE = GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE + 10; private static final int DEFAULT_OBJECT_CRC32C = 185327488; - private static Object DEFAULT_OBJECT = + private static final Object DEFAULT_OBJECT = Object.newBuilder() - .setBucket(BUCKET_NAME) + .setBucket(V1_BUCKET_NAME) // returning bucket name in v1 format as metadata is + // fetched via json api .setName(OBJECT_NAME) .setSize(OBJECT_SIZE) - .setCrc32C(UInt32Value.newBuilder().setValue(DEFAULT_OBJECT_CRC32C)) + .setChecksums(ObjectChecksums.newBuilder().setCrc32C(DEFAULT_OBJECT_CRC32C).build()) .setGeneration(OBJECT_GENERATION) .build(); - private static GetObjectMediaRequest GET_OBJECT_MEDIA_REQUEST = - GetObjectMediaRequest.newBuilder() + private static final ReadObjectRequest GET_OBJECT_MEDIA_REQUEST = + ReadObjectRequest.newBuilder() .setBucket(BUCKET_NAME) .setObject(OBJECT_NAME) .setGeneration(OBJECT_GENERATION) .build(); - @Rule - public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); + @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); private StorageBlockingStub stub; private FakeService fakeService; - @Mock - private Credentials mockCredentials; + @Mock private Credentials mockCredentials; private Storage storage; private ApiErrorExtractor errorExtractor; private Get get; @@ -120,9 +118,9 @@ public void setUp() throws Exception { get = mock(Get.class); Objects objects = mock(Objects.class); when(storage.objects()).thenReturn(objects); - when(objects.get(BUCKET_NAME, OBJECT_NAME)).thenReturn(get); + when(objects.get(V1_BUCKET_NAME, OBJECT_NAME)).thenReturn(get); storageObject = new StorageObject(); - storageObject.setBucket(BUCKET_NAME); + storageObject.setBucket(V1_BUCKET_NAME); storageObject.setGeneration(OBJECT_GENERATION); storageObject.setSize(BigInteger.valueOf(OBJECT_SIZE)); when(get.setFields(any())).thenCallRealMethod(); @@ -145,17 +143,25 @@ public void readSingleChunkSucceeds() throws Exception { verify(get).setFields(METADATA_FIELDS); verify(get).execute(); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(objectSize - 2) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(objectSize - 2) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(objectSize - 2) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(objectSize - 2) + .build()), + any()); assertArrayEquals(fakeService.data.substring(0, 100).toByteArray(), buffer.array()); verifyNoMoreInteractions(fakeService); } @@ -176,19 +182,26 @@ public void readMultipleChunksSucceeds() throws Exception { verify(get).setFields(METADATA_FIELDS); verify(get).execute(); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(objectSize - 2) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(objectSize - 2) - .build()), any()); - assertArrayEquals( - fakeService.data.substring(0, objectSize).toByteArray(), buffer.array()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(objectSize - 2) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(objectSize - 2) + .build()), + any()); + assertArrayEquals(fakeService.data.substring(0, objectSize).toByteArray(), buffer.array()); verifyNoMoreInteractions(fakeService); } @@ -196,12 +209,10 @@ public void readMultipleChunksSucceeds() throws Exception { public void readAfterRepositioningAfterSkippingSucceeds() throws Exception { int objectSize = GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE * 10; storageObject.setSize(BigInteger.valueOf(objectSize)); - fakeService.setObject(DEFAULT_OBJECT.toBuilder() - .setSize(objectSize).build()); + fakeService.setObject(DEFAULT_OBJECT.toBuilder().setSize(objectSize).build()); verify(fakeService, times(1)).setObject(any()); GoogleCloudStorageReadOptions options = - GoogleCloudStorageReadOptions.builder().setInplaceSeekLimit(10) - .build(); + GoogleCloudStorageReadOptions.builder().setInplaceSeekLimit(10).build(); GoogleCloudStorageGrpcReadChannel readChannel = newReadChannel(options); ByteBuffer bufferAtBeginning = ByteBuffer.allocate(20); @@ -221,25 +232,39 @@ public void readAfterRepositioningAfterSkippingSucceeds() throws Exception { assertArrayEquals(fakeService.data.substring(0, 20).toByteArray(), bufferAtBeginning.array()); int footerOffset = objectSize - (GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE / 2); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(footerOffset) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(footerOffset) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadOffset(1) - .setReadLimit( - objectSize - (GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE / 2) - 1) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(footerOffset) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(footerOffset) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadOffset(1) + .setReadLimit( + objectSize + - (GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE / 2) + - 1) + .build()), + any()); assertArrayEquals( fakeService.data.substring(25, 30).toByteArray(), bufferFromSkippedSection1.array()); assertArrayEquals( @@ -253,12 +278,10 @@ public void readAfterRepositioningAfterSkippingSucceeds() throws Exception { public void multipleSequentialReads() throws Exception { int objectSize = GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE * 10; storageObject.setSize(BigInteger.valueOf(objectSize)); - fakeService.setObject(DEFAULT_OBJECT.toBuilder() - .setSize(objectSize).build()); + fakeService.setObject(DEFAULT_OBJECT.toBuilder().setSize(objectSize).build()); verify(fakeService, times(1)).setObject(any()); GoogleCloudStorageReadOptions options = - GoogleCloudStorageReadOptions.builder().setInplaceSeekLimit(10) - .build(); + GoogleCloudStorageReadOptions.builder().setInplaceSeekLimit(10).build(); GoogleCloudStorageGrpcReadChannel readChannel = newReadChannel(options); ByteBuffer first_buffer = ByteBuffer.allocate(10); @@ -272,17 +295,25 @@ public void multipleSequentialReads() throws Exception { assertArrayEquals(fakeService.data.substring(10, 30).toByteArray(), second_buffer.array()); int footerOffset = objectSize - (GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE / 2); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(footerOffset) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(footerOffset) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(footerOffset) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(footerOffset) + .build()), + any()); verifyNoMoreInteractions(fakeService); } @@ -290,8 +321,7 @@ public void multipleSequentialReads() throws Exception { public void randomReadRequestsExactBytes() throws Exception { int objectSize = GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE * 10; storageObject.setSize(BigInteger.valueOf(objectSize)); - fakeService.setObject(DEFAULT_OBJECT.toBuilder() - .setSize(objectSize).build()); + fakeService.setObject(DEFAULT_OBJECT.toBuilder().setSize(objectSize).build()); verify(fakeService, times(1)).setObject(any()); GoogleCloudStorageReadOptions options = GoogleCloudStorageReadOptions.builder() @@ -307,23 +337,27 @@ public void randomReadRequestsExactBytes() throws Exception { verify(get).setFields(METADATA_FIELDS); verify(get).execute(); - GetObjectMediaRequest expectedRequest = - GetObjectMediaRequest.newBuilder() + ReadObjectRequest expectedRequest = + ReadObjectRequest.newBuilder() .setBucket(BUCKET_NAME) .setObject(OBJECT_NAME) .setGeneration(OBJECT_GENERATION) .setReadLimit(GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE) .setReadOffset(10) .build(); - verify(fakeService, times(1)).getObjectMedia(eq(expectedRequest), any()); + verify(fakeService, times(1)).readObject(eq(expectedRequest), any()); assertArrayEquals(fakeService.data.substring(10, 60).toByteArray(), buffer.array()); int footerOffset = objectSize - (GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE / 2); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(footerOffset) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(footerOffset) + .build()), + any()); verifyNoMoreInteractions(fakeService); } @@ -331,8 +365,7 @@ public void randomReadRequestsExactBytes() throws Exception { public void repeatedRandomReadsWorkAsExpected() throws Exception { int objectSize = GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE * 10; storageObject.setSize(BigInteger.valueOf(objectSize)); - fakeService.setObject(DEFAULT_OBJECT.toBuilder() - .setSize(objectSize).build()); + fakeService.setObject(DEFAULT_OBJECT.toBuilder().setSize(objectSize).build()); verify(fakeService, times(1)).setObject(any()); GoogleCloudStorageReadOptions options = GoogleCloudStorageReadOptions.builder() @@ -352,16 +385,16 @@ public void repeatedRandomReadsWorkAsExpected() throws Exception { readChannel.read(buffer); assertArrayEquals(fakeService.data.substring(20, 45).toByteArray(), buffer.array()); - GetObjectMediaRequest firstExpectedRequest = - GetObjectMediaRequest.newBuilder() + ReadObjectRequest firstExpectedRequest = + ReadObjectRequest.newBuilder() .setBucket(BUCKET_NAME) .setObject(OBJECT_NAME) .setGeneration(OBJECT_GENERATION) .setReadLimit(GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE) .setReadOffset(10) .build(); - GetObjectMediaRequest secondExpectedRequest = - GetObjectMediaRequest.newBuilder() + ReadObjectRequest secondExpectedRequest = + ReadObjectRequest.newBuilder() .setBucket(BUCKET_NAME) .setObject(OBJECT_NAME) .setGeneration(OBJECT_GENERATION) @@ -373,13 +406,17 @@ public void repeatedRandomReadsWorkAsExpected() throws Exception { verify(get).execute(); int footerOffset = objectSize - (GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE / 2); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(footerOffset) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(firstExpectedRequest), any()); - verify(fakeService, times(1)).getObjectMedia(eq(secondExpectedRequest), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(footerOffset) + .build()), + any()); + verify(fakeService, times(1)).readObject(eq(firstExpectedRequest), any()); + verify(fakeService, times(1)).readObject(eq(secondExpectedRequest), any()); verifyNoMoreInteractions(fakeService); } @@ -387,8 +424,7 @@ public void repeatedRandomReadsWorkAsExpected() throws Exception { public void randomReadRequestsExpectedBytes() throws Exception { int objectSize = GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE * 10; storageObject.setSize(BigInteger.valueOf(objectSize)); - fakeService.setObject(DEFAULT_OBJECT.toBuilder() - .setSize(objectSize).build()); + fakeService.setObject(DEFAULT_OBJECT.toBuilder().setSize(objectSize).build()); verify(fakeService, times(1)).setObject(any()); GoogleCloudStorageReadOptions options = GoogleCloudStorageReadOptions.builder() @@ -415,16 +451,16 @@ public void randomReadRequestsExpectedBytes() throws Exception { .toByteArray(), buffer.array()); - GetObjectMediaRequest firstExpectedRequest = - GetObjectMediaRequest.newBuilder() + ReadObjectRequest firstExpectedRequest = + ReadObjectRequest.newBuilder() .setBucket(BUCKET_NAME) .setObject(OBJECT_NAME) .setGeneration(OBJECT_GENERATION) .setReadLimit(GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE) .setReadOffset(10) .build(); - GetObjectMediaRequest secondExpectedRequest = - GetObjectMediaRequest.newBuilder() + ReadObjectRequest secondExpectedRequest = + ReadObjectRequest.newBuilder() .setBucket(BUCKET_NAME) .setObject(OBJECT_NAME) .setGeneration(OBJECT_GENERATION) @@ -437,13 +473,17 @@ public void randomReadRequestsExpectedBytes() throws Exception { int footerOffset = objectSize - (GoogleCloudStorageReadOptions.DEFAULT_MIN_RANGE_REQUEST_SIZE / 2); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(footerOffset) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(firstExpectedRequest), any()); - verify(fakeService, times(1)).getObjectMedia(eq(secondExpectedRequest), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(footerOffset) + .build()), + any()); + verify(fakeService, times(1)).readObject(eq(firstExpectedRequest), any()); + verify(fakeService, times(1)).readObject(eq(secondExpectedRequest), any()); verifyNoMoreInteractions(fakeService); } @@ -466,17 +506,25 @@ public void readToBufferWithArrayOffset() throws Exception { verify(get).execute(); byte[] expected = ByteString.copyFrom(array, 50, objectSize).toByteArray(); int footerOffset = 98; - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(footerOffset) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(footerOffset) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(footerOffset) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(footerOffset) + .build()), + any()); assertArrayEquals(fakeService.data.substring(0, objectSize).toByteArray(), expected); verifyNoMoreInteractions(fakeService); } @@ -488,7 +536,9 @@ public void readSucceedsAfterSeek() throws Exception { storageObject.setSize(BigInteger.valueOf(objectSize)); verify(fakeService, times(1)).setObject(any()); GoogleCloudStorageReadOptions options = - GoogleCloudStorageReadOptions.builder().setMinRangeRequestSize(4).setInplaceSeekLimit(10) + GoogleCloudStorageReadOptions.builder() + .setMinRangeRequestSize(4) + .setInplaceSeekLimit(10) .build(); GoogleCloudStorageGrpcReadChannel readChannel = newReadChannel(options); @@ -499,14 +549,18 @@ public void readSucceedsAfterSeek() throws Exception { verify(get).setFields(METADATA_FIELDS); verify(get).execute(); verify(fakeService, times(1)) - .getObjectMedia( + .readObject( eq(GET_OBJECT_MEDIA_REQUEST.toBuilder().setReadOffset(50).setReadLimit(48).build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(98) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(98) + .build()), + any()); assertArrayEquals(fakeService.data.substring(50, 60).toByteArray(), buffer.array()); verifyNoMoreInteractions(fakeService); } @@ -514,8 +568,9 @@ public void readSucceedsAfterSeek() throws Exception { @Test public void singleReadSucceedsWithValidObjectChecksum() throws Exception { fakeService.setObject( - DEFAULT_OBJECT.toBuilder() - .setCrc32C(UInt32Value.newBuilder().setValue(DEFAULT_OBJECT_CRC32C)) + DEFAULT_OBJECT + .toBuilder() + .setChecksums(ObjectChecksums.newBuilder().setCrc32C(DEFAULT_OBJECT_CRC32C)) .build()); GoogleCloudStorageReadOptions options = GoogleCloudStorageReadOptions.builder().setGrpcChecksumsEnabled(true).build(); @@ -530,7 +585,10 @@ public void singleReadSucceedsWithValidObjectChecksum() throws Exception { @Test public void partialReadSucceedsWithInvalidObjectChecksum() throws Exception { fakeService.setObject( - DEFAULT_OBJECT.toBuilder().setCrc32C(UInt32Value.newBuilder().setValue(0)).build()); + DEFAULT_OBJECT + .toBuilder() + .setChecksums(ObjectChecksums.newBuilder().setCrc32C(DEFAULT_OBJECT_CRC32C)) + .build()); GoogleCloudStorageReadOptions options = GoogleCloudStorageReadOptions.builder().setGrpcChecksumsEnabled(true).build(); GoogleCloudStorageGrpcReadChannel readChannel = newReadChannel(options); @@ -543,10 +601,11 @@ public void partialReadSucceedsWithInvalidObjectChecksum() throws Exception { } @Test - public void multipleSequentialsReadsSucceedWithValidObjectChecksum() throws Exception { + public void multipleSequentialReadsSucceedWithValidObjectChecksum() throws Exception { fakeService.setObject( - DEFAULT_OBJECT.toBuilder() - .setCrc32C(UInt32Value.newBuilder().setValue(DEFAULT_OBJECT_CRC32C)) + DEFAULT_OBJECT + .toBuilder() + .setChecksums(ObjectChecksums.newBuilder().setCrc32C(DEFAULT_OBJECT_CRC32C)) .build()); GoogleCloudStorageReadOptions options = GoogleCloudStorageReadOptions.builder().setGrpcChecksumsEnabled(true).build(); @@ -609,7 +668,10 @@ public void readToBufferWithArrayOffsetFailsWithInvalidMessageChecksum() throws @Test public void multipleReadsIgnoreObjectChecksumForLatestGenerationReads() throws Exception { fakeService.setObject( - DEFAULT_OBJECT.toBuilder().setCrc32C(UInt32Value.newBuilder().setValue(0)).build()); + DEFAULT_OBJECT + .toBuilder() + .setChecksums(ObjectChecksums.newBuilder().setCrc32C(DEFAULT_OBJECT_CRC32C)) + .build()); GoogleCloudStorageReadOptions options = GoogleCloudStorageReadOptions.builder().setGrpcChecksumsEnabled(true).build(); GoogleCloudStorageGrpcReadChannel readChannel = newReadChannel(options); @@ -628,10 +690,9 @@ public void testOpenReadsMetadata() throws IOException { int objectSize = 8 * 1024; fakeService.setObject(DEFAULT_OBJECT.toBuilder().setSize(objectSize).build()); storageObject.setSize(BigInteger.valueOf(objectSize)); - GoogleCloudStorageReadOptions options = - GoogleCloudStorageReadOptions.builder().build(); - StorageResourceId storageResourceId = new StorageResourceId(BUCKET_NAME, OBJECT_NAME, - OBJECT_GENERATION); + GoogleCloudStorageReadOptions options = GoogleCloudStorageReadOptions.builder().build(); + StorageResourceId storageResourceId = + new StorageResourceId(V1_BUCKET_NAME, OBJECT_NAME, OBJECT_GENERATION); GoogleCloudStorageGrpcReadChannel readChannel = newReadChannel(storageResourceId, options); assertTrue(readChannel.isOpen()); @@ -652,17 +713,13 @@ public void testOpenThrowsIOExceptionOnGetError() throws IOException { GoogleCloudStorageReadOptions.builder().setFastFailOnNotFound(false).build(); IOException thrown = assertThrows(IOException.class, () -> newReadChannel(storage, options)); - assertThat(thrown) - .hasCauseThat() - .hasCauseThat() - .hasMessageThat() - .contains("backendError"); + assertThat(thrown).hasCauseThat().hasCauseThat().hasMessageThat().contains("backendError"); } @Test public void readHandlesGetMediaError() throws Exception { GoogleCloudStorageGrpcReadChannel readChannel = newReadChannel(); - fakeService.setGetMediaException( + fakeService.setReadObjectException( Status.fromCode(Status.Code.INTERNAL) .withDescription("Custom error message.") .asException()); @@ -678,11 +735,11 @@ public void readHandlesGetMediaError() throws Exception { @Test public void testOpenThrowsIOExceptionOnGetMediaError() throws IOException { - fakeService.setGetMediaException( + fakeService.setReadObjectException( Status.fromCode(Status.Code.INTERNAL) .withDescription("Custom error message.") .asException()); - verify(fakeService).setGetMediaException(any()); + verify(fakeService).setReadObjectException(any()); IOException thrown = assertThrows(IOException.class, this::newReadChannel); assertThat(thrown) .hasCauseThat() @@ -691,17 +748,15 @@ public void testOpenThrowsIOExceptionOnGetMediaError() throws IOException { .contains("Custom error message."); verify(get).setFields(METADATA_FIELDS); verify(get).execute(); - verify(fakeService).getObjectMedia(any(), any()); + verify(fakeService).readObject(any(), any()); verifyNoMoreInteractions(fakeService); } @Test public void testOpenThrowsIOExceptionOnGetMediaFileNotFound() throws IOException { - fakeService.setGetMediaException( - Status.fromCode(Code.NOT_FOUND) - .withDescription("Custom error message.") - .asException()); - verify(fakeService).setGetMediaException(any()); + fakeService.setReadObjectException( + Status.fromCode(Code.NOT_FOUND).withDescription("Custom error message.").asException()); + verify(fakeService).setReadObjectException(any()); IOException thrown = assertThrows(IOException.class, this::newReadChannel); assertThat(thrown) .hasCauseThat() @@ -711,13 +766,12 @@ public void testOpenThrowsIOExceptionOnGetMediaFileNotFound() throws IOException .contains("Custom error message."); verify(get).setFields(METADATA_FIELDS); verify(get).execute(); - verify(fakeService).getObjectMedia(any(), any()); + verify(fakeService).readObject(any(), any()); verifyNoMoreInteractions(fakeService); } @Test - public void testOpenThrowsIOExceptionOnGzipContent() - throws Exception { + public void testOpenThrowsIOExceptionOnGzipContent() throws Exception { MockHttpTransport transport = mockTransport( jsonDataResponse( @@ -725,8 +779,7 @@ public void testOpenThrowsIOExceptionOnGzipContent() Storage storage = new Storage(transport, JSON_FACTORY, r -> {}); - GoogleCloudStorageReadOptions readOptions = - GoogleCloudStorageReadOptions.builder().build(); + GoogleCloudStorageReadOptions readOptions = GoogleCloudStorageReadOptions.builder().build(); IOException e = assertThrows(IOException.class, () -> newReadChannel(storage, readOptions)); assertThat(e) @@ -738,7 +791,7 @@ public void testOpenThrowsIOExceptionOnGzipContent() @Test public void retryGetMediaError() throws Exception { GoogleCloudStorageGrpcReadChannel readChannel = newReadChannel(); - fakeService.setGetMediaException( + fakeService.setReadObjectException( Status.fromCode(Status.Code.INTERNAL) .withDescription("Custom error message.") .asException()); @@ -766,8 +819,8 @@ public void readWithStrictGenerationReadConsistencySucceeds() throws Exception { int objectSize = 100; storageObject.setSize(BigInteger.valueOf(objectSize)); fakeService.setObject(DEFAULT_OBJECT.toBuilder().setSize(objectSize).setGeneration(1).build()); - GoogleCloudStorageReadOptions options = GoogleCloudStorageReadOptions.builder() - .setMinRangeRequestSize(4).build(); + GoogleCloudStorageReadOptions options = + GoogleCloudStorageReadOptions.builder().setMinRangeRequestSize(4).build(); GoogleCloudStorageGrpcReadChannel readChannel = newReadChannel(options); ByteBuffer buffer = ByteBuffer.allocate(10); @@ -777,19 +830,11 @@ public void readWithStrictGenerationReadConsistencySucceeds() throws Exception { buffer.clear(); readChannel.read(buffer); - List expectedRequests = - Arrays.asList( - GET_OBJECT_MEDIA_REQUEST, - GET_OBJECT_MEDIA_REQUEST.toBuilder() - .setReadOffset(10) - .setReadLimit(20) - .setGeneration(1) - .build()); - ArgumentCaptor requestCaptor = - ArgumentCaptor.forClass(GetObjectMediaRequest.class); + ArgumentCaptor requestCaptor = + ArgumentCaptor.forClass(ReadObjectRequest.class); verify(get).setFields(METADATA_FIELDS); verify(get).execute(); - verify(fakeService, times(3)).getObjectMedia(requestCaptor.capture(), any()); + verify(fakeService, times(3)).readObject(requestCaptor.capture(), any()); } @Test @@ -797,8 +842,8 @@ public void readWithLatestGenerationReadConsistencySucceeds() throws Exception { int objectSize = 100; fakeService.setObject(DEFAULT_OBJECT.toBuilder().setSize(objectSize).setGeneration(1).build()); storageObject.setSize(BigInteger.valueOf(objectSize)); - GoogleCloudStorageReadOptions options = GoogleCloudStorageReadOptions.builder() - .setMinRangeRequestSize(4).build(); + GoogleCloudStorageReadOptions options = + GoogleCloudStorageReadOptions.builder().setMinRangeRequestSize(4).build(); GoogleCloudStorageGrpcReadChannel readChannel = newReadChannel(options); ByteBuffer buffer = ByteBuffer.allocate(10); @@ -808,15 +853,11 @@ public void readWithLatestGenerationReadConsistencySucceeds() throws Exception { buffer.clear(); readChannel.read(buffer); - List expectedRequests = - Arrays.asList( - GET_OBJECT_MEDIA_REQUEST, - GET_OBJECT_MEDIA_REQUEST.toBuilder().setReadOffset(10).setReadLimit(20).build()); - ArgumentCaptor requestCaptor = - ArgumentCaptor.forClass(GetObjectMediaRequest.class); + ArgumentCaptor requestCaptor = + ArgumentCaptor.forClass(ReadObjectRequest.class); verify(get).setFields(METADATA_FIELDS); verify(get).execute(); - verify(fakeService, times(3)).getObjectMedia(requestCaptor.capture(), any()); + verify(fakeService, times(3)).readObject(requestCaptor.capture(), any()); } @Test @@ -827,8 +868,10 @@ public void seekUnderInplaceSeekLimitReadsCorrectBufferedData() throws Exception verify(fakeService, times(1)).setObject(any()); int minRangeRequestSize = 10; GoogleCloudStorageReadOptions options = - GoogleCloudStorageReadOptions.builder().setMinRangeRequestSize(minRangeRequestSize) - .setInplaceSeekLimit(10).build(); + GoogleCloudStorageReadOptions.builder() + .setMinRangeRequestSize(minRangeRequestSize) + .setInplaceSeekLimit(10) + .build(); GoogleCloudStorageGrpcReadChannel readChannel = newReadChannel(options); ByteBuffer buffer = ByteBuffer.allocate(20); @@ -839,17 +882,25 @@ public void seekUnderInplaceSeekLimitReadsCorrectBufferedData() throws Exception verify(get).setFields(METADATA_FIELDS); verify(get).execute(); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(95) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(95) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(95) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(95) + .build()), + any()); assertArrayEquals(fakeService.data.substring(25, 45).toByteArray(), buffer.array()); verifyNoMoreInteractions(fakeService); } @@ -896,24 +947,36 @@ public void seekBeyondInplaceSeekLimitReadsNoBufferedData() throws Exception { verify(get).setFields(METADATA_FIELDS); verify(get).execute(); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(95) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(95) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadOffset(35) - .setReadLimit(20) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(95) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(95) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadOffset(35) + .setReadLimit(20) + .build()), + any()); assertArrayEquals(fakeService.data.substring(35, 55).toByteArray(), buffer.array()); verifyNoMoreInteractions(fakeService); } @@ -926,9 +989,7 @@ public void testReadPrefetchedFooter() throws Exception { verify(fakeService, times(1)).setObject(any()); int minRangeRequestSize = 2 * 1024; GoogleCloudStorageReadOptions options = - GoogleCloudStorageReadOptions.builder() - .setMinRangeRequestSize(minRangeRequestSize) - .build(); + GoogleCloudStorageReadOptions.builder().setMinRangeRequestSize(minRangeRequestSize).build(); GoogleCloudStorageGrpcReadChannel readChannel = newReadChannel(options); ByteBuffer buffer = ByteBuffer.allocate(20); readChannel.position(80); @@ -937,11 +998,15 @@ public void testReadPrefetchedFooter() throws Exception { verify(get).setFields(METADATA_FIELDS); verify(get).execute(); // footerSize is bigger than object size, footer content essential is entire object content - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(0) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(0) + .build()), + any()); assertArrayEquals(fakeService.data.substring(80).toByteArray(), buffer.array()); verifyNoMoreInteractions(fakeService); } @@ -967,17 +1032,25 @@ public void testReadCachedFooter() throws Exception { verify(get).execute(); int footerOffset = 7 * 1024; - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(footerOffset) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(footerOffset) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(footerOffset) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(footerOffset) + .build()), + any()); buffer.clear(); readChannel.position(footerOffset); @@ -1014,31 +1087,45 @@ public void testReadCachedFooterPartially() throws Exception { verify(get).execute(); int footerOffset = 14 * 1024; - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(footerOffset) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(footerOffset) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(footerOffset) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(footerOffset) + .build()), + any()); buffer.clear(); int readOffset = 13 * 1024; readChannel.position(readOffset); readChannel.read(buffer); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadOffset(readOffset) - .setReadLimit(1024) - .build()), any()); - assertArrayEquals(fakeService.data.substring(readOffset, readOffset + (2 * 1024)).toByteArray(), buffer.array()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadOffset(readOffset) + .setReadLimit(1024) + .build()), + any()); + assertArrayEquals( + fakeService.data.substring(readOffset, readOffset + (2 * 1024)).toByteArray(), + buffer.array()); // reading the footer twice to ensure there are no additional calls to GCS buffer.clear(); @@ -1069,31 +1156,45 @@ public void testReadCachedFooterPartiallyWithInplaceSeek() throws Exception { verify(get).execute(); int footerOffset = 14 * 1024; - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(footerOffset) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(footerOffset) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(footerOffset) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(footerOffset) + .build()), + any()); buffer.clear(); int readOffset = 13 * 1024; readChannel.position(readOffset); readChannel.read(buffer); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadOffset(readOffset) - .setReadLimit(1024) - .build()), any()); - assertArrayEquals(fakeService.data.substring(readOffset, readOffset + (2 * 1024)).toByteArray(), buffer.array()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadOffset(readOffset) + .setReadLimit(1024) + .build()), + any()); + assertArrayEquals( + fakeService.data.substring(readOffset, readOffset + (2 * 1024)).toByteArray(), + buffer.array()); // reading the footer twice to ensure there are no additional calls to GCS buffer.clear(); @@ -1126,17 +1227,25 @@ public void testReadWithInplaceSeekAndFadviseRandom() throws Exception { verify(get).execute(); int footerOffset = 14 * 1024; - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(footerOffset) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(inplaceSeekLimit) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(footerOffset) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(inplaceSeekLimit) + .build()), + any()); buffer.clear(); int readOffset = 7 * 1024; @@ -1146,14 +1255,19 @@ public void testReadWithInplaceSeekAndFadviseRandom() throws Exception { readChannel.read(buffer); // verify new request after seek - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadOffset(readOffset) - .setReadLimit(inplaceSeekLimit) - .build()), any()); - assertArrayEquals(fakeService.data.substring(readOffset, readOffset + (capacity)).toByteArray(), + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadOffset(readOffset) + .setReadLimit(inplaceSeekLimit) + .build()), + any()); + assertArrayEquals( + fakeService.data.substring(readOffset, readOffset + (capacity)).toByteArray(), buffer.array()); verifyNoMoreInteractions(fakeService); @@ -1181,17 +1295,25 @@ public void testReadWithInplaceSeekAndFadviseAuto() throws Exception { verify(get).execute(); int footerOffset = 14 * 1024; - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(footerOffset) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(footerOffset) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(footerOffset) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(footerOffset) + .build()), + any()); buffer.clear(); int readOffset = 7 * 1024; @@ -1200,7 +1322,8 @@ public void testReadWithInplaceSeekAndFadviseAuto() throws Exception { buffer = ByteBuffer.allocate(capacity); readChannel.read(buffer); - assertArrayEquals(fakeService.data.substring(readOffset, readOffset + (capacity)).toByteArray(), + assertArrayEquals( + fakeService.data.substring(readOffset, readOffset + (capacity)).toByteArray(), buffer.array()); verifyNoMoreInteractions(fakeService); @@ -1228,17 +1351,25 @@ public void testReadWithInplaceSeekAndFadviseSequential() throws Exception { verify(get).execute(); int footerOffset = 14 * 1024; - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setReadOffset(footerOffset) - .build()), any()); - verify(fakeService, times(1)).getObjectMedia(eq(GetObjectMediaRequest.newBuilder() - .setBucket(BUCKET_NAME) - .setObject(OBJECT_NAME) - .setGeneration(OBJECT_GENERATION) - .setReadLimit(footerOffset) - .build()), any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setReadOffset(footerOffset) + .build()), + any()); + verify(fakeService, times(1)) + .readObject( + eq( + ReadObjectRequest.newBuilder() + .setBucket(BUCKET_NAME) + .setObject(OBJECT_NAME) + .setGeneration(OBJECT_GENERATION) + .setReadLimit(footerOffset) + .build()), + any()); buffer.clear(); int readOffset = 7 * 1024; @@ -1247,7 +1378,8 @@ public void testReadWithInplaceSeekAndFadviseSequential() throws Exception { buffer = ByteBuffer.allocate(capacity); readChannel.read(buffer); - assertArrayEquals(fakeService.data.substring(readOffset, readOffset + (capacity)).toByteArray(), + assertArrayEquals( + fakeService.data.substring(readOffset, readOffset + (capacity)).toByteArray(), buffer.array()); verifyNoMoreInteractions(fakeService); @@ -1380,8 +1512,8 @@ private GoogleCloudStorageGrpcReadChannel newReadChannel() throws IOException { return newReadChannel(GoogleCloudStorageReadOptions.DEFAULT); } - private GoogleCloudStorageGrpcReadChannel newReadChannel(Storage storage, GoogleCloudStorageReadOptions options) - throws IOException { + private GoogleCloudStorageGrpcReadChannel newReadChannel( + Storage storage, GoogleCloudStorageReadOptions options) throws IOException { return GoogleCloudStorageGrpcReadChannel.open( new FakeStubProvider(mockCredentials), storage, @@ -1397,13 +1529,13 @@ private GoogleCloudStorageGrpcReadChannel newReadChannel(GoogleCloudStorageReadO new FakeStubProvider(mockCredentials), storage, errorExtractor, - new StorageResourceId(BUCKET_NAME, OBJECT_NAME), + new StorageResourceId(V1_BUCKET_NAME, OBJECT_NAME), options, () -> BackOff.STOP_BACKOFF); } - private GoogleCloudStorageGrpcReadChannel newReadChannel(StorageResourceId storageResourceId, - GoogleCloudStorageReadOptions options) + private GoogleCloudStorageGrpcReadChannel newReadChannel( + StorageResourceId storageResourceId, GoogleCloudStorageReadOptions options) throws IOException { return GoogleCloudStorageGrpcReadChannel.open( new FakeStubProvider(mockCredentials), @@ -1443,8 +1575,7 @@ private static class FakeService extends StorageImplBase { private static final int CHUNK_SIZE = 2048; ByteString data; private Object object; - private Throwable getException; - private Throwable getMediaException; + private Throwable readObjectException; private boolean alterMessageChecksum = false; public FakeService() { @@ -1461,20 +1592,10 @@ private static ByteString createTestData(int numBytes) { } @Override - public void getObject(GetObjectRequest request, StreamObserver responseObserver) { - if (getException != null) { - responseObserver.onError(getException); - } else { - responseObserver.onNext(object); - responseObserver.onCompleted(); - } - } - - @Override - public void getObjectMedia( - GetObjectMediaRequest request, StreamObserver responseObserver) { - if (getMediaException != null) { - responseObserver.onError(getMediaException); + public void readObject( + ReadObjectRequest request, StreamObserver responseObserver) { + if (readObjectException != null) { + responseObserver.onError(readObjectException); } else { int readStart = (int) request.getReadOffset(); int readEnd = @@ -1484,18 +1605,15 @@ public void getObjectMedia( for (int position = readStart; position < readEnd; position += CHUNK_SIZE) { int endIndex = Math.min((int) object.getSize(), position + CHUNK_SIZE); endIndex = Math.min(endIndex, readEnd); - ByteString messageData = - data.substring(position, endIndex); + ByteString messageData = data.substring(position, endIndex); int crc32c = Hashing.crc32c().hashBytes(messageData.toByteArray()).asInt(); if (alterMessageChecksum) { crc32c += 1; } - GetObjectMediaResponse response = - GetObjectMediaResponse.newBuilder() + ReadObjectResponse response = + ReadObjectResponse.newBuilder() .setChecksummedData( - ChecksummedData.newBuilder() - .setContent(messageData) - .setCrc32C(UInt32Value.newBuilder().setValue(crc32c))) + ChecksummedData.newBuilder().setContent(messageData).setCrc32C((crc32c))) .build(); responseObserver.onNext(response); } @@ -1508,12 +1626,8 @@ public void setObject(Object object) { data = createTestData((int) object.getSize()); } - void setGetException(Throwable t) { - getException = t; - } - - void setGetMediaException(Throwable t) { - getMediaException = t; + void setReadObjectException(Throwable t) { + readObjectException = t; } void setReturnIncorrectMessageChecksum() { diff --git a/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcWriteChannelTest.java b/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcWriteChannelTest.java index 69fd8a9712..7e8f842e32 100644 --- a/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcWriteChannelTest.java +++ b/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageGrpcWriteChannelTest.java @@ -19,25 +19,24 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.google.storage.v1.ChecksummedData; -import com.google.google.storage.v1.InsertObjectRequest; -import com.google.google.storage.v1.InsertObjectSpec; -import com.google.google.storage.v1.Object; -import com.google.google.storage.v1.ObjectChecksums; -import com.google.google.storage.v1.QueryWriteStatusRequest; -import com.google.google.storage.v1.QueryWriteStatusResponse; -import com.google.google.storage.v1.StartResumableWriteRequest; -import com.google.google.storage.v1.StartResumableWriteResponse; -import com.google.google.storage.v1.StorageGrpc; -import com.google.google.storage.v1.StorageGrpc.StorageImplBase; -import com.google.google.storage.v1.StorageGrpc.StorageStub; import com.google.protobuf.ByteString; -import com.google.protobuf.Int64Value; import com.google.protobuf.Timestamp; -import com.google.protobuf.UInt32Value; +import com.google.storage.v2.ChecksummedData; +import com.google.storage.v2.Object; +import com.google.storage.v2.ObjectChecksums; +import com.google.storage.v2.QueryWriteStatusRequest; +import com.google.storage.v2.QueryWriteStatusResponse; +import com.google.storage.v2.StartResumableWriteRequest; +import com.google.storage.v2.StartResumableWriteResponse; +import com.google.storage.v2.StorageGrpc; +import com.google.storage.v2.StorageGrpc.StorageImplBase; +import com.google.storage.v2.StorageGrpc.StorageStub; +import com.google.storage.v2.WriteObjectRequest; +import com.google.storage.v2.WriteObjectResponse; +import com.google.storage.v2.WriteObjectSpec; +import io.grpc.ManagedChannelBuilder; import io.grpc.Status; import io.grpc.StatusException; -import io.grpc.ManagedChannelBuilder; import io.grpc.inprocess.InProcessChannelBuilder; import io.grpc.inprocess.InProcessServerBuilder; import io.grpc.stub.AbstractStub; @@ -45,7 +44,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; -import java.util.Arrays; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -58,18 +56,20 @@ import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.MockitoAnnotations; + @RunWith(JUnit4.class) public final class GoogleCloudStorageGrpcWriteChannelTest { private static final int GCS_MINIMUM_CHUNK_SIZE = 256 * 1024; - private static final String BUCKET_NAME = "bucket-name"; + private static final String V1_BUCKET_NAME = "bucket-name"; + private static final String BUCKET_NAME = GrpcChannelUtils.toV2BucketName(V1_BUCKET_NAME); private static final String OBJECT_NAME = "object-name"; private static final String UPLOAD_ID = "upload-id"; private static final String CONTENT_TYPE = "image/jpeg"; private static final StartResumableWriteRequest START_REQUEST = StartResumableWriteRequest.newBuilder() - .setInsertObjectSpec( - InsertObjectSpec.newBuilder() + .setWriteObjectSpec( + WriteObjectSpec.newBuilder() .setResource( Object.newBuilder() .setBucket(BUCKET_NAME) @@ -116,16 +116,12 @@ public void writeSendsSingleInsertObjectRequestWithChecksums() throws Exception writeChannel.write(data.asReadOnlyByteBuffer()); writeChannel.close(); - InsertObjectRequest expectedInsertRequest = - InsertObjectRequest.newBuilder() + WriteObjectRequest expectedInsertRequest = + WriteObjectRequest.newBuilder() .setUploadId(UPLOAD_ID) .setChecksummedData( - ChecksummedData.newBuilder() - .setContent(data) - .setCrc32C(UInt32Value.newBuilder().setValue(uInt32Value(863614154)))) - .setObjectChecksums( - ObjectChecksums.newBuilder() - .setCrc32C(UInt32Value.newBuilder().setValue(uInt32Value(863614154)))) + ChecksummedData.newBuilder().setContent(data).setCrc32C((uInt32Value(863614154)))) + .setObjectChecksums(ObjectChecksums.newBuilder().setCrc32C((uInt32Value(863614154)))) .setFinishWrite(true) .build(); @@ -147,8 +143,8 @@ public void writeSendsSingleInsertObjectRequestWithoutChecksums() throws Excepti writeChannel.write(data.asReadOnlyByteBuffer()); writeChannel.close(); - InsertObjectRequest expectedInsertRequest = - InsertObjectRequest.newBuilder() + WriteObjectRequest expectedInsertRequest = + WriteObjectRequest.newBuilder() .setUploadId(UPLOAD_ID) .setChecksummedData(ChecksummedData.newBuilder().setContent(data)) .setFinishWrite(true) @@ -177,8 +173,8 @@ public void writeSendsMultipleInsertObjectRequests() throws Exception { writeChannel.write(data.asReadOnlyByteBuffer()); writeChannel.close(); - ArgumentCaptor requestCaptor = - ArgumentCaptor.forClass(InsertObjectRequest.class); + ArgumentCaptor requestCaptor = + ArgumentCaptor.forClass(WriteObjectRequest.class); verify(fakeService, times(1)).startResumableWrite(eq(START_REQUEST), any()); verify(fakeService.insertRequestObserver, times(1)).onNext(requestCaptor.capture()); @@ -207,8 +203,8 @@ public void writeSendsMultipleInsertObjectRequestsWithChecksums() throws Excepti writeChannel.write(data.asReadOnlyByteBuffer()); writeChannel.close(); - ArgumentCaptor requestCaptor = - ArgumentCaptor.forClass(InsertObjectRequest.class); + ArgumentCaptor requestCaptor = + ArgumentCaptor.forClass(WriteObjectRequest.class); verify(fakeService, times(1)).startResumableWrite(eq(START_REQUEST), any()); verify(fakeService.insertRequestObserver, times(1)).onNext(requestCaptor.capture()); @@ -230,8 +226,8 @@ public void writeHandlesUncommittedData() throws Exception { writeChannel.write(data.asReadOnlyByteBuffer()); writeChannel.close(); - ArgumentCaptor requestCaptor = - ArgumentCaptor.forClass(InsertObjectRequest.class); + ArgumentCaptor requestCaptor = + ArgumentCaptor.forClass(WriteObjectRequest.class); verify(fakeService, times(1)).startResumableWrite(eq(START_REQUEST), any()); // TODO(b/150892988): Use this mock when implement resuming after a transient error. @@ -254,9 +250,7 @@ public void writeUsesContentGenerationIfProvided() throws Exception { writeChannel.close(); StartResumableWriteRequest.Builder expectedRequestBuilder = START_REQUEST.toBuilder(); - expectedRequestBuilder - .getInsertObjectSpecBuilder() - .setIfGenerationMatch(Int64Value.newBuilder().setValue(1L)); + expectedRequestBuilder.getWriteObjectSpecBuilder().setIfGenerationMatch(1L); verify(fakeService, times(1)).startResumableWrite(eq(expectedRequestBuilder.build()), any()); } @@ -274,9 +268,7 @@ public void writeUsesMetaGenerationIfProvided() throws Exception { writeChannel.close(); StartResumableWriteRequest.Builder expectedRequestBuilder = START_REQUEST.toBuilder(); - expectedRequestBuilder - .getInsertObjectSpecBuilder() - .setIfMetagenerationMatch(Int64Value.newBuilder().setValue(1L)); + expectedRequestBuilder.getWriteObjectSpecBuilder().setIfMetagenerationMatch(1L); verify(fakeService, times(1)).startResumableWrite(eq(expectedRequestBuilder.build()), any()); } @@ -293,7 +285,7 @@ public void writeUsesRequesterPaysProjectIfProvided() throws Exception { writeChannel.close(); StartResumableWriteRequest.Builder expectedRequestBuilder = START_REQUEST.toBuilder(); - expectedRequestBuilder.getInsertObjectSpecBuilder().setUserProject("project-id"); + expectedRequestBuilder.getCommonRequestParamsBuilder().setUserProject("project-id"); verify(fakeService, times(1)).startResumableWrite(eq(expectedRequestBuilder.build()), any()); } @@ -379,21 +371,8 @@ public void writeOneChunkWithSingleErrorAndResume() throws Exception { ImmutableList.of(QueryWriteStatusResponse.newBuilder().setCommittedSize(1).build()) .iterator()); ByteString chunk = createTestData(GCS_MINIMUM_CHUNK_SIZE); - List expectedRequests = - Arrays.asList( - InsertObjectRequest.newBuilder() - .setUploadId(UPLOAD_ID) - .setChecksummedData( - ChecksummedData.newBuilder() - .setContent(chunk) - .setCrc32C(UInt32Value.newBuilder().setValue(uInt32Value(1916767651)))) - .setObjectChecksums( - ObjectChecksums.newBuilder() - .setCrc32C(UInt32Value.newBuilder().setValue(uInt32Value(1916767651)))) - .setFinishWrite(true) - .build()); - ArgumentCaptor requestCaptor = - ArgumentCaptor.forClass(InsertObjectRequest.class); + ArgumentCaptor requestCaptor = + ArgumentCaptor.forClass(WriteObjectRequest.class); writeChannel.initialize(); writeChannel.write(chunk.asReadOnlyByteBuffer()); @@ -487,17 +466,27 @@ public void closeFailsBeforeInitilize() { @Test public void getItemInfoReturnsCorrectItemInfo() throws Exception { + byte[] expectedMd5Hash = { + -109, 66, -75, 122, -93, -111, 86, -26, 54, -45, -55, -64, 0, 58, 115, -21 + }; + byte[] expectedCrc32C = {51, 121, -76, -54}; + fakeService.setObject( - FakeService.DEFAULT_OBJECT.toBuilder() + FakeService.DEFAULT_OBJECT + .toBuilder() .setSize(9) .setGeneration(1) .setMetageneration(2) - .setTimeCreated(Timestamp.newBuilder().setSeconds(1560485630).setNanos(7000000)) - .setUpdated(Timestamp.newBuilder().setSeconds(1560495630).setNanos(123000000)) + .setCreateTime(Timestamp.newBuilder().setSeconds(1560485630).setNanos(7000000)) + .setUpdateTime(Timestamp.newBuilder().setSeconds(1560495630).setNanos(123000000)) + .setContentType(CONTENT_TYPE) .setContentEncoding("content-encoding") .putMetadata("metadata-key-1", "dGVzdC1tZXRhZGF0YQ==") - .setMd5Hash("k0K1eqORVuY208nAADpz6w==") - .setCrc32C(UInt32Value.newBuilder().setValue(uInt32Value(863614154))) + .setChecksums( + ObjectChecksums.newBuilder() + .setMd5Hash(ByteString.copyFrom(expectedMd5Hash)) + .setCrc32C(uInt32Value(863614154)) + .build()) .build()); GoogleCloudStorageGrpcWriteChannel writeChannel = newWriteChannel(); @@ -507,17 +496,13 @@ public void getItemInfoReturnsCorrectItemInfo() throws Exception { writeChannel.close(); GoogleCloudStorageItemInfo itemInfo = writeChannel.getItemInfo(); - byte[] expectedMd5Hash = { - -109, 66, -75, 122, -93, -111, 86, -26, 54, -45, -55, -64, 0, 58, 115, -21 - }; - byte[] expectedCrc32C = {51, 121, -76, -54}; Map expectedMetadata = ImmutableMap.of( "metadata-key-1", new byte[] {116, 101, 115, 116, 45, 109, 101, 116, 97, 100, 97, 116, 97}); GoogleCloudStorageItemInfo expectedItemInfo = GoogleCloudStorageItemInfo.createObject( - new StorageResourceId(BUCKET_NAME, OBJECT_NAME), + new StorageResourceId(V1_BUCKET_NAME, OBJECT_NAME), 1560485630007L, 1560495630123L, /* size= */ 9, @@ -583,7 +568,7 @@ private GoogleCloudStorageGrpcWriteChannel newWriteChannel( new FakeStubProvider(mockCredentials), executor, options, - new StorageResourceId(BUCKET_NAME, OBJECT_NAME), + new StorageResourceId(V1_BUCKET_NAME, OBJECT_NAME), CreateObjectOptions.DEFAULT_NO_OVERWRITE.toBuilder().setContentType(CONTENT_TYPE).build(), writeConditions, requesterPaysProject, @@ -617,6 +602,7 @@ private ByteString createTestData(int numBytes) { } private static class FakeGrpcDecorator implements StorageStubProvider.GrpcDecorator { + @Override public ManagedChannelBuilder createChannelBuilder(String target) { return null; @@ -629,6 +615,7 @@ public AbstractStub applyCallOption(AbstractStub stub) { } private class FakeStubProvider extends StorageStubProvider { + FakeStubProvider(Credentials credentials) { super(GoogleCloudStorageOptions.DEFAULT, null, new FakeGrpcDecorator()); } @@ -640,6 +627,7 @@ public StorageStub newAsyncStub() { } private static class FakeService extends StorageImplBase { + static final Object DEFAULT_OBJECT = Object.newBuilder() .setBucket(BUCKET_NAME) @@ -683,8 +671,8 @@ public void queryWriteStatus( } @Override - public StreamObserver insertObject( - StreamObserver responseObserver) { + public StreamObserver writeObject( + StreamObserver responseObserver) { if (insertObjectExceptions != null && insertObjectExceptions.size() > 0) { Throwable throwable = insertObjectExceptions.remove(0); if (!throwable.getClass().isAssignableFrom(Throwable.class) @@ -699,6 +687,8 @@ public StreamObserver insertObject( public void setObject(Object object) { this.insertRequestObserver.object = object; + this.insertRequestObserver.writeObjectResponse = + WriteObjectResponse.newBuilder().setResource(object).build(); } void setQueryWriteStatusResponses(Iterator responses) { @@ -718,26 +708,29 @@ void setInsertRequestException(Throwable t) { } public void setInsertObjectExceptions(List insertObjectExceptions) { - // Make a copy so caller can pass in an immutable list (this implementation needs to update - // the list). + // Make a copy so caller can pass in an immutable list (this implementation needs to + // update the list). this.insertObjectExceptions = Lists.newArrayList(insertObjectExceptions); } - private static class InsertRequestObserver implements StreamObserver { - private StreamObserver responseObserver; + private static class InsertRequestObserver implements StreamObserver { + + private StreamObserver responseObserver; private Object object = DEFAULT_OBJECT; + WriteObjectResponse writeObjectResponse = + WriteObjectResponse.newBuilder().setResource(object).build(); Throwable insertRequestException; boolean resumeFromInsertException = false; @Override - public void onNext(InsertObjectRequest request) { + public void onNext(WriteObjectRequest request) { if (insertRequestException != null) { responseObserver.onError(insertRequestException); if (resumeFromInsertException) { insertRequestException = null; } } else { - responseObserver.onNext(object); + responseObserver.onNext(writeObjectResponse); } } diff --git a/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/GrpcChannelUtilsTest.java b/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/GrpcChannelUtilsTest.java new file mode 100644 index 0000000000..8f1ce11501 --- /dev/null +++ b/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/GrpcChannelUtilsTest.java @@ -0,0 +1,46 @@ +/* + * Copyright 2021 Google Inc. All Rights Reserved. + * + * Licensed 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 com.google.cloud.hadoop.gcsio; + +import static com.google.cloud.hadoop.gcsio.GrpcChannelUtils.V2_BUCKET_NAME_PREFIX; +import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertThrows; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class GrpcChannelUtilsTest { + + @Test + public void toV2BucketNameConvertsV1BucketName() { + String bucketName = "bucketName"; + String v2BucketName = GrpcChannelUtils.toV2BucketName(bucketName); + assertThat(v2BucketName).isNotNull(); + assertThat(v2BucketName).isNotEmpty(); + assertThat(v2BucketName).isEqualTo(V2_BUCKET_NAME_PREFIX + bucketName); + } + + @Test + public void toV2BucketNameThrowsExceptionOnNull() { + assertThrows(IllegalArgumentException.class, () -> GrpcChannelUtils.toV2BucketName(null)); + } + + @Test + public void toV2BucketNameThrowsExceptionOnEmpty() { + assertThrows(IllegalArgumentException.class, () -> GrpcChannelUtils.toV2BucketName("")); + } +} diff --git a/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/ZeroCopyMessageMarshallerTest.java b/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/ZeroCopyMessageMarshallerTest.java index b2b7671036..96b734c030 100644 --- a/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/ZeroCopyMessageMarshallerTest.java +++ b/gcsio/src/test/java/com/google/cloud/hadoop/gcsio/ZeroCopyMessageMarshallerTest.java @@ -17,7 +17,7 @@ import static com.google.common.truth.Truth.assertThat; import static org.junit.Assert.assertThrows; -import com.google.google.storage.v1.GetObjectRequest; +import com.google.storage.v2.ReadObjectRequest; import io.grpc.StatusRuntimeException; import io.grpc.internal.ReadableBuffer; import io.grpc.internal.ReadableBuffers; @@ -31,11 +31,11 @@ @RunWith(JUnit4.class) public class ZeroCopyMessageMarshallerTest { - private GetObjectRequest REQUEST = - GetObjectRequest.newBuilder().setBucket("b").setObject("o").build(); + private ReadObjectRequest REQUEST = + ReadObjectRequest.newBuilder().setBucket("b").setObject("o").build(); - private ZeroCopyMessageMarshaller createMarshaller() { - return new ZeroCopyMessageMarshaller<>(GetObjectRequest.getDefaultInstance()); + private ZeroCopyMessageMarshaller createMarshaller() { + return new ZeroCopyMessageMarshaller<>(ReadObjectRequest.getDefaultInstance()); } private byte[] dropLastOneByte(byte[] bytes) { @@ -51,8 +51,8 @@ private InputStream createInputStream(byte[] bytes, boolean isZeroCopyable) { @Test public void testParseOnFastPath() throws IOException { InputStream stream = createInputStream(REQUEST.toByteArray(), true); - ZeroCopyMessageMarshaller marshaller = createMarshaller(); - GetObjectRequest request = marshaller.parse(stream); + ZeroCopyMessageMarshaller marshaller = createMarshaller(); + ReadObjectRequest request = marshaller.parse(stream); assertThat(request).isEqualTo(REQUEST); InputStream stream2 = marshaller.popStream(request); assertThat(stream2).isNotNull(); @@ -64,8 +64,8 @@ public void testParseOnFastPath() throws IOException { @Test public void testParseOnSlowPath() { InputStream stream = createInputStream(REQUEST.toByteArray(), false); - ZeroCopyMessageMarshaller marshaller = createMarshaller(); - GetObjectRequest request = marshaller.parse(stream); + ZeroCopyMessageMarshaller marshaller = createMarshaller(); + ReadObjectRequest request = marshaller.parse(stream); assertThat(request).isEqualTo(REQUEST); InputStream stream2 = marshaller.popStream(request); assertThat(stream2).isNull(); @@ -74,7 +74,7 @@ public void testParseOnSlowPath() { @Test public void testParseBrokenMessageOnFastPath() { InputStream stream = createInputStream(dropLastOneByte(REQUEST.toByteArray()), true); - ZeroCopyMessageMarshaller marshaller = createMarshaller(); + ZeroCopyMessageMarshaller marshaller = createMarshaller(); assertThrows( StatusRuntimeException.class, () -> { @@ -85,7 +85,7 @@ public void testParseBrokenMessageOnFastPath() { @Test public void testParseBrokenMessageOnSlowPath() { InputStream stream = createInputStream(dropLastOneByte(REQUEST.toByteArray()), false); - ZeroCopyMessageMarshaller marshaller = createMarshaller(); + ZeroCopyMessageMarshaller marshaller = createMarshaller(); assertThrows( StatusRuntimeException.class, () -> { diff --git a/pom.xml b/pom.xml index 0e8d4b0f67..bf825e8ee9 100644 --- a/pom.xml +++ b/pom.xml @@ -93,6 +93,7 @@ 0.26.0 1.8.1 1.22.5 + 2.0.1-alpha 0.6 2.8.7 30.1.1-jre @@ -481,6 +482,11 @@ ${google.flogger.version} runtime + + com.google.api.grpc + grpc-google-cloud-storage-v2 + ${google.cloud-storage.grpc.version} + @@ -619,24 +625,6 @@ - - kr.motd.maven - os-maven-plugin - 1.7.0 - - - initialize - - detect - - - - - - org.xolstice.maven.plugins - protobuf-maven-plugin - 0.6.1 -