From 1b52a1053130620011515060787bada10c324c0b Mon Sep 17 00:00:00 2001 From: BenWhitehead Date: Thu, 13 Jul 2023 19:20:35 -0400 Subject: [PATCH] feat: BlobWriteChannelV2 - same throughput less GC (#2110) Use stable buffer allocation with laziness. Leverage new JsonResumableSession to provide more robustness and easier separation of concerns compared to BlobWriteChannel * rename blobWriteChannel.ser.properties to the correct blobReadChannel.ser.properties ### Runtime improvments Throughput is on par with the existing v1 implementation, however GC impact has been lightened with the new implementation. Below is the summary of the GC improvement between v1 and v2. These GC numbers were collected while uploading 4096 randomly sized objects, from 128KiB..2GiB across 16 concurrent threads, using a default chunkSize of 16MiB. | metric | unit | v1 | v2 | % decrease | |---------------------------------|--------|-------------:|-------------:|-----------:| | gc.alloc.rate | MB/sec | 2240.056 | 1457.731 | 34.924 | | gc.alloc.rate.norm | B/op | 955796726217 | 638403730507 | 33.207 | | gc.churn.G1_Eden_Space | MB/sec | 1597.009 | 1454.304 | 8.936 | | gc.churn.G1_Eden_Space.norm | B/op | 681418424320 | 636902965248 | 6.533 | | gc.churn.G1_Old_Gen | MB/sec | 691.877 | 11.316 | 98.364 | | gc.churn.G1_Old_Gen.norm | B/op | 295213237398 | 4955944331 | 98.321 | | gc.churn.G1_Survivor_Space | MB/sec | 0.004 | 0.002 | 50.000 | | gc.churn.G1_Survivor_Space.norm | B/op | 1572864 | 786432 | 50.000 | | gc.count | counts | 1670 | 1319 | 21.018 | | gc.time | ms | 15936 | 9527 | 40.217 | Overall allocation rate is decreased, while Old_Gen use is almost entirely eliminated. ``` openjdk version "11.0.18" 2023-01-17 OpenJDK Runtime Environment (build 11.0.18+10-post-Debian-1deb11u1) OpenJDK 64-Bit Server VM (build 11.0.18+10-post-Debian-1deb11u1, mixed mode, sharing) -Xms12g -Xmx12g ``` All other java parameters are defaults. --- .../clirr-ignored-differences.xml | 4 +- .../ApiaryUnbufferedWritableByteChannel.java | 109 ++ .../storage/BaseStorageWriteChannel.java | 178 ++++ .../cloud/storage/BlobReadChannelV2.java | 18 +- .../cloud/storage/BlobWriteChannel.java | 367 +------ .../cloud/storage/BlobWriteChannelV2.java | 153 +++ .../com/google/cloud/storage/Buffers.java | 4 + .../com/google/cloud/storage/Conversions.java | 4 + .../GapicUnbufferedWritableByteChannel.java | 10 - .../cloud/storage/GrpcBlobWriteChannel.java | 119 +-- .../cloud/storage/HttpContentRange.java | 22 + .../storage/HttpDownloadSessionBuilder.java | 2 +- .../storage/HttpUploadSessionBuilder.java | 34 + ...HttpWritableByteChannelSessionBuilder.java | 190 ++++ .../cloud/storage/JsonResumableSession.java | 8 +- .../storage/JsonResumableSessionPutTask.java | 25 +- .../cloud/storage/JsonResumableWrite.java | 38 +- .../cloud/storage/LazyWriteChannel.java | 69 ++ .../google/cloud/storage/ResumableMedia.java | 4 + .../com/google/cloud/storage/Retrying.java | 11 +- .../cloud/storage/RewindableHttpContent.java | 84 +- .../com/google/cloud/storage/StorageImpl.java | 72 +- .../cloud/storage/StorageWriteChannel.java | 24 + .../UnbufferedWritableByteChannelSession.java | 14 +- .../java/com/google/cloud/storage/Utils.java | 9 + .../cloud/storage/spi/v1/HttpStorageRpc.java | 9 +- .../com/google/cloud/storage/BlobTest.java | 38 - .../cloud/storage/BlobWriteChannelTest.java | 961 ------------------ .../ITJsonResumableSessionPutTaskTest.java | 24 + .../storage/ITJsonResumableSessionTest.java | 141 ++- ...onResumableSessionFailureScenarioTest.java | 2 +- .../PackagePrivateMethodWorkarounds.java | 14 +- .../RewindableByteBufferContentTest.java | 160 +++ .../RewindableHttpContentPropertyTest.java | 164 ++- .../cloud/storage/SerializationTest.java | 52 +- .../cloud/storage/StorageImplMockitoTest.java | 206 ---- .../com/google/cloud/storage/TestUtils.java | 18 +- .../com/google/cloud/storage/TmpFile.java | 6 + .../retry/ITRetryConformanceTest.java | 5 + .../storage/it/ITBlobWriteChannelTest.java | 187 +--- .../cloud/storage/it/TemporaryBucket.java | 18 +- .../storage/blobReadChannel.ser.properties | 70 ++ .../storage/blobWriteChannel.ser.properties | 78 +- 43 files changed, 1802 insertions(+), 1923 deletions(-) create mode 100644 google-cloud-storage/src/main/java/com/google/cloud/storage/ApiaryUnbufferedWritableByteChannel.java create mode 100644 google-cloud-storage/src/main/java/com/google/cloud/storage/BaseStorageWriteChannel.java create mode 100644 google-cloud-storage/src/main/java/com/google/cloud/storage/BlobWriteChannelV2.java create mode 100644 google-cloud-storage/src/main/java/com/google/cloud/storage/HttpUploadSessionBuilder.java create mode 100644 google-cloud-storage/src/main/java/com/google/cloud/storage/HttpWritableByteChannelSessionBuilder.java create mode 100644 google-cloud-storage/src/main/java/com/google/cloud/storage/LazyWriteChannel.java create mode 100644 google-cloud-storage/src/main/java/com/google/cloud/storage/StorageWriteChannel.java delete mode 100644 google-cloud-storage/src/test/java/com/google/cloud/storage/BlobWriteChannelTest.java create mode 100644 google-cloud-storage/src/test/java/com/google/cloud/storage/RewindableByteBufferContentTest.java create mode 100644 google-cloud-storage/src/test/resources/com/google/cloud/storage/blobReadChannel.ser.properties diff --git a/google-cloud-storage/clirr-ignored-differences.xml b/google-cloud-storage/clirr-ignored-differences.xml index 703b8ad65..ad681c4be 100644 --- a/google-cloud-storage/clirr-ignored-differences.xml +++ b/google-cloud-storage/clirr-ignored-differences.xml @@ -4,8 +4,8 @@ 7012 - com/google/cloud/storage/UnbufferedReadableByteChannelSession$UnbufferedReadableByteChannel - * read(*) + com/google/cloud/storage/UnbufferedWritableByteChannelSession$UnbufferedWritableByteChannel + * write(*) diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/ApiaryUnbufferedWritableByteChannel.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/ApiaryUnbufferedWritableByteChannel.java new file mode 100644 index 000000000..5fce18ae5 --- /dev/null +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/ApiaryUnbufferedWritableByteChannel.java @@ -0,0 +1,109 @@ +/* + * Copyright 2023 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. + */ + +package com.google.cloud.storage; + +import com.google.api.core.SettableApiFuture; +import com.google.api.gax.retrying.ResultRetryAlgorithm; +import com.google.api.services.storage.model.StorageObject; +import com.google.cloud.storage.Retrying.RetryingDependencies; +import com.google.cloud.storage.UnbufferedWritableByteChannelSession.UnbufferedWritableByteChannel; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; +import java.util.function.LongConsumer; +import javax.annotation.ParametersAreNonnullByDefault; +import org.checkerframework.checker.nullness.qual.Nullable; + +@ParametersAreNonnullByDefault +final class ApiaryUnbufferedWritableByteChannel implements UnbufferedWritableByteChannel { + + private final ResumableSession session; + + private final SettableApiFuture result; + private final LongConsumer committedBytesCallback; + + private boolean open = true; + private long cumulativeByteCount; + private boolean finished = false; + + ApiaryUnbufferedWritableByteChannel( + HttpClientContext httpClientContext, + RetryingDependencies deps, + ResultRetryAlgorithm alg, + JsonResumableWrite resumableWrite, + SettableApiFuture result, + LongConsumer committedBytesCallback) { + this.session = ResumableSession.json(httpClientContext, deps, alg, resumableWrite); + this.result = result; + this.committedBytesCallback = committedBytesCallback; + } + + @Override + public long write(ByteBuffer[] srcs, int offset, int length) throws IOException { + if (!open) { + throw new ClosedChannelException(); + } + RewindableHttpContent content = RewindableHttpContent.of(Utils.subArray(srcs, offset, length)); + long available = content.getLength(); + long newFinalByteOffset = cumulativeByteCount + available; + final HttpContentRange header; + ByteRangeSpec rangeSpec = ByteRangeSpec.explicit(cumulativeByteCount, newFinalByteOffset); + if (available % ByteSizeConstants._256KiB == 0) { + header = HttpContentRange.of(rangeSpec); + } else { + header = HttpContentRange.of(rangeSpec, newFinalByteOffset); + finished = true; + } + try { + ResumableOperationResult<@Nullable StorageObject> operationResult = + session.put(content, header); + long persistedSize = operationResult.getPersistedSize(); + committedBytesCallback.accept(persistedSize); + this.cumulativeByteCount = persistedSize; + if (finished) { + StorageObject storageObject = operationResult.getObject(); + result.set(storageObject); + } + return available; + } catch (Exception e) { + result.setException(e); + throw StorageException.coalesce(e); + } + } + + @Override + public boolean isOpen() { + return open; + } + + @Override + public void close() throws IOException { + open = false; + if (!finished) { + try { + ResumableOperationResult<@Nullable StorageObject> operationResult = + session.put(RewindableHttpContent.empty(), HttpContentRange.of(cumulativeByteCount)); + long persistedSize = operationResult.getPersistedSize(); + committedBytesCallback.accept(persistedSize); + result.set(operationResult.getObject()); + } catch (Exception e) { + result.setException(e); + throw StorageException.coalesce(e); + } + } + } +} diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/BaseStorageWriteChannel.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/BaseStorageWriteChannel.java new file mode 100644 index 000000000..f5f076327 --- /dev/null +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/BaseStorageWriteChannel.java @@ -0,0 +1,178 @@ +/* + * Copyright 2023 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. + */ + +package com.google.cloud.storage; + +import static com.google.cloud.storage.ByteSizeConstants._16MiB; +import static com.google.cloud.storage.ByteSizeConstants._256KiB; + +import com.google.api.core.ApiFuture; +import com.google.api.core.ApiFutureCallback; +import com.google.api.core.ApiFutures; +import com.google.api.core.SettableApiFuture; +import com.google.cloud.storage.BufferedWritableByteChannelSession.BufferedWritableByteChannel; +import com.google.cloud.storage.Conversions.Decoder; +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.MoreExecutors; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; +import org.checkerframework.checker.nullness.qual.Nullable; + +abstract class BaseStorageWriteChannel implements StorageWriteChannel { + + private final Decoder objectDecoder; + private final SettableApiFuture result; + + private long position; + private boolean open; + private int chunkSize; + private LazyWriteChannel lazyWriteChannel; + private BufferHandle bufferHandle; + + /** + * This is tracked for compatibility with BlobWriteChannel, such that simply creating a writer + * will create an object. + * + *

In the future we should move away from this behavior, and only create an object if write is + * called. + */ + protected boolean writeCalledAtLeastOnce; + + protected BaseStorageWriteChannel(Decoder objectDecoder) { + this.objectDecoder = objectDecoder; + this.result = SettableApiFuture.create(); + this.open = true; + this.chunkSize = _16MiB; + this.writeCalledAtLeastOnce = false; + } + + @Override + public final synchronized void setChunkSize(int chunkSize) { + Preconditions.checkArgument(chunkSize > 0, "chunkSize must be > 0, received %d", chunkSize); + Preconditions.checkState( + bufferHandle == null || bufferHandle.position() == 0, + "unable to change chunk size with data buffered"); + this.chunkSize = chunkSize; + } + + @Override + public final synchronized boolean isOpen() { + return open; + } + + @Override + public final synchronized void close() throws IOException { + try { + if (open && !writeCalledAtLeastOnce) { + this.write(ByteBuffer.allocate(0)); + } + if (internalGetLazyChannel().isOpen()) { + StorageException.wrapIOException(internalGetLazyChannel().getChannel()::close); + } + } finally { + open = false; + } + } + + @Override + public final synchronized int write(ByteBuffer src) throws IOException { + if (!open) { + throw new ClosedChannelException(); + } + writeCalledAtLeastOnce = true; + try { + BufferedWritableByteChannel tmp = internalGetLazyChannel().getChannel(); + if (!tmp.isOpen()) { + return 0; + } + int write = tmp.write(src); + return write; + } catch (StorageException e) { + throw new IOException(e); + } catch (IOException e) { + throw e; + } catch (Exception e) { + throw new IOException(StorageException.coalesce(e)); + } + } + + @Override + public final ApiFuture getObject() { + return ApiFutures.transform(result, objectDecoder::decode, MoreExecutors.directExecutor()); + } + + protected final BufferHandle getBufferHandle() { + if (bufferHandle == null) { + bufferHandle = BufferHandle.allocate(Buffers.alignSize(getChunkSize(), _256KiB)); + } + return bufferHandle; + } + + protected final int getChunkSize() { + return chunkSize; + } + + @Nullable + protected final T getResolvedObject() { + if (result.isDone()) { + return StorageException.wrapFutureGet(result); + } else { + return null; + } + } + + protected final long getCommittedPosition() { + return position; + } + + protected final void setCommittedPosition(long l) { + position = l; + } + + protected final void setOpen(boolean isOpen) { + this.open = isOpen; + } + + protected abstract LazyWriteChannel newLazyWriteChannel(); + + private LazyWriteChannel internalGetLazyChannel() { + if (lazyWriteChannel == null) { + LazyWriteChannel tmp = newLazyWriteChannel(); + ApiFuture future = tmp.getSession().getResult(); + ApiFutures.addCallback( + future, + new ApiFutureCallback() { + @Override + public void onFailure(Throwable t) { + if (!result.isDone()) { + result.setException(t); + } + } + + @Override + public void onSuccess(T t) { + if (!result.isDone()) { + result.set(t); + } + } + }, + MoreExecutors.directExecutor()); + lazyWriteChannel = tmp; + } + return lazyWriteChannel; + } +} diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/BlobReadChannelV2.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/BlobReadChannelV2.java index 1f3c5b52c..82aba8b13 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/BlobReadChannelV2.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/BlobReadChannelV2.java @@ -143,15 +143,18 @@ public String toString() { static final class BlobReadChannelContext { private final HttpStorageOptions storageOptions; private final HttpRetryAlgorithmManager retryAlgorithmManager; + private final HttpClientContext httpClientContext; private final Storage apiaryClient; private BlobReadChannelContext( HttpStorageOptions storageOptions, - Storage apiaryClient, - HttpRetryAlgorithmManager retryAlgorithmManager) { + HttpRetryAlgorithmManager retryAlgorithmManager, + HttpClientContext httpClientContext, + Storage apiaryClient) { this.storageOptions = storageOptions; - this.apiaryClient = apiaryClient; this.retryAlgorithmManager = retryAlgorithmManager; + this.httpClientContext = httpClientContext; + this.apiaryClient = apiaryClient; } public HttpStorageOptions getStorageOptions() { @@ -162,13 +165,20 @@ public HttpRetryAlgorithmManager getRetryAlgorithmManager() { return retryAlgorithmManager; } + public HttpClientContext getHttpClientContext() { + return httpClientContext; + } + public Storage getApiaryClient() { return apiaryClient; } static BlobReadChannelContext from(HttpStorageOptions options) { return new BlobReadChannelContext( - options, options.getStorageRpcV1().getStorage(), options.getRetryAlgorithmManager()); + options, + options.getRetryAlgorithmManager(), + HttpClientContext.from(options.getStorageRpcV1()), + options.getStorageRpcV1().getStorage()); } static BlobReadChannelContext from(com.google.cloud.storage.Storage s) { diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/BlobWriteChannel.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/BlobWriteChannel.java index 7b1925879..28c4b8d1e 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/BlobWriteChannel.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/BlobWriteChannel.java @@ -16,371 +16,78 @@ package com.google.cloud.storage; -import static java.util.Objects.requireNonNull; -import static java.util.concurrent.Executors.callable; - -import com.google.api.core.InternalApi; +import com.google.api.core.ApiFuture; import com.google.api.gax.retrying.ResultRetryAlgorithm; import com.google.api.services.storage.model.StorageObject; import com.google.cloud.BaseWriteChannel; import com.google.cloud.RestorableState; -import com.google.cloud.RetryHelper; import com.google.cloud.WriteChannel; -import java.math.BigInteger; -import java.util.function.Function; -import java.util.function.Supplier; -import org.checkerframework.checker.nullness.qual.NonNull; - -/** Write channel implementation to upload Google Cloud Storage blobs. */ -class BlobWriteChannel extends BaseWriteChannel { - - private final ResultRetryAlgorithm algorithmForWrite; - // Detect if flushBuffer() is being retried or not. - // TODO: I don't think this is thread safe, and there's probably a better way to detect a retry - // occuring. - private boolean retrying = false; - private boolean checkingForLastChunk = false; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableMap; - // Contains metadata of the updated object or null if upload is not completed. - private StorageObject storageObject; - - BlobWriteChannel( - HttpStorageOptions storageOptions, - BlobInfo blobInfo, - String uploadId, - ResultRetryAlgorithm algorithmForWrite) { - super(storageOptions, blobInfo, uploadId); - this.algorithmForWrite = algorithmForWrite; - } +/** + * Hierarchy retained for {@link RestorableState#restore()}. Will be removed in next major version! + */ +@Deprecated +class BlobWriteChannel extends BaseWriteChannel + implements StorageWriteChannel { - boolean isRetrying() { - return retrying; + private BlobWriteChannel() { + super(null, null, null); + throw new IllegalStateException("Illegal method access"); } - StorageObject getStorageObject() { - return storageObject; + @Override + public ApiFuture getObject() { + throw new IllegalStateException("Illegal method access"); } @Override protected HttpStorageOptions getOptions() { - return (HttpStorageOptions) super.getOptions(); - } - - private StorageObject transmitChunk( - int chunkOffset, int chunkLength, long position, boolean last) { - return getOptions() - .getStorageRpcV1() - .writeWithResponse(getUploadId(), getBuffer(), chunkOffset, position, chunkLength, last); + throw new IllegalStateException("Illegal method access"); } - private long getRemotePosition() { - return getOptions().getStorageRpcV1().getCurrentUploadOffset(getUploadId()); - } - - private static StorageException unrecoverableState( - String uploadId, - int chunkOffset, - int chunkLength, - long localPosition, - long remotePosition, - boolean last) { - return unrecoverableState( - uploadId, - chunkOffset, - chunkLength, - localPosition, - remotePosition, - last, - "Unable to recover in upload.\nThis may be a symptom of multiple clients uploading to the same upload session."); - } - - private static StorageException errorResolvingMetadataLastChunk( - String uploadId, - int chunkOffset, - int chunkLength, - long localPosition, - long remotePosition, - boolean last) { - return unrecoverableState( - uploadId, - chunkOffset, - chunkLength, - localPosition, - remotePosition, - last, - "Unable to load object metadata to determine if last chunk was successfully written"); - } - - private static StorageException unrecoverableState( - String uploadId, - int chunkOffset, - int chunkLength, - long localPosition, - long remotePosition, - boolean last, - String message) { - StringBuilder sb = new StringBuilder(); - sb.append(message).append("\n\n"); - sb.append("For debugging purposes:\n"); - sb.append("uploadId: ").append(uploadId).append('\n'); - sb.append("chunkOffset: ").append(chunkOffset).append('\n'); - sb.append("chunkLength: ").append(chunkLength).append('\n'); - sb.append("localOffset: ").append(localPosition).append('\n'); - sb.append("remoteOffset: ").append(remotePosition).append('\n'); - sb.append("lastChunk: ").append(last).append("\n\n"); - return new StorageException(0, sb.toString()); - } - - // Retriable interruption occurred. - // Variables: - // chunk = getBuffer() - // localNextByteOffset == getPosition() - // chunkSize = getChunkSize() - // - // Case 1: localNextByteOffset == remoteNextByteOffset: - // Retrying the entire chunk - // - // Case 2: localNextByteOffset < remoteNextByteOffset - // && driftOffset < chunkSize: - // Upload progressed and localNextByteOffset is not in-sync with - // remoteNextByteOffset and driftOffset is less than chunkSize. - // driftOffset must be less than chunkSize for it to retry using - // chunk maintained in memory. - // Find the driftOffset by subtracting localNextByteOffset from - // remoteNextByteOffset. - // Use driftOffset to determine where to restart from using the chunk in - // memory. - // - // Case 3: localNextByteOffset < remoteNextByteOffset - // && driftOffset == chunkSize: - // Special case of Case 2. - // If chunkSize is equal to driftOffset then remoteNextByteOffset has moved on - // to the next chunk. - // - // Case 4: localNextByteOffset < remoteNextByteOffset - // && driftOffset > chunkSize: - // Throw exception as remoteNextByteOffset has drifted beyond the retriable - // chunk maintained in memory. This is not possible unless there's multiple - // clients uploading to the same resumable upload session. - // - // Case 5: localNextByteOffset > remoteNextByteOffset: - // For completeness, this case is not possible because it would require retrying - // a 400 status code which is not allowed. - // - // Case 6: remoteNextByteOffset==-1 && last == true - // Upload is complete and retry occurred in the "last" chunk. Data sent was - // received by the service. - // - // Case 7: remoteNextByteOffset==-1 && last == false && !checkingForLastChunk - // Not last chunk and are not checkingForLastChunk, allow for the client to - // catch up to final chunk which meets - // Case 6. - // - // Case 8: remoteNextByteOffset==-1 && last == false && checkingForLastChunk - // Not last chunk and checkingForLastChunk means this is the second time we - // hit this case, meaning the upload was completed by a different client. - // - // Case 9: Only possible if the client local offset continues beyond the remote - // offset which is not possible. - // @Override protected void flushBuffer(final int length, final boolean lastChunk) { - try { - Retrying.run( - getOptions(), - algorithmForWrite, - callable( - new Runnable() { - @Override - public void run() { - // Get remote offset from API - final long localPosition = getPosition(); - // For each request it should be possible to retry from its location in this code - final long remotePosition = isRetrying() ? getRemotePosition() : localPosition; - final int chunkOffset = (int) (remotePosition - localPosition); - final int chunkLength = length - chunkOffset; - final boolean uploadAlreadyComplete = remotePosition == -1; - // Enable isRetrying state to reduce number of calls to getRemotePosition() - if (!isRetrying()) { - retrying = true; - } - if (uploadAlreadyComplete && lastChunk) { - // Case 6 - // Request object metadata if not available - long totalBytes = getPosition() + length; - if (storageObject == null) { - storageObject = - getOptions() - .getStorageRpcV1() - .queryCompletedResumableUpload(getUploadId(), totalBytes); - } - // the following checks are defined here explicitly to provide a more - // informative if either storageObject is unable to be resolved or it's size is - // unable to be determined. This scenario is a very rare case of failure that - // can arise when packets are lost. - if (storageObject == null) { - throw errorResolvingMetadataLastChunk( - getUploadId(), - chunkOffset, - chunkLength, - localPosition, - remotePosition, - lastChunk); - } - // Verify that with the final chunk we match the blob length - BigInteger size = storageObject.getSize(); - if (size == null) { - throw errorResolvingMetadataLastChunk( - getUploadId(), - chunkOffset, - chunkLength, - localPosition, - remotePosition, - lastChunk); - } - if (size.longValue() != totalBytes) { - throw unrecoverableState( - getUploadId(), - chunkOffset, - chunkLength, - localPosition, - remotePosition, - lastChunk); - } - retrying = false; - } else if (uploadAlreadyComplete && !lastChunk && !checkingForLastChunk) { - // Case 7 - // Make sure this is the second to last chunk. - checkingForLastChunk = true; - // Continue onto next chunk in case this is the last chunk - } else if (localPosition <= remotePosition && chunkOffset < getChunkSize()) { - // Case 1 && Case 2 - // We are in a position to send a chunk - storageObject = - transmitChunk(chunkOffset, chunkLength, remotePosition, lastChunk); - retrying = false; - } else if (localPosition < remotePosition && chunkOffset == getChunkSize()) { - // Case 3 - // Continue to next chunk to catch up with remotePosition we are one chunk - // behind - retrying = false; - } else { - // Case 4 && Case 8 && Case 9 - throw unrecoverableState( - getUploadId(), - chunkOffset, - chunkLength, - localPosition, - remotePosition, - lastChunk); - } - } - }), - Function.identity()); - } catch (RetryHelper.RetryHelperException e) { - throw StorageException.translateAndThrow(e); - } + throw new IllegalStateException("Illegal method access"); } protected StateImpl.Builder stateBuilder() { - return StateImpl.builder(getOptions(), getEntity(), getUploadId()) - .setResultRetryAlgorithm(algorithmForWrite); + throw new IllegalStateException("Illegal method access"); } - static Builder newBuilder() { - return new Builder(); - } - - static final class Builder { - private HttpStorageOptions storageOptions; - private BlobInfo blobInfo; - private Supplier<@NonNull String> uploadIdSupplier; - private ResultRetryAlgorithm algorithmForWrite; - - public Builder setStorageOptions(HttpStorageOptions storageOptions) { - this.storageOptions = storageOptions; - return this; - } - - public Builder setBlobInfo(BlobInfo blobInfo) { - this.blobInfo = blobInfo; - return this; - } - - public Builder setUploadIdSupplier(Supplier uploadIdSupplier) { - this.uploadIdSupplier = uploadIdSupplier; - return this; - } - - public Builder setAlgorithmForWrite(ResultRetryAlgorithm algorithmForWrite) { - this.algorithmForWrite = algorithmForWrite; - return this; - } - - BlobWriteChannel build() { - String uploadId = requireNonNull(uploadIdSupplier, "uploadId must be non null").get(); - return new BlobWriteChannel( - requireNonNull(storageOptions, "storageOptions must be non null"), - blobInfo, - requireNonNull(uploadId, "uploadId must be non null"), - requireNonNull(algorithmForWrite, "algorithmForWrite must be non null")); - } + @Override + public RestorableState capture() { + throw new IllegalStateException("Illegal method access"); } + /** Retained for binary compatibility. Will be removed at next major version! */ + @SuppressWarnings("unused") + @Deprecated + @VisibleForTesting static class StateImpl extends BaseWriteChannel.BaseState { private static final long serialVersionUID = -6700378962714601115L; - private final ResultRetryAlgorithm algorithmForWrite; + private ResultRetryAlgorithm algorithmForWrite; - StateImpl(Builder builder) { + private StateImpl(Builder builder) { super(builder); - this.algorithmForWrite = builder.algorithmForWrite; - } - - @InternalApi - Builder toBuilder(StorageOptions options) { - Builder builder = new Builder(options, entity, uploadId); - builder.setPosition(position).setBuffer(buffer).setIsOpen(isOpen).setChunkSize(chunkSize); - return builder; - } - - static class Builder extends BaseWriteChannel.BaseState.Builder { - private ResultRetryAlgorithm algorithmForWrite; - - private Builder(StorageOptions options, BlobInfo blobInfo, String uploadId) { - super(options, blobInfo, uploadId); - } - - public Builder setResultRetryAlgorithm(ResultRetryAlgorithm algorithmForWrite) { - this.algorithmForWrite = algorithmForWrite; - return this; - } - - @Override - public RestorableState build() { - return new StateImpl(this); - } - } - - static Builder builder(StorageOptions options, BlobInfo blobInfo, String uploadId) { - return new Builder(options, blobInfo, uploadId); } @Override public WriteChannel restore() { try { - BlobWriteChannel channel = - BlobWriteChannel.newBuilder() - // should be okay, as the value always comes from an instance we gate to have - // HttpStorageOptions - .setStorageOptions((HttpStorageOptions) serviceOptions) - .setBlobInfo(entity) - .setUploadIdSupplier(() -> uploadId) - .setAlgorithmForWrite(algorithmForWrite) - .build(); - channel.restore(this); - return channel; + StorageObject encode = + entity != null ? Conversions.apiary().blobInfo().encode(entity) : null; + return new BlobWriteChannelV2.BlobWriteChannelV2State( + (HttpStorageOptions) serviceOptions, + JsonResumableWrite.of(encode, ImmutableMap.of(), uploadId), + position, + isOpen, + chunkSize, + buffer) + .restore(); } catch (Exception e) { throw StorageException.coalesce(e); } diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/BlobWriteChannelV2.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/BlobWriteChannelV2.java new file mode 100644 index 000000000..1bb074251 --- /dev/null +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/BlobWriteChannelV2.java @@ -0,0 +1,153 @@ +/* + * Copyright 2023 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. + */ + +package com.google.cloud.storage; + +import com.google.api.core.ApiFutures; +import com.google.api.services.storage.model.StorageObject; +import com.google.cloud.RestorableState; +import com.google.cloud.WriteChannel; +import com.google.cloud.storage.BlobReadChannelV2.BlobReadChannelContext; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Objects; + +final class BlobWriteChannelV2 extends BaseStorageWriteChannel { + + private final BlobReadChannelContext blobChannelContext; + private final JsonResumableWrite start; + + BlobWriteChannelV2(BlobReadChannelContext blobChannelContext, JsonResumableWrite start) { + super(Conversions.apiary().blobInfo()); + this.start = start; + this.blobChannelContext = blobChannelContext; + } + + @Override + public synchronized RestorableState capture() { + final byte[] bufferSnapshot; + BufferHandle handle = getBufferHandle(); + if (handle.position() > 0) { + ByteBuffer byteBuffer = handle.get(); + // duplicate so we don't actually modify the existing instance + ByteBuffer dup = byteBuffer.duplicate(); + dup.flip(); + int remaining = dup.remaining(); + bufferSnapshot = new byte[remaining]; + dup.get(bufferSnapshot); + } else { + bufferSnapshot = new byte[0]; + } + return new BlobWriteChannelV2State( + blobChannelContext.getStorageOptions(), + start, + getCommittedPosition(), + isOpen(), + getChunkSize(), + bufferSnapshot); + } + + @Override + protected LazyWriteChannel newLazyWriteChannel() { + return new LazyWriteChannel<>( + () -> + ResumableMedia.http() + .write() + .byteChannel(blobChannelContext.getHttpClientContext()) + .resumable() + .setCommittedBytesCallback(this::setCommittedPosition) + .withRetryConfig( + blobChannelContext.getStorageOptions().asRetryDependencies(), + blobChannelContext.getRetryAlgorithmManager().idempotent()) + .buffered(getBufferHandle()) + .setStartAsync(ApiFutures.immediateFuture(start)) + .build()); + } + + static final class BlobWriteChannelV2State + implements RestorableState, Serializable { + private static final long serialVersionUID = -1901664719924133474L; + + private final HttpStorageOptions options; + private final JsonResumableWrite resumableWrite; + + private final Long position; + private final Boolean isOpen; + private final Integer chunkSize; + private final byte[] bufferSnapshot; + + BlobWriteChannelV2State( + HttpStorageOptions options, + JsonResumableWrite resumableWrite, + Long position, + Boolean isOpen, + Integer chunkSize, + byte[] bufferSnapshot) { + this.options = options; + this.resumableWrite = resumableWrite; + this.position = position; + this.isOpen = isOpen; + this.chunkSize = chunkSize; + this.bufferSnapshot = bufferSnapshot; + } + + @Override + public WriteChannel restore() { + BlobWriteChannelV2 channel = + new BlobWriteChannelV2(BlobReadChannelContext.from(options), resumableWrite); + if (chunkSize != null) { + channel.setChunkSize(chunkSize); + } + if (bufferSnapshot != null && bufferSnapshot.length > 0) { + BufferHandle handle = channel.getBufferHandle(); + ByteBuffer byteBuffer = handle.get(); + byteBuffer.put(bufferSnapshot); + } + if (position != null) { + channel.setCommittedPosition(position); + } + if (isOpen != null) { + channel.setOpen(isOpen); + } + return channel; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof BlobWriteChannelV2State)) { + return false; + } + BlobWriteChannelV2State that = (BlobWriteChannelV2State) o; + return Objects.equals(options, that.options) + && Objects.equals(resumableWrite, that.resumableWrite) + && Objects.equals(position, that.position) + && Objects.equals(isOpen, that.isOpen) + && Objects.equals(chunkSize, that.chunkSize) + && Arrays.equals(bufferSnapshot, that.bufferSnapshot); + } + + @Override + public int hashCode() { + int result = Objects.hash(options, resumableWrite, position, isOpen, chunkSize); + result = 31 * result + Arrays.hashCode(bufferSnapshot); + return result; + } + } +} diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/Buffers.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/Buffers.java index 1e397608a..a828523ca 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/Buffers.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/Buffers.java @@ -54,6 +54,10 @@ static void position(Buffer b, int position) { b.position(position); } + static int position(Buffer b) { + return b.position(); + } + /** * attempt to drain all of {@code content} into {@code dsts} starting from {@code dsts[0]} through * {@code dsts[dsts.length - 1]} diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/Conversions.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/Conversions.java index 680dd1aec..3b81639a0 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/Conversions.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/Conversions.java @@ -46,6 +46,10 @@ default Decoder andThen(Decoder d) { static Decoder identity() { return (x) -> x; } + + default Decoder compose(Decoder before) { + return in -> this.decode(before.decode(in)); + } } interface Codec extends Encoder, Decoder { diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/GapicUnbufferedWritableByteChannel.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/GapicUnbufferedWritableByteChannel.java index 007e48021..951a52edb 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/GapicUnbufferedWritableByteChannel.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/GapicUnbufferedWritableByteChannel.java @@ -62,16 +62,6 @@ final class GapicUnbufferedWritableByteChannel< requestFactory.bucketName(), writeCtx.getConfirmedBytes()::set, resultFuture::set); } - @Override - public int write(ByteBuffer src) throws IOException { - return Math.toIntExact(write(new ByteBuffer[] {src})); - } - - @Override - public long write(ByteBuffer[] srcs) throws IOException { - return write(srcs, 0, srcs.length); - } - @Override public long write(ByteBuffer[] srcs, int srcsOffset, int srcLength) throws IOException { if (!open) { diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/GrpcBlobWriteChannel.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/GrpcBlobWriteChannel.java index 7eaae0339..f3520180b 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/GrpcBlobWriteChannel.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/GrpcBlobWriteChannel.java @@ -16,38 +16,23 @@ package com.google.cloud.storage; -import static com.google.cloud.storage.ByteSizeConstants._16MiB; -import static com.google.cloud.storage.ByteSizeConstants._256KiB; - import com.google.api.core.ApiFuture; import com.google.api.gax.retrying.ResultRetryAlgorithm; import com.google.api.gax.rpc.ClientStreamingCallable; import com.google.cloud.RestorableState; import com.google.cloud.WriteChannel; -import com.google.cloud.storage.BufferedWritableByteChannelSession.BufferedWritableByteChannel; import com.google.cloud.storage.Retrying.RetryingDependencies; -import com.google.common.base.Preconditions; -import com.google.common.base.Suppliers; import com.google.storage.v2.WriteObjectRequest; import com.google.storage.v2.WriteObjectResponse; -import java.io.IOException; -import java.nio.ByteBuffer; import java.util.function.Supplier; -final class GrpcBlobWriteChannel implements WriteChannel { - - private final LazyWriteChannel lazyWriteChannel; - - private int chunkSize = _16MiB; +final class GrpcBlobWriteChannel extends BaseStorageWriteChannel { - /** - * This is tracked for compatibility with BlobWriteChannel, such that simply creating a writer - * will create an object. - * - *

In the future we should move away from this behavior, and only create an object if write is - * called. - */ - private boolean writeCalledAtLeastOnce = false; + private final ClientStreamingCallable write; + private final RetryingDependencies deps; + private final ResultRetryAlgorithm alg; + private final Supplier> start; + private final Hasher hasher; GrpcBlobWriteChannel( ClientStreamingCallable write, @@ -55,27 +40,12 @@ final class GrpcBlobWriteChannel implements WriteChannel { ResultRetryAlgorithm alg, Supplier> start, Hasher hasher) { - lazyWriteChannel = - new LazyWriteChannel( - Suppliers.memoize( - () -> - ResumableMedia.gapic() - .write() - .byteChannel(write) - .setHasher(hasher) - .setByteStringStrategy(ByteStringStrategy.copy()) - .resumable() - .withRetryConfig(deps, alg) - .buffered(BufferHandle.allocate(Buffers.alignSize(chunkSize, _256KiB))) - .setStartAsync(start.get()) - .build())); - } - - @Override - public void setChunkSize(int chunkSize) { - Preconditions.checkState( - !lazyWriteChannel.isOpened(), "Unable to change chunkSize after write"); - this.chunkSize = chunkSize; + super(Conversions.grpc().blobInfo().compose(WriteObjectResponse::getResource)); + this.write = write; + this.deps = deps; + this.alg = alg; + this.start = start; + this.hasher = hasher; } @Override @@ -84,57 +54,18 @@ public RestorableState capture() { } @Override - public int write(ByteBuffer src) throws IOException { - writeCalledAtLeastOnce = true; - return lazyWriteChannel.getChannel().write(src); - } - - @Override - public boolean isOpen() { - if (!writeCalledAtLeastOnce) { - return true; - } else { - return lazyWriteChannel.isOpened() && lazyWriteChannel.getChannel().isOpen(); - } - } - - @Override - public void close() throws IOException { - if (!writeCalledAtLeastOnce) { - lazyWriteChannel.getChannel().write(ByteBuffer.allocate(0)); - } - if (isOpen()) { - lazyWriteChannel.getChannel().close(); - } - } - - ApiFuture getResults() { - return lazyWriteChannel.session.get().getResult(); - } - - private static final class LazyWriteChannel { - private final Supplier> session; - private final Supplier channel; - - private boolean opened = false; - - public LazyWriteChannel( - Supplier> session) { - this.session = session; - this.channel = - Suppliers.memoize( - () -> { - opened = true; - return session.get().open(); - }); - } - - public BufferedWritableByteChannel getChannel() { - return channel.get(); - } - - public boolean isOpened() { - return opened; - } + protected LazyWriteChannel newLazyWriteChannel() { + return new LazyWriteChannel<>( + () -> + ResumableMedia.gapic() + .write() + .byteChannel(write) + .setHasher(hasher) + .setByteStringStrategy(ByteStringStrategy.copy()) + .resumable() + .withRetryConfig(deps, alg) + .buffered(getBufferHandle()) + .setStartAsync(start.get()) + .build()); } } diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpContentRange.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpContentRange.java index 8961f5239..49fc5a194 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpContentRange.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpContentRange.java @@ -32,6 +32,8 @@ private HttpContentRange(boolean finalizing) { public abstract String getHeaderValue(); + public abstract boolean endOffsetEquals(long e); + public boolean isFinalizing() { return finalizing; } @@ -92,6 +94,11 @@ public String getHeaderValue() { return String.format("bytes %d-%d/*", spec.beginOffset(), spec.endOffsetInclusive()); } + @Override + public boolean endOffsetEquals(long e) { + return e == spec.endOffset(); + } + @Override public ByteRangeSpec range() { return spec; @@ -141,6 +148,11 @@ public String getHeaderValue() { return String.format("bytes %d-%d/%d", spec.beginOffset(), spec.endOffsetInclusive(), size); } + @Override + public boolean endOffsetEquals(long e) { + return e == spec.endOffset(); + } + @Override public long getSize() { return size; @@ -193,6 +205,11 @@ public String getHeaderValue() { return String.format("bytes */%d", size); } + @Override + public boolean endOffsetEquals(long e) { + return false; + } + @Override public long getSize() { return size; @@ -229,6 +246,11 @@ private Query() { super(false); } + @Override + public boolean endOffsetEquals(long e) { + return false; + } + @Override public String getHeaderValue() { return "bytes */*"; diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpDownloadSessionBuilder.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpDownloadSessionBuilder.java index 370c9d6ce..2cf21d767 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpDownloadSessionBuilder.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpDownloadSessionBuilder.java @@ -38,7 +38,7 @@ final class HttpDownloadSessionBuilder { private HttpDownloadSessionBuilder() {} - public static HttpDownloadSessionBuilder create() { + static HttpDownloadSessionBuilder create() { return INSTANCE; } diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpUploadSessionBuilder.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpUploadSessionBuilder.java new file mode 100644 index 000000000..a673c1f9b --- /dev/null +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpUploadSessionBuilder.java @@ -0,0 +1,34 @@ +/* + * Copyright 2023 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. + */ + +package com.google.cloud.storage; + +import org.checkerframework.checker.nullness.qual.NonNull; + +final class HttpUploadSessionBuilder { + private static final HttpUploadSessionBuilder INSTANCE = new HttpUploadSessionBuilder(); + + private HttpUploadSessionBuilder() {} + + static HttpUploadSessionBuilder create() { + return INSTANCE; + } + + @NonNull + HttpWritableByteChannelSessionBuilder byteChannel(@NonNull HttpClientContext httpClientContext) { + return new HttpWritableByteChannelSessionBuilder(httpClientContext); + } +} diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpWritableByteChannelSessionBuilder.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpWritableByteChannelSessionBuilder.java new file mode 100644 index 000000000..19abf0928 --- /dev/null +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/HttpWritableByteChannelSessionBuilder.java @@ -0,0 +1,190 @@ +/* + * Copyright 2023 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. + */ + +package com.google.cloud.storage; + +import static java.util.Objects.requireNonNull; + +import com.google.api.core.ApiFuture; +import com.google.api.core.SettableApiFuture; +import com.google.api.gax.retrying.ResultRetryAlgorithm; +import com.google.api.services.storage.model.StorageObject; +import com.google.cloud.storage.ChannelSession.BufferedWriteSession; +import com.google.cloud.storage.ChannelSession.UnbufferedWriteSession; +import com.google.cloud.storage.Retrying.RetryingDependencies; +import com.google.cloud.storage.UnbufferedWritableByteChannelSession.UnbufferedWritableByteChannel; +import java.nio.ByteBuffer; +import java.util.function.BiFunction; +import java.util.function.LongConsumer; +import org.checkerframework.checker.nullness.qual.NonNull; + +final class HttpWritableByteChannelSessionBuilder { + + private static final int DEFAULT_BUFFER_CAPACITY = ByteSizeConstants._16MiB; + @NonNull private final HttpClientContext httpClientContext; + + HttpWritableByteChannelSessionBuilder(@NonNull HttpClientContext httpClientContext) { + this.httpClientContext = + requireNonNull(httpClientContext, "httpClientContext must be non null"); + } + + /** + * The build {@link WritableByteChannelSession} will perform a "Resumable" upload. + * + *

A "Resumable" upload will sync the transmitted data with GCS upon each individual flush and + * when the channel is closed. + * + *

If an error is returned the individual flush can be transparently retried. + */ + ResumableUploadBuilder resumable() { + return new ResumableUploadBuilder(httpClientContext); + } + + static final class ResumableUploadBuilder { + + @NonNull private final HttpClientContext httpClientContext; + private RetryingDependencies deps; + private ResultRetryAlgorithm alg; + private LongConsumer committedBytesCallback; + + ResumableUploadBuilder(@NonNull HttpClientContext httpClientContext) { + this.httpClientContext = httpClientContext; + this.deps = RetryingDependencies.attemptOnce(); + this.alg = Retrying.neverRetry(); + this.committedBytesCallback = l -> {}; + } + + ResumableUploadBuilder setCommittedBytesCallback(@NonNull LongConsumer committedBytesCallback) { + this.committedBytesCallback = + requireNonNull(committedBytesCallback, "committedBytesCallback must be non null"); + return this; + } + + ResumableUploadBuilder withRetryConfig( + @NonNull RetryingDependencies deps, @NonNull ResultRetryAlgorithm alg) { + this.deps = requireNonNull(deps, "deps must be non null"); + this.alg = requireNonNull(alg, "alg must be non null"); + return this; + } + + /** + * Do not apply any intermediate buffering. Any call to {@link + * java.nio.channels.WritableByteChannel#write(ByteBuffer)} will be segmented as is and sent to + * GCS. + * + *

Note: this is considered an advanced API, and should not be used in circumstances in which + * control of {@link ByteBuffer}s sent to {@code write} is not self-contained. + */ + UnbufferedResumableUploadBuilder unbuffered() { + return new UnbufferedResumableUploadBuilder(); + } + + /** Buffer up to {@link #DEFAULT_BUFFER_CAPACITY} worth of bytes before attempting to flush */ + BufferedResumableUploadBuilder buffered() { + return buffered(BufferHandle.allocate(DEFAULT_BUFFER_CAPACITY)); + } + + /** + * Buffer using {@code byteBuffer} worth of space before attempting to flush. + * + *

The provided {@link ByteBuffer} should be aligned with GCSs block size of 256 + * KiB. + */ + BufferedResumableUploadBuilder buffered(ByteBuffer byteBuffer) { + return buffered(BufferHandle.handleOf(byteBuffer)); + } + + BufferedResumableUploadBuilder buffered(BufferHandle bufferHandle) { + return new BufferedResumableUploadBuilder(bufferHandle); + } + + /** + * When constructing any of our channel sessions, there is always a {@link + * GapicUnbufferedWritableByteChannel} at the bottom of it. This method creates a BiFunction + * which will instantiate the {@link GapicUnbufferedWritableByteChannel} when provided with a + * {@code StartT} value and a {@code SettableApiFuture}. + * + *

As part of providing the function, the provided parameters {@code FlusherFactory} and + * {@code f} are "bound" into the returned function. In conjunction with the configured fields + * of this class a new instance of {@link GapicUnbufferedWritableByteChannel} can be + * constructed. + */ + private BiFunction< + JsonResumableWrite, SettableApiFuture, UnbufferedWritableByteChannel> + bindFunction() { + // it is theoretically possible that the setter methods for the following variables could + // be called again between when this method is invoked and the resulting function is invoked. + // To ensure we are using the specified values at the point in time they are bound to the + // function read them into local variables which will be closed over rather than the class + // fields. + RetryingDependencies boundDeps = deps; + ResultRetryAlgorithm boundAlg = alg; + return (start, resultFuture) -> + new ApiaryUnbufferedWritableByteChannel( + httpClientContext, boundDeps, boundAlg, start, resultFuture, committedBytesCallback); + } + + final class UnbufferedResumableUploadBuilder { + + private ApiFuture start; + + /** + * Set the Future which will contain the ResumableWrite information necessary to open the + * Write stream. + */ + UnbufferedResumableUploadBuilder setStartAsync(ApiFuture start) { + this.start = requireNonNull(start, "start must be non null"); + return this; + } + + UnbufferedWritableByteChannelSession build() { + return new UnbufferedWriteSession<>( + requireNonNull(start, "start must be non null"), + bindFunction().andThen(StorageByteChannels.writable()::createSynchronized)); + } + } + + final class BufferedResumableUploadBuilder { + + private final BufferHandle bufferHandle; + + private ApiFuture start; + + BufferedResumableUploadBuilder(BufferHandle bufferHandle) { + this.bufferHandle = bufferHandle; + } + + /** + * Set the Future which will contain the ResumableWrite information necessary to open the + * Write stream. + */ + BufferedResumableUploadBuilder setStartAsync(ApiFuture start) { + this.start = requireNonNull(start, "start must be non null"); + return this; + } + + BufferedWritableByteChannelSession build() { + return new BufferedWriteSession<>( + requireNonNull(start, "start must be non null"), + bindFunction() + .andThen(c -> new DefaultBufferedWritableByteChannel(bufferHandle, c)) + .andThen(StorageByteChannels.writable()::createSynchronized)); + } + } + } +} diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/JsonResumableSession.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/JsonResumableSession.java index f59355502..debd611b0 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/JsonResumableSession.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/JsonResumableSession.java @@ -20,6 +20,7 @@ import com.google.api.services.storage.model.StorageObject; import com.google.cloud.storage.Conversions.Decoder; import com.google.cloud.storage.Retrying.RetryingDependencies; +import com.google.cloud.storage.spi.v1.HttpRpcContext; import com.google.cloud.storage.spi.v1.HttpStorageRpc; import io.opencensus.trace.EndSpanOptions; import java.util.concurrent.atomic.AtomicBoolean; @@ -63,6 +64,8 @@ final class JsonResumableSession extends ResumableSession { JsonResumableSessionPutTask task = new JsonResumableSessionPutTask( context, resumableWrite.getUploadId(), content, contentRange); + HttpRpcContext httpRpcContext = HttpRpcContext.getInstance(); + httpRpcContext.newInvocationId(); AtomicBoolean dirty = new AtomicBoolean(false); return Retrying.run( deps, @@ -70,10 +73,11 @@ final class JsonResumableSession extends ResumableSession { () -> { if (dirty.getAndSet(true)) { ResumableOperationResult<@Nullable StorageObject> query = query(); - if (query.getObject() != null) { + long persistedSize = query.getPersistedSize(); + if (contentRange.endOffsetEquals(persistedSize) || query.getObject() != null) { return query; } else { - task.rewindTo(query.getPersistedSize()); + task.rewindTo(persistedSize); } } return task.call(); diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/JsonResumableSessionPutTask.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/JsonResumableSessionPutTask.java index 73b7d14a4..de905fb48 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/JsonResumableSessionPutTask.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/JsonResumableSessionPutTask.java @@ -21,8 +21,10 @@ import com.google.api.client.http.HttpResponse; import com.google.api.client.http.HttpResponseException; import com.google.api.services.storage.model.StorageObject; +import com.google.cloud.storage.HttpContentRange.HasRange; import com.google.cloud.storage.StorageException.IOExceptionCallable; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import io.opencensus.common.Scope; import io.opencensus.trace.Span; import io.opencensus.trace.Status; @@ -56,7 +58,22 @@ final class JsonResumableSessionPutTask } public void rewindTo(long offset) { - content.rewindTo(offset); + if (originalContentRange instanceof HasRange) { + HasRange hasRange = (HasRange) originalContentRange; + ByteRangeSpec range = hasRange.range(); + long originalBegin = range.beginOffset(); + long contentOffset = offset - originalBegin; + Preconditions.checkArgument( + 0 <= contentOffset && contentOffset < content.getLength(), + "Rewind offset is out of bounds. (%s <= %s < %s)", + range.beginOffset(), + offset, + range.endOffset()); + content.rewindTo(contentOffset); + } else { + content.rewindTo(0); + } + if (contentRange instanceof HttpContentRange.HasRange) { HttpContentRange.HasRange range = (HttpContentRange.HasRange) contentRange; contentRange = range.map(s -> s.withNewBeginOffset(offset)); @@ -195,7 +212,11 @@ public void rewindTo(long offset) { span.setStatus(Status.UNKNOWN.withDescription(se.getMessage())); throw se; } - } catch (StorageException e) { + } catch (StorageException | IllegalArgumentException e) { + // IllegalArgumentException can happen if there is no json in the body and we try to parse it + // Our retry algorithms have special case for this, so in an effort to keep compatibility + // with those existing behaviors, explicitly rethrow an IllegalArgumentException that may have + // happened span.setStatus(Status.UNKNOWN.withDescription(e.getMessage())); throw e; } catch (Exception e) { diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/JsonResumableWrite.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/JsonResumableWrite.java index c5e17aad5..4ec1fc891 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/JsonResumableWrite.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/JsonResumableWrite.java @@ -19,19 +19,31 @@ import com.google.api.services.storage.model.StorageObject; import com.google.cloud.storage.spi.v1.StorageRpc; import com.google.common.base.MoreObjects; +import com.google.gson.Gson; +import com.google.gson.stream.JsonReader; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.io.StringReader; import java.util.Map; import java.util.Objects; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.NonNull; -final class JsonResumableWrite { - @MonotonicNonNull private final StorageObject object; +final class JsonResumableWrite implements Serializable { + private static final long serialVersionUID = 7934407897802252292L; + private static final Gson gson = new Gson(); + + @MonotonicNonNull private transient StorageObject object; @MonotonicNonNull private final Map options; @MonotonicNonNull private final String signedUrl; @NonNull private final String uploadId; + private volatile String objectJson; + private JsonResumableWrite( StorageObject object, Map options, @@ -77,6 +89,28 @@ public String toString() { .toString(); } + private String getObjectJson() { + if (objectJson == null) { + synchronized (this) { + if (objectJson == null) { + objectJson = gson.toJson(object); + } + } + } + return objectJson; + } + + private void writeObject(ObjectOutputStream out) throws IOException { + String ignore = getObjectJson(); + out.defaultWriteObject(); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + JsonReader jsonReader = gson.newJsonReader(new StringReader(this.objectJson)); + this.object = gson.fromJson(jsonReader, StorageObject.class); + } + static JsonResumableWrite of( StorageObject req, Map options, String uploadId) { return new JsonResumableWrite(req, options, null, uploadId); diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/LazyWriteChannel.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/LazyWriteChannel.java new file mode 100644 index 000000000..1c14eda85 --- /dev/null +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/LazyWriteChannel.java @@ -0,0 +1,69 @@ +/* + * Copyright 2022 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. + */ + +package com.google.cloud.storage; + +import com.google.cloud.storage.BufferedWritableByteChannelSession.BufferedWritableByteChannel; +import java.util.function.Supplier; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.NonNull; + +final class LazyWriteChannel { + + private final Supplier> sessionSupplier; + + @MonotonicNonNull private volatile BufferedWritableByteChannelSession session; + @MonotonicNonNull private volatile BufferedWritableByteChannel channel; + + private boolean open = false; + + LazyWriteChannel(Supplier> sessionSupplier) { + this.sessionSupplier = sessionSupplier; + } + + @NonNull + BufferedWritableByteChannel getChannel() { + if (channel != null) { + return channel; + } else { + synchronized (this) { + if (channel == null) { + open = true; + channel = getSession().open(); + } + return channel; + } + } + } + + @NonNull + BufferedWritableByteChannelSession getSession() { + if (session != null) { + return session; + } else { + synchronized (this) { + if (session == null) { + session = sessionSupplier.get(); + } + return session; + } + } + } + + boolean isOpen() { + return open && getChannel().isOpen(); + } +} diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/ResumableMedia.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/ResumableMedia.java index 85c96bd8d..2d3fbf939 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/ResumableMedia.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/ResumableMedia.java @@ -103,6 +103,10 @@ static final class HttpMediaSession { private HttpMediaSession() {} + HttpUploadSessionBuilder write() { + return HttpUploadSessionBuilder.create(); + } + HttpDownloadSessionBuilder read() { return HttpDownloadSessionBuilder.create(); } diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/Retrying.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/Retrying.java index f625b58b9..0b5f66ce3 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/Retrying.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/Retrying.java @@ -103,11 +103,16 @@ static U run( } catch (StorageException se) { // we hope for this case throw se; + } catch (IllegalArgumentException iae) { + // IllegalArgumentException can happen if there is no json in the body and we try + // to parse it Our retry algorithms have special case for this, so in an effort to + // keep compatibility with those existing behaviors, explicitly rethrow an + // IllegalArgumentException that may have happened + throw iae; } catch (Exception e) { - // but wire in this fall through just in case. + // Wire in this fall through just in case. // all of our retry algorithms are centered around StorageException so this helps - // those - // be more effective + // those be more effective throw StorageException.coalesce(e); } }, diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/RewindableHttpContent.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/RewindableHttpContent.java index fd171e054..68bf4e22f 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/RewindableHttpContent.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/RewindableHttpContent.java @@ -22,11 +22,15 @@ import com.google.common.io.ByteStreams; import java.io.IOException; import java.io.OutputStream; +import java.nio.Buffer; +import java.nio.ByteBuffer; import java.nio.channels.Channels; import java.nio.channels.SeekableByteChannel; +import java.nio.channels.WritableByteChannel; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; +import java.util.Arrays; abstract class RewindableHttpContent extends AbstractHttpContent { @@ -48,6 +52,10 @@ static RewindableHttpContent empty() { return EmptyRewindableContent.INSTANCE; } + static RewindableHttpContent of(ByteBuffer... buffers) { + return new ByteBufferHttpContent(buffers); + } + static RewindableHttpContent of(Path path) throws IOException { return new PathRewindableHttpContent(path); } @@ -88,7 +96,7 @@ public long getLength() { } @Override - protected void rewindTo(long offset) { + void rewindTo(long offset) { Preconditions.checkArgument( offset < size, "provided offset must be less than size (%d < %d)", offset, size); this.readOffset = offset; @@ -103,4 +111,78 @@ public void writeTo(OutputStream out) throws IOException { } } } + + private static final class ByteBufferHttpContent extends RewindableHttpContent { + + private final ByteBuffer[] buffers; + // keep an array of the positions in case we need to rewind them for retries + // doing this is simpler than duplicating the buffers and using marks, as we don't need to + // advance the position of the original buffers upon success. + // We generally expect success, and in this case are planning in case of failure. + private final int[] positions; + private final long totalLength; + // track whether we have changed any state + private boolean dirty; + + private long offset; + + private ByteBufferHttpContent(ByteBuffer[] buffers) { + this.buffers = buffers; + this.positions = Arrays.stream(buffers).mapToInt(Buffers::position).toArray(); + this.totalLength = Arrays.stream(buffers).mapToLong(Buffer::remaining).sum(); + this.dirty = false; + } + + @Override + public long getLength() { + return totalLength - offset; + } + + @Override + public void writeTo(OutputStream out) throws IOException { + dirty = true; + WritableByteChannel c = Channels.newChannel(out); + for (ByteBuffer buffer : buffers) { + c.write(buffer); + } + out.flush(); + } + + @Override + void rewindTo(long offset) { + Preconditions.checkArgument( + offset < totalLength, + "provided offset must be less than totalLength (%s < %s)", + offset, + totalLength); + if (dirty || offset != this.offset) { + // starting from the end of our data, walk back the buffers updating their position + // to coincide with the rewind of the overall content + int idx = buffers.length - 1; + for (long currentOffset = totalLength; currentOffset > 0; ) { + int position = positions[idx]; + ByteBuffer buf = buffers[idx]; + + int origRemaining = buf.limit() - position; + + long begin = currentOffset - origRemaining; + + if (begin <= offset && offset < currentOffset) { + long diff = offset - begin; + Buffers.position(buf, position + Math.toIntExact(diff)); + } else if (offset >= currentOffset) { + // the desired offset is after this buf + // ensure it does not have any available + Buffers.position(buf, buf.limit()); + } else { + Buffers.position(buf, position); + } + + currentOffset = begin; + idx -= 1; + } + } + this.offset = offset; + } + } } diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/StorageImpl.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/StorageImpl.java index a97a325c5..fc8601cb6 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/StorageImpl.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/StorageImpl.java @@ -23,6 +23,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.concurrent.Executors.callable; +import com.google.api.core.ApiFuture; import com.google.api.gax.paging.Page; import com.google.api.gax.retrying.ResultRetryAlgorithm; import com.google.api.services.storage.model.BucketAccessControl; @@ -84,7 +85,9 @@ import java.util.Set; import java.util.TimeZone; import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.function.Function; import java.util.function.Supplier; import org.checkerframework.checker.nullness.qual.Nullable; @@ -274,14 +277,20 @@ public Blob createFrom( BlobInfo blobInfo, InputStream content, int bufferSize, BlobWriteOption... options) throws IOException { - BlobWriteChannel blobWriteChannel; - try (WriteChannel writer = writer(blobInfo, options)) { - blobWriteChannel = (BlobWriteChannel) writer; + ApiFuture objectFuture; + try (StorageWriteChannel writer = writer(blobInfo, options)) { + objectFuture = writer.getObject(); uploadHelper(Channels.newChannel(content), writer, bufferSize); } - StorageObject objectProto = blobWriteChannel.getStorageObject(); - BlobInfo info = Conversions.apiary().blobInfo().decode(objectProto); - return info.asBlob(this); + // keep these two try blocks separate for the time being + // leaving the above will cause the writer to close writing and finalizing the session and + // (hopefully, on successful finalization) resolve our future + try { + BlobInfo info = objectFuture.get(10, TimeUnit.SECONDS); + return info.asBlob(this); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + throw StorageException.coalesce(e); + } } /* @@ -646,38 +655,41 @@ public void downloadTo(BlobId blob, OutputStream outputStream, BlobSourceOption. } @Override - public BlobWriteChannel writer(BlobInfo blobInfo, BlobWriteOption... options) { + public StorageWriteChannel writer(BlobInfo blobInfo, BlobWriteOption... options) { Opts opts = Opts.unwrap(options).resolveFrom(blobInfo); final Map optionsMap = opts.getRpcOptions(); BlobInfo.Builder builder = blobInfo.toBuilder().setMd5(null).setCrc32c(null); BlobInfo updated = opts.blobInfoMapper().apply(builder).build(); - return BlobWriteChannel.newBuilder() - .setStorageOptions(getOptions()) - .setUploadIdSupplier( - ResumableMedia.startUploadForBlobInfo( - getOptions(), - updated, - optionsMap, - retryAlgorithmManager.getForResumableUploadSessionCreate(optionsMap))) - .setAlgorithmForWrite(retryAlgorithmManager.getForResumableUploadSessionWrite(optionsMap)) - .build(); + + StorageObject encode = codecs.blobInfo().encode(updated); + // open the resumable session outside the write channel + // the exception behavior of open is different from #write(ByteBuffer) + Supplier uploadIdSupplier = + ResumableMedia.startUploadForBlobInfo( + getOptions(), + updated, + optionsMap, + retryAlgorithmManager.getForResumableUploadSessionCreate(optionsMap)); + JsonResumableWrite jsonResumableWrite = + JsonResumableWrite.of(encode, optionsMap, uploadIdSupplier.get()); + return new BlobWriteChannelV2(BlobReadChannelContext.from(getOptions()), jsonResumableWrite); } @Override - public BlobWriteChannel writer(URL signedURL) { + public StorageWriteChannel writer(URL signedURL) { + // TODO: is it possible to know if a signed url is configured to have a constraint which makes + // it idempotent? ResultRetryAlgorithm forResumableUploadSessionCreate = - retryAlgorithmManager.getForResumableUploadSessionCreate( - Collections - .emptyMap()); // TODO: is it possible to know if a signed url is configured to have - // a constraint which makes it idempotent? - return BlobWriteChannel.newBuilder() - .setStorageOptions(getOptions()) - .setUploadIdSupplier( - ResumableMedia.startUploadForSignedUrl( - getOptions(), signedURL, forResumableUploadSessionCreate)) - .setAlgorithmForWrite( - retryAlgorithmManager.getForResumableUploadSessionWrite(Collections.emptyMap())) - .build(); + retryAlgorithmManager.getForResumableUploadSessionCreate(Collections.emptyMap()); + // open the resumable session outside the write channel + // the exception behavior of open is different from #write(ByteBuffer) + String signedUrlString = signedURL.toString(); + Supplier uploadIdSupplier = + ResumableMedia.startUploadForSignedUrl( + getOptions(), signedURL, forResumableUploadSessionCreate); + JsonResumableWrite jsonResumableWrite = + JsonResumableWrite.of(signedUrlString, uploadIdSupplier.get()); + return new BlobWriteChannelV2(BlobReadChannelContext.from(getOptions()), jsonResumableWrite); } @Override diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/StorageWriteChannel.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/StorageWriteChannel.java new file mode 100644 index 000000000..d1badc0b1 --- /dev/null +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/StorageWriteChannel.java @@ -0,0 +1,24 @@ +/* + * Copyright 2023 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. + */ + +package com.google.cloud.storage; + +import com.google.api.core.ApiFuture; +import com.google.cloud.WriteChannel; + +interface StorageWriteChannel extends WriteChannel { + ApiFuture getObject(); +} diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/UnbufferedWritableByteChannelSession.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/UnbufferedWritableByteChannelSession.java index 1df74aba7..8affde6b5 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/UnbufferedWritableByteChannelSession.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/UnbufferedWritableByteChannelSession.java @@ -17,11 +17,23 @@ package com.google.cloud.storage; import com.google.cloud.storage.UnbufferedWritableByteChannelSession.UnbufferedWritableByteChannel; +import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.channels.GatheringByteChannel; import java.nio.channels.WritableByteChannel; interface UnbufferedWritableByteChannelSession extends WritableByteChannelSession { - interface UnbufferedWritableByteChannel extends WritableByteChannel, GatheringByteChannel {} + interface UnbufferedWritableByteChannel extends WritableByteChannel, GatheringByteChannel { + @Override + default int write(ByteBuffer src) throws IOException { + return Math.toIntExact(write(new ByteBuffer[] {src})); + } + + @Override + default long write(ByteBuffer[] srcs) throws IOException { + return write(srcs, 0, srcs.length); + } + } } diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/Utils.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/Utils.java index 67bafab86..c24a68d4d 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/Utils.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/Utils.java @@ -37,6 +37,7 @@ import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; import java.time.temporal.ChronoUnit; +import java.util.Arrays; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -283,6 +284,14 @@ static void diffMaps( keys.map(NamedField::literal).map(k -> NamedField.nested(parent, k)).forEach(sink); } + static T[] subArray(T[] ts, int offset, int length) { + if (offset == 0 && length == ts.length) { + return ts; + } else { + return Arrays.copyOfRange(ts, offset, length); + } + } + private static int crc32cDecode(String from) { byte[] decodeCrc32c = BaseEncoding.base64().decode(from); return Ints.fromByteArray(decodeCrc32c); diff --git a/google-cloud-storage/src/main/java/com/google/cloud/storage/spi/v1/HttpStorageRpc.java b/google-cloud-storage/src/main/java/com/google/cloud/storage/spi/v1/HttpStorageRpc.java index 100f5b6ce..98d9476f8 100644 --- a/google-cloud-storage/src/main/java/com/google/cloud/storage/spi/v1/HttpStorageRpc.java +++ b/google-cloud-storage/src/main/java/com/google/cloud/storage/spi/v1/HttpStorageRpc.java @@ -102,11 +102,11 @@ public class HttpStorageRpc implements StorageRpc { // declare this HttpStatus code here as it's not included in java.net.HttpURLConnection private static final int SC_REQUESTED_RANGE_NOT_SATISFIABLE = 416; + private static final boolean IS_RECORD_EVENTS = true; private final StorageOptions options; private final Storage storage; private final Tracer tracer = Tracing.getTracer(); - private final CensusHttpModule censusHttpModule; private final HttpRequestInitializer batchRequestInitializer; private static final long MEGABYTE = 1024L * 1024L; @@ -123,7 +123,7 @@ public HttpStorageRpc(StorageOptions options, JsonFactory jsonFactory) { this.options = options; // Open Census initialization - censusHttpModule = new CensusHttpModule(tracer, true); + CensusHttpModule censusHttpModule = new CensusHttpModule(tracer, IS_RECORD_EVENTS); initializer = censusHttpModule.getHttpRequestInitializer(initializer); initializer = new InvocationIdInitializer(initializer); batchRequestInitializer = censusHttpModule.getHttpRequestInitializer(null); @@ -318,10 +318,7 @@ private static void setEncryptionHeaders( /** Helper method to start a span. */ private Span startSpan(String spanName) { - return tracer - .spanBuilder(spanName) - .setRecordEvents(censusHttpModule.isRecordEvents()) - .startSpan(); + return tracer.spanBuilder(spanName).setRecordEvents(IS_RECORD_EVENTS).startSpan(); } @Override diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/BlobTest.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/BlobTest.java index 68ac4683b..4cb1ee86a 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/BlobTest.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/BlobTest.java @@ -40,7 +40,6 @@ import com.google.cloud.storage.Acl.User; import com.google.cloud.storage.Blob.BlobSourceOption; import com.google.cloud.storage.BlobInfo.BuilderImpl; -import com.google.cloud.storage.Storage.BlobWriteOption; import com.google.cloud.storage.Storage.CopyRequest; import com.google.cloud.storage.spi.v1.HttpStorageRpc; import com.google.cloud.storage.spi.v1.StorageRpc; @@ -387,43 +386,6 @@ public void testReaderWithDecryptionKey() throws Exception { assertSame(channel, blob.reader(BlobSourceOption.decryptionKey(KEY))); } - @Test - public void testWriter() throws Exception { - initializeExpectedBlob(); - BlobWriteChannel channel = createMock(BlobWriteChannel.class); - expect(storage.getOptions()).andReturn(mockOptions).anyTimes(); - expect(storage.writer(eq(expectedBlob))).andReturn(channel); - replay(storage); - initializeBlob(); - assertSame(channel, blob.writer()); - } - - @Test - public void testWriterWithEncryptionKey() throws Exception { - initializeExpectedBlob(); - BlobWriteChannel channel = createMock(BlobWriteChannel.class); - expect(storage.getOptions()).andReturn(mockOptions).anyTimes(); - expect(storage.writer(eq(expectedBlob), eq(BlobWriteOption.encryptionKey(BASE64_KEY)))) - .andReturn(channel) - .times(2); - replay(storage); - initializeBlob(); - assertSame(channel, blob.writer(BlobWriteOption.encryptionKey(BASE64_KEY))); - assertSame(channel, blob.writer(BlobWriteOption.encryptionKey(KEY))); - } - - @Test - public void testWriterWithKmsKeyName() throws Exception { - initializeExpectedBlob(); - BlobWriteChannel channel = createMock(BlobWriteChannel.class); - expect(storage.getOptions()).andReturn(mockOptions).anyTimes(); - expect(storage.writer(eq(expectedBlob), eq(BlobWriteOption.kmsKeyName(KMS_KEY_NAME)))) - .andReturn(channel); - replay(storage); - initializeBlob(); - assertSame(channel, blob.writer(BlobWriteOption.kmsKeyName(KMS_KEY_NAME))); - } - @Test public void testSignUrl() throws Exception { initializeExpectedBlob(); diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/BlobWriteChannelTest.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/BlobWriteChannelTest.java deleted file mode 100644 index 2898064aa..000000000 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/BlobWriteChannelTest.java +++ /dev/null @@ -1,961 +0,0 @@ -/* - * Copyright 2015 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. - */ - -package com.google.cloud.storage; - -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.capture; -import static org.easymock.EasyMock.captureLong; -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.eq; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.expectLastCall; -import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.verify; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import com.google.api.gax.retrying.ResultRetryAlgorithm; -import com.google.api.services.storage.model.StorageObject; -import com.google.cloud.NoCredentials; -import com.google.cloud.RestorableState; -import com.google.cloud.WriteChannel; -import com.google.cloud.storage.spi.StorageRpcFactory; -import com.google.cloud.storage.spi.v1.StorageRpc; -import com.google.common.collect.ImmutableMap; -import java.io.IOException; -import java.math.BigInteger; -import java.net.MalformedURLException; -import java.net.SocketException; -import java.net.URL; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Collections; -import java.util.Map; -import java.util.Random; -import org.easymock.Capture; -import org.easymock.CaptureType; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public class BlobWriteChannelTest { - - private static final String BUCKET_NAME = "b"; - private static final String BLOB_NAME = "n"; - private static final String UPLOAD_ID = "uploadid"; - private static final BlobInfo BLOB_INFO = BlobInfo.newBuilder(BUCKET_NAME, BLOB_NAME).build(); - private static final BlobInfo BLOB_INFO_WITH_GENERATION = - BlobInfo.newBuilder(BUCKET_NAME, BLOB_NAME, 1L).build(); - private static final StorageObject UPDATED_BLOB = new StorageObject(); - private static final Map EMPTY_RPC_OPTIONS = ImmutableMap.of(); - private static final Map RPC_OPTIONS_GENERATION = - ImmutableMap.of(StorageRpc.Option.IF_GENERATION_MATCH, 1L); - private static final int MIN_CHUNK_SIZE = 256 * 1024; - private static final int DEFAULT_CHUNK_SIZE = 60 * MIN_CHUNK_SIZE; // 15MiB - private static final int CUSTOM_CHUNK_SIZE = 4 * MIN_CHUNK_SIZE; - private static final Random RANDOM = new Random(); - private static final String SIGNED_URL = - "http://www.test.com/test-bucket/test1.txt?GoogleAccessId=testClient-test@test.com&Expires=1553839761&Signature=MJUBXAZ7"; - private static final StorageException socketClosedException = - new StorageException(new SocketException("Socket closed")); - private HttpStorageOptions options; - private StorageRpcFactory rpcFactoryMock; - private StorageRpc storageRpcMock; - private BlobWriteChannel writer; - private HttpRetryAlgorithmManager retryAlgorithmManager; - - @Before - public void setUp() { - rpcFactoryMock = createMock(StorageRpcFactory.class); - storageRpcMock = createMock(StorageRpc.class); - expect(rpcFactoryMock.create(anyObject(StorageOptions.class))).andReturn(storageRpcMock); - replay(rpcFactoryMock); - options = - HttpStorageOptions.newBuilder() - .setProjectId("projectid") - .setServiceRpcFactory(rpcFactoryMock) - .setCredentials(NoCredentials.getInstance()) - .build(); - retryAlgorithmManager = options.getRetryAlgorithmManager(); - } - - @After - public void tearDown() throws Exception { - verify(rpcFactoryMock, storageRpcMock); - } - - @Test - public void testCreate() { - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO), EMPTY_RPC_OPTIONS)) - .andReturn(UPLOAD_ID); - replay(storageRpcMock); - writer = newWriter(); - assertTrue(writer.isOpen()); - assertNull(writer.getStorageObject()); - } - - @Test - public void testCreateRetryableError() { - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO_WITH_GENERATION), - RPC_OPTIONS_GENERATION)) - .andThrow(socketClosedException); - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO_WITH_GENERATION), - RPC_OPTIONS_GENERATION)) - .andReturn(UPLOAD_ID); - replay(storageRpcMock); - writer = newWriter(true); - assertTrue(writer.isOpen()); - assertNull(writer.getStorageObject()); - } - - @Test - public void testCreateNonRetryableError() { - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO), EMPTY_RPC_OPTIONS)) - .andThrow(new RuntimeException()); - replay(storageRpcMock); - try { - newWriter(); - Assert.fail(); - } catch (RuntimeException ex) { - assertNotNull(ex.getMessage()); - } - } - - @Test - public void testWriteWithoutFlush() throws Exception { - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO), EMPTY_RPC_OPTIONS)) - .andReturn(UPLOAD_ID); - replay(storageRpcMock); - writer = newWriter(); - assertEquals(MIN_CHUNK_SIZE, writer.write(ByteBuffer.allocate(MIN_CHUNK_SIZE))); - } - - @Test - public void testWriteWithFlushRetryChunk() throws Exception { - ByteBuffer buffer = randomBuffer(MIN_CHUNK_SIZE); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO_WITH_GENERATION), - RPC_OPTIONS_GENERATION)) - .andReturn(UPLOAD_ID); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andThrow(socketClosedException); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(0L); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andReturn(null); - replay(storageRpcMock); - writer = newWriter(true); - writer.setChunkSize(MIN_CHUNK_SIZE); - assertEquals(MIN_CHUNK_SIZE, writer.write(buffer)); - assertTrue(writer.isOpen()); - assertNull(writer.getStorageObject()); - assertArrayEquals(buffer.array(), capturedBuffer.getValue()); - } - - @Test - public void testWriteWithRetryFullChunk() throws Exception { - ByteBuffer buffer = randomBuffer(MIN_CHUNK_SIZE); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO_WITH_GENERATION), - RPC_OPTIONS_GENERATION)) - .andReturn(UPLOAD_ID); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), (byte[]) anyObject(), eq(0), eq(0L), eq(MIN_CHUNK_SIZE), eq(false))) - .andThrow(socketClosedException); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(0L); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andReturn(null); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - (byte[]) anyObject(), - eq(0), - eq((long) MIN_CHUNK_SIZE), - eq(0), - eq(true))) - .andReturn(Conversions.apiary().blobInfo().encode(BLOB_INFO)); - replay(storageRpcMock); - writer = newWriter(true); - writer.setChunkSize(MIN_CHUNK_SIZE); - assertEquals(MIN_CHUNK_SIZE, writer.write(buffer)); - writer.close(); - assertFalse(writer.isOpen()); - assertNotNull(writer.getStorageObject()); - assertArrayEquals(buffer.array(), capturedBuffer.getValue()); - } - - @Test - public void testWriteWithRemoteProgressMade() throws Exception { - ByteBuffer buffer = randomBuffer(MIN_CHUNK_SIZE); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO_WITH_GENERATION), - RPC_OPTIONS_GENERATION)) - .andReturn(UPLOAD_ID); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andThrow(socketClosedException); - // Simulate GCS received 10 bytes but not the rest of the chunk - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(10L); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(10), - eq(10L), - eq(MIN_CHUNK_SIZE - 10), - eq(false))) - .andReturn(null); - replay(storageRpcMock); - writer = newWriter(true); - writer.setChunkSize(MIN_CHUNK_SIZE); - assertEquals(MIN_CHUNK_SIZE, writer.write(buffer)); - assertTrue(writer.isOpen()); - assertNull(writer.getStorageObject()); - assertArrayEquals(buffer.array(), capturedBuffer.getValue()); - } - - @Test - public void testWriteWithDriftRetryCase4() throws Exception { - ByteBuffer buffer = randomBuffer(MIN_CHUNK_SIZE); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO_WITH_GENERATION), - RPC_OPTIONS_GENERATION)) - .andReturn(UPLOAD_ID); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andThrow(socketClosedException); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn((long) MIN_CHUNK_SIZE); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq((long) MIN_CHUNK_SIZE), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andReturn(null); - replay(storageRpcMock); - writer = newWriter(true); - writer.setChunkSize(MIN_CHUNK_SIZE); - assertEquals(MIN_CHUNK_SIZE, writer.write(buffer)); - assertArrayEquals(buffer.array(), capturedBuffer.getValue()); - capturedBuffer.reset(); - buffer.rewind(); - assertEquals(MIN_CHUNK_SIZE, writer.write(buffer)); - assertArrayEquals(buffer.array(), capturedBuffer.getValue()); - assertTrue(writer.isOpen()); - assertNull(writer.getStorageObject()); - } - - @Test - public void testWriteWithUnreachableRemoteOffset() throws Exception { - ByteBuffer buffer = randomBuffer(MIN_CHUNK_SIZE); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO_WITH_GENERATION), - RPC_OPTIONS_GENERATION)) - .andReturn(UPLOAD_ID); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andThrow(socketClosedException); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(MIN_CHUNK_SIZE + 10L); - replay(storageRpcMock); - writer = newWriter(true); - writer.setChunkSize(MIN_CHUNK_SIZE); - try { - writer.write(buffer); - fail("Expected StorageException"); - } catch (StorageException storageException) { - // expected storageException - } - assertTrue(writer.isOpen()); - assertNull(writer.getStorageObject()); - assertArrayEquals(buffer.array(), capturedBuffer.getValue()); - } - - @Test - public void testWriteWithRetryAndObjectMetadata() throws Exception { - ByteBuffer buffer = randomBuffer(MIN_CHUNK_SIZE); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO_WITH_GENERATION), - RPC_OPTIONS_GENERATION)) - .andReturn(UPLOAD_ID); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andThrow(socketClosedException); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(10L); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(10), - eq(10L), - eq(MIN_CHUNK_SIZE - 10), - eq(false))) - .andThrow(socketClosedException); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(-1L); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(-1L); - expect(storageRpcMock.queryCompletedResumableUpload(eq(UPLOAD_ID), eq((long) MIN_CHUNK_SIZE))) - .andThrow(socketClosedException); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(-1L); - expect(storageRpcMock.queryCompletedResumableUpload(eq(UPLOAD_ID), eq((long) MIN_CHUNK_SIZE))) - .andReturn( - Conversions.apiary() - .blobInfo() - .encode(BLOB_INFO) - .setSize(BigInteger.valueOf(MIN_CHUNK_SIZE))); - replay(storageRpcMock); - writer = newWriter(true); - writer.setChunkSize(MIN_CHUNK_SIZE); - assertEquals(MIN_CHUNK_SIZE, writer.write(buffer)); - writer.close(); - assertFalse(writer.isOpen()); - assertNotNull(writer.getStorageObject()); - assertArrayEquals(buffer.array(), capturedBuffer.getValue()); - } - - @Test - public void testWriteWithUploadCompletedByAnotherClient() throws Exception { - ByteBuffer buffer = randomBuffer(MIN_CHUNK_SIZE); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO_WITH_GENERATION), - RPC_OPTIONS_GENERATION)) - .andReturn(UPLOAD_ID); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andReturn(null); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq((long) MIN_CHUNK_SIZE), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andThrow(socketClosedException); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(-1L); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(-1L); - replay(storageRpcMock); - writer = newWriter(true); - writer.setChunkSize(MIN_CHUNK_SIZE); - try { - writer.write(buffer); - buffer.rewind(); - writer.write(buffer); - buffer.rewind(); - writer.write(buffer); - fail("Expected completed exception."); - } catch (StorageException ex) { - - } - assertTrue(writer.isOpen()); - } - - @Test - public void testWriteWithLocalOffsetGoingBeyondRemoteOffset() throws Exception { - ByteBuffer buffer = randomBuffer(MIN_CHUNK_SIZE); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO_WITH_GENERATION), - RPC_OPTIONS_GENERATION)) - .andReturn(UPLOAD_ID); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andReturn(null); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq((long) MIN_CHUNK_SIZE), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andThrow(socketClosedException); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(0L); - replay(storageRpcMock); - writer = newWriter(true); - writer.setChunkSize(MIN_CHUNK_SIZE); - try { - writer.write(buffer); - buffer.rewind(); - writer.write(buffer); - writer.close(); - fail("Expected completed exception."); - } catch (StorageException ex) { - } - assertTrue(writer.isOpen()); - } - - @Test - public void testGetCurrentUploadOffset() throws Exception { - ByteBuffer buffer = randomBuffer(MIN_CHUNK_SIZE); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO_WITH_GENERATION), - RPC_OPTIONS_GENERATION)) - .andReturn(UPLOAD_ID); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andThrow(socketClosedException); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andThrow(socketClosedException); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(0L); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(false))) - .andReturn(null); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - (byte[]) anyObject(), - eq(0), - eq((long) MIN_CHUNK_SIZE), - eq(0), - eq(true))) - .andReturn(Conversions.apiary().blobInfo().encode(BLOB_INFO)); - replay(storageRpcMock); - writer = newWriter(true); - writer.setChunkSize(MIN_CHUNK_SIZE); - assertEquals(MIN_CHUNK_SIZE, writer.write(buffer)); - writer.close(); - assertFalse(writer.isOpen()); - assertNotNull(writer.getStorageObject()); - assertArrayEquals(buffer.array(), capturedBuffer.getValue()); - } - - @Test - public void testWriteWithLastFlushRetryChunkButCompleted() throws Exception { - ByteBuffer buffer = randomBuffer(MIN_CHUNK_SIZE); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO_WITH_GENERATION), - RPC_OPTIONS_GENERATION)) - .andReturn(UPLOAD_ID); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(true))) - .andThrow(socketClosedException); - expect(storageRpcMock.getCurrentUploadOffset(eq(UPLOAD_ID))).andReturn(-1L); - expect(storageRpcMock.queryCompletedResumableUpload(eq(UPLOAD_ID), eq((long) MIN_CHUNK_SIZE))) - .andReturn( - Conversions.apiary() - .blobInfo() - .encode(BLOB_INFO) - .setSize(BigInteger.valueOf(MIN_CHUNK_SIZE))); - replay(storageRpcMock); - writer = newWriter(true); - assertEquals(MIN_CHUNK_SIZE, writer.write(buffer)); - writer.close(); - assertFalse(writer.isRetrying()); - assertFalse(writer.isOpen()); - assertNotNull(writer.getStorageObject()); - // Capture captures entire buffer of a chunk even when not completely used. - // Making assert selective up to the size of MIN_CHUNK_SIZE - assertArrayEquals(Arrays.copyOf(capturedBuffer.getValue(), MIN_CHUNK_SIZE), buffer.array()); - } - - @Test - public void testWriteWithFlush() throws Exception { - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO), EMPTY_RPC_OPTIONS)) - .andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(CUSTOM_CHUNK_SIZE), - eq(false))) - .andReturn(null); - replay(storageRpcMock); - writer = newWriter(); - writer.setChunkSize(CUSTOM_CHUNK_SIZE); - ByteBuffer buffer = randomBuffer(CUSTOM_CHUNK_SIZE); - assertEquals(CUSTOM_CHUNK_SIZE, writer.write(buffer)); - assertArrayEquals(buffer.array(), capturedBuffer.getValue()); - assertNull(writer.getStorageObject()); - } - - @Test - public void testWritesAndFlush() throws Exception { - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO), EMPTY_RPC_OPTIONS)) - .andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(DEFAULT_CHUNK_SIZE), - eq(false))) - .andReturn(null); - replay(storageRpcMock); - writer = newWriter(); - ByteBuffer[] buffers = new ByteBuffer[DEFAULT_CHUNK_SIZE / MIN_CHUNK_SIZE]; - for (int i = 0; i < buffers.length; i++) { - buffers[i] = randomBuffer(MIN_CHUNK_SIZE); - assertEquals(MIN_CHUNK_SIZE, writer.write(buffers[i])); - assertNull(writer.getStorageObject()); - } - for (int i = 0; i < buffers.length; i++) { - assertArrayEquals( - buffers[i].array(), - Arrays.copyOfRange( - capturedBuffer.getValue(), MIN_CHUNK_SIZE * i, MIN_CHUNK_SIZE * (i + 1))); - } - } - - @Test - public void testCloseWithoutFlush() throws Exception { - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO), EMPTY_RPC_OPTIONS)) - .andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), capture(capturedBuffer), eq(0), eq(0L), eq(0), eq(true))) - .andReturn(UPDATED_BLOB); - replay(storageRpcMock); - writer = newWriter(); - assertTrue(writer.isOpen()); - writer.close(); - assertArrayEquals(new byte[0], capturedBuffer.getValue()); - assertFalse(writer.isOpen()); - assertSame(UPDATED_BLOB, writer.getStorageObject()); - } - - @Test - public void testCloseWithFlush() throws Exception { - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO), EMPTY_RPC_OPTIONS)) - .andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(); - ByteBuffer buffer = randomBuffer(MIN_CHUNK_SIZE); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(true))) - .andReturn(UPDATED_BLOB); - replay(storageRpcMock); - writer = newWriter(); - assertTrue(writer.isOpen()); - writer.write(buffer); - writer.close(); - assertEquals(DEFAULT_CHUNK_SIZE, capturedBuffer.getValue().length); - assertArrayEquals(buffer.array(), Arrays.copyOf(capturedBuffer.getValue(), MIN_CHUNK_SIZE)); - assertFalse(writer.isOpen()); - assertSame(UPDATED_BLOB, writer.getStorageObject()); - } - - @Test - public void testWriteClosed() throws Exception { - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO), EMPTY_RPC_OPTIONS)) - .andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), capture(capturedBuffer), eq(0), eq(0L), eq(0), eq(true))) - .andReturn(UPDATED_BLOB); - replay(storageRpcMock); - writer = newWriter(); - writer.close(); - try { - writer.write(ByteBuffer.allocate(MIN_CHUNK_SIZE)); - fail("Expected BlobWriteChannel write to throw IOException"); - } catch (IOException ex) { - // expected - } - assertSame(UPDATED_BLOB, writer.getStorageObject()); - } - - @Test - public void testSaveAndRestore() throws Exception { - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO), EMPTY_RPC_OPTIONS)) - .andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(CaptureType.ALL); - Capture capturedPosition = Capture.newInstance(CaptureType.ALL); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - captureLong(capturedPosition), - eq(DEFAULT_CHUNK_SIZE), - eq(false))) - .andReturn(null); - expectLastCall().times(2); - replay(storageRpcMock); - ByteBuffer buffer1 = randomBuffer(DEFAULT_CHUNK_SIZE); - ByteBuffer buffer2 = randomBuffer(DEFAULT_CHUNK_SIZE); - writer = newWriter(); - assertEquals(DEFAULT_CHUNK_SIZE, writer.write(buffer1)); - assertArrayEquals(buffer1.array(), capturedBuffer.getValues().get(0)); - assertEquals(new Long(0L), capturedPosition.getValues().get(0)); - RestorableState writerState = writer.capture(); - WriteChannel restoredWriter = writerState.restore(); - assertEquals(DEFAULT_CHUNK_SIZE, restoredWriter.write(buffer2)); - assertArrayEquals(buffer2.array(), capturedBuffer.getValues().get(1)); - assertEquals(new Long(DEFAULT_CHUNK_SIZE), capturedPosition.getValues().get(1)); - } - - @Test - public void testSaveAndRestoreClosed() throws Exception { - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO), EMPTY_RPC_OPTIONS)) - .andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), capture(capturedBuffer), eq(0), eq(0L), eq(0), eq(true))) - .andReturn(UPDATED_BLOB); - replay(storageRpcMock); - writer = newWriter(); - writer.close(); - RestorableState writerState = writer.capture(); - RestorableState expectedWriterState = - BlobWriteChannel.StateImpl.builder(options, BLOB_INFO, UPLOAD_ID) - .setBuffer(null) - .setChunkSize(DEFAULT_CHUNK_SIZE) - .setIsOpen(false) - .setPosition(0) - .build(); - WriteChannel restoredWriter = writerState.restore(); - assertArrayEquals(new byte[0], capturedBuffer.getValue()); - assertEquals(expectedWriterState, restoredWriter.capture()); - } - - @Test - public void testStateEquals() { - expect( - storageRpcMock.open( - Conversions.apiary().blobInfo().encode(BLOB_INFO), EMPTY_RPC_OPTIONS)) - .andReturn(UPLOAD_ID) - .times(2); - replay(storageRpcMock); - writer = newWriter(); - // avoid closing when you don't want partial writes to GCS upon failure - @SuppressWarnings("resource") - WriteChannel writer2 = newWriter(); - RestorableState state = writer.capture(); - RestorableState state2 = writer2.capture(); - assertEquals(state, state2); - assertEquals(state.hashCode(), state2.hashCode()); - assertEquals(state.toString(), state2.toString()); - } - - @Test - public void testWriteWithSignedURLAndWithFlush() throws Exception { - expect(storageRpcMock.open(SIGNED_URL)).andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(CUSTOM_CHUNK_SIZE), - eq(false))) - .andReturn(null); - replay(storageRpcMock); - writer = newWriterForSignedUrl(); - writer.setChunkSize(CUSTOM_CHUNK_SIZE); - ByteBuffer buffer = randomBuffer(CUSTOM_CHUNK_SIZE); - assertEquals(CUSTOM_CHUNK_SIZE, writer.write(buffer)); - assertArrayEquals(buffer.array(), capturedBuffer.getValue()); - } - - @Test - public void testWriteWithSignedURLAndFlush() throws Exception { - expect(storageRpcMock.open(SIGNED_URL)).andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(DEFAULT_CHUNK_SIZE), - eq(false))) - .andReturn(null); - replay(storageRpcMock); - writer = newWriterForSignedUrl(); - ByteBuffer[] buffers = new ByteBuffer[DEFAULT_CHUNK_SIZE / MIN_CHUNK_SIZE]; - for (int i = 0; i < buffers.length; i++) { - buffers[i] = randomBuffer(MIN_CHUNK_SIZE); - assertEquals(MIN_CHUNK_SIZE, writer.write(buffers[i])); - } - for (int i = 0; i < buffers.length; i++) { - assertArrayEquals( - buffers[i].array(), - Arrays.copyOfRange( - capturedBuffer.getValue(), MIN_CHUNK_SIZE * i, MIN_CHUNK_SIZE * (i + 1))); - } - } - - @Test - public void testCloseWithSignedURLWithoutFlush() throws Exception { - expect(storageRpcMock.open(SIGNED_URL)).andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), capture(capturedBuffer), eq(0), eq(0L), eq(0), eq(true))) - .andReturn(UPDATED_BLOB); - replay(storageRpcMock); - writer = newWriterForSignedUrl(); - assertTrue(writer.isOpen()); - writer.close(); - assertArrayEquals(new byte[0], capturedBuffer.getValue()); - assertTrue(!writer.isOpen()); - } - - @Test - public void testCloseWithSignedURLWithFlush() throws Exception { - expect(storageRpcMock.open(SIGNED_URL)).andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(); - ByteBuffer buffer = randomBuffer(MIN_CHUNK_SIZE); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - eq(0L), - eq(MIN_CHUNK_SIZE), - eq(true))) - .andReturn(UPDATED_BLOB); - replay(storageRpcMock); - writer = newWriterForSignedUrl(); - assertTrue(writer.isOpen()); - writer.write(buffer); - writer.close(); - assertEquals(DEFAULT_CHUNK_SIZE, capturedBuffer.getValue().length); - assertArrayEquals(buffer.array(), Arrays.copyOf(capturedBuffer.getValue(), MIN_CHUNK_SIZE)); - assertTrue(!writer.isOpen()); - } - - @Test - public void testWriteWithSignedURLClosed() throws Exception { - expect(storageRpcMock.open(SIGNED_URL)).andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), capture(capturedBuffer), eq(0), eq(0L), eq(0), eq(true))) - .andReturn(UPDATED_BLOB); - replay(storageRpcMock); - writer = newWriterForSignedUrl(); - writer.close(); - try { - writer.write(ByteBuffer.allocate(MIN_CHUNK_SIZE)); - fail("Expected BlobWriteChannel write to throw IOException"); - } catch (IOException ex) { - // expected - } - } - - @Test - public void testSaveAndRestoreWithSignedURL() throws Exception { - expect(storageRpcMock.open(SIGNED_URL)).andReturn(UPLOAD_ID); - Capture capturedBuffer = Capture.newInstance(CaptureType.ALL); - Capture capturedPosition = Capture.newInstance(CaptureType.ALL); - expect( - storageRpcMock.writeWithResponse( - eq(UPLOAD_ID), - capture(capturedBuffer), - eq(0), - captureLong(capturedPosition), - eq(DEFAULT_CHUNK_SIZE), - eq(false))) - .andReturn(null); - expectLastCall().times(2); - replay(storageRpcMock); - ByteBuffer buffer1 = randomBuffer(DEFAULT_CHUNK_SIZE); - ByteBuffer buffer2 = randomBuffer(DEFAULT_CHUNK_SIZE); - writer = newWriterForSignedUrl(); - assertEquals(DEFAULT_CHUNK_SIZE, writer.write(buffer1)); - assertArrayEquals(buffer1.array(), capturedBuffer.getValues().get(0)); - assertEquals(new Long(0L), capturedPosition.getValues().get(0)); - RestorableState writerState = writer.capture(); - WriteChannel restoredWriter = writerState.restore(); - assertEquals(DEFAULT_CHUNK_SIZE, restoredWriter.write(buffer2)); - assertArrayEquals(buffer2.array(), capturedBuffer.getValues().get(1)); - assertEquals(new Long(DEFAULT_CHUNK_SIZE), capturedPosition.getValues().get(1)); - } - - private BlobWriteChannel newWriter() { - return newWriter(false); - } - - private BlobWriteChannel newWriter(boolean withGeneration) { - Map optionsMap = - withGeneration ? RPC_OPTIONS_GENERATION : EMPTY_RPC_OPTIONS; - ResultRetryAlgorithm createResultAlgorithm = - retryAlgorithmManager.getForResumableUploadSessionCreate(optionsMap); - ResultRetryAlgorithm writeResultAlgorithm = - retryAlgorithmManager.getForResumableUploadSessionWrite(optionsMap); - final BlobInfo blobInfo = withGeneration ? BLOB_INFO_WITH_GENERATION : BLOB_INFO; - return BlobWriteChannel.newBuilder() - .setStorageOptions(options) - .setBlobInfo(blobInfo) - .setUploadIdSupplier( - ResumableMedia.startUploadForBlobInfo( - options, blobInfo, optionsMap, createResultAlgorithm)) - .setAlgorithmForWrite(writeResultAlgorithm) - .build(); - } - - private BlobWriteChannel newWriterForSignedUrl() throws MalformedURLException { - Map optionsMap = Collections.emptyMap(); - ResultRetryAlgorithm createResultAlgorithm = - retryAlgorithmManager.getForResumableUploadSessionCreate(optionsMap); - ResultRetryAlgorithm writeResultAlgorithm = - retryAlgorithmManager.getForResumableUploadSessionWrite(optionsMap); - return BlobWriteChannel.newBuilder() - .setStorageOptions(options) - .setUploadIdSupplier( - ResumableMedia.startUploadForSignedUrl( - options, new URL(SIGNED_URL), createResultAlgorithm)) - .setAlgorithmForWrite(writeResultAlgorithm) - .build(); - } - - private static ByteBuffer randomBuffer(int size) { - byte[] byteArray = new byte[size]; - RANDOM.nextBytes(byteArray); - return ByteBuffer.wrap(byteArray); - } -} diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/ITJsonResumableSessionPutTaskTest.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/ITJsonResumableSessionPutTaskTest.java index 18704dc28..7ae68773d 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/ITJsonResumableSessionPutTaskTest.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/ITJsonResumableSessionPutTaskTest.java @@ -811,4 +811,28 @@ public void jsonDeserializationOnlyAttemptedWhenContentPresent() throws Exceptio assertThat(operationResult.getPersistedSize()).isEqualTo(0L); } } + + @Test + public void attemptToRewindOutOfBoundsThrows_lower() { + RewindableHttpContent content = RewindableHttpContent.of(); + JsonResumableSessionPutTask task = + new JsonResumableSessionPutTask( + null, null, content, HttpContentRange.of(ByteRangeSpec.relativeLength(10L, 10L))); + + IllegalArgumentException iae = + assertThrows(IllegalArgumentException.class, () -> task.rewindTo(9)); + assertThat(iae).hasMessageThat().isEqualTo("Rewind offset is out of bounds. (10 <= 9 < 20)"); + } + + @Test + public void attemptToRewindOutOfBoundsThrows_upper() { + RewindableHttpContent content = RewindableHttpContent.of(); + JsonResumableSessionPutTask task = + new JsonResumableSessionPutTask( + null, null, content, HttpContentRange.of(ByteRangeSpec.relativeLength(10L, 10L))); + + IllegalArgumentException iae = + assertThrows(IllegalArgumentException.class, () -> task.rewindTo(20)); + assertThat(iae).hasMessageThat().isEqualTo("Rewind offset is out of bounds. (10 <= 20 < 20)"); + } } diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/ITJsonResumableSessionTest.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/ITJsonResumableSessionTest.java index a71f5cf49..1cc917e34 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/ITJsonResumableSessionTest.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/ITJsonResumableSessionTest.java @@ -16,11 +16,15 @@ package com.google.cloud.storage; +import static com.google.cloud.storage.ByteSizeConstants._256KiB; import static com.google.cloud.storage.ByteSizeConstants._256KiBL; +import static com.google.cloud.storage.ByteSizeConstants._512KiB; import static com.google.cloud.storage.ByteSizeConstants._512KiBL; +import static com.google.cloud.storage.ByteSizeConstants._768KiBL; import static com.google.common.truth.Truth.assertThat; import static io.grpc.netty.shaded.io.netty.handler.codec.http.HttpHeaderNames.CONTENT_RANGE; import static io.grpc.netty.shaded.io.netty.handler.codec.http.HttpHeaderNames.RANGE; +import static io.grpc.netty.shaded.io.netty.handler.codec.http.HttpResponseStatus.SERVICE_UNAVAILABLE; import com.google.api.client.http.javanet.NetHttpTransport; import com.google.api.client.json.JsonObjectParser; @@ -38,10 +42,12 @@ import io.grpc.netty.shaded.io.netty.handler.codec.http.HttpRequest; import io.grpc.netty.shaded.io.netty.handler.codec.http.HttpResponseStatus; import java.net.URI; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Before; @@ -54,8 +60,6 @@ public final class ITJsonResumableSessionTest { private static final NetHttpTransport transport = new NetHttpTransport.Builder().build(); private static final HttpResponseStatus RESUME_INCOMPLETE = HttpResponseStatus.valueOf(308, "Resume Incomplete"); - private static final HttpResponseStatus APPEND_GREATER_THAN_CURRENT_SIZE = - HttpResponseStatus.valueOf(503, ""); private static final RetryingDependencies RETRYING_DEPENDENCIES = new RetryingDependencies() { @Override @@ -91,7 +95,6 @@ public void rewindWillQueryStatusOnlyWhenDirty() throws Exception { req -> { requests.add(req); String contentRange = req.headers().get(CONTENT_RANGE); - System.out.println("contentRange = " + contentRange); DefaultFullHttpResponse resp = new DefaultFullHttpResponse(req.protocolVersion(), RESUME_INCOMPLETE); if (range1.getHeaderValue().equals(contentRange)) { @@ -122,7 +125,6 @@ public void rewindWillQueryStatusOnlyWhenDirty() throws Exception { assertThat(operationResult.getPersistedSize()).isEqualTo(_512KiBL); } - assertThat(requests).hasSize(3); List actual = requests.stream().map(r -> r.headers().get(CONTENT_RANGE)).collect(Collectors.toList()); @@ -131,4 +133,135 @@ public void rewindWillQueryStatusOnlyWhenDirty() throws Exception { assertThat(actual).isEqualTo(expected); } + + @Test + public void retryAttemptWillReturnQueryResultIfPersistedSizeMatchesSpecifiedEndOffset() + throws Exception { + HttpContentRange range1 = HttpContentRange.of(ByteRangeSpec.explicit(0L, _512KiBL)); + HttpContentRange range2 = HttpContentRange.query(); + HttpContentRange range3 = HttpContentRange.of(ByteRangeSpec.explicit(_512KiBL, _768KiBL)); + + final List requests = Collections.synchronizedList(new ArrayList<>()); + HttpRequestHandler handler = + req -> { + requests.add(req); + String contentRange = req.headers().get(CONTENT_RANGE); + DefaultFullHttpResponse resp; + if (range1.getHeaderValue().equals(contentRange)) { + resp = new DefaultFullHttpResponse(req.protocolVersion(), SERVICE_UNAVAILABLE); + } else if (range2.getHeaderValue().equals(contentRange)) { + resp = new DefaultFullHttpResponse(req.protocolVersion(), RESUME_INCOMPLETE); + resp.headers().set(RANGE, ByteRangeSpec.explicit(0L, _512KiBL).getHttpRangeHeader()); + } else { + resp = new DefaultFullHttpResponse(req.protocolVersion(), RESUME_INCOMPLETE); + resp.headers() + .set(RANGE, ByteRangeSpec.explicit(_512KiBL, _768KiBL).getHttpRangeHeader()); + } + return resp; + }; + + ByteBuffer buf1 = DataGenerator.base64Characters().genByteBuffer(_512KiB); + ByteBuffer buf2 = DataGenerator.base64Characters().genByteBuffer(_256KiB); + + try (FakeHttpServer fakeHttpServer = FakeHttpServer.of(handler)) { + URI endpoint = fakeHttpServer.getEndpoint(); + String uploadUrl = String.format("%s/upload/%s", endpoint.toString(), UUID.randomUUID()); + + JsonResumableWrite resumableWrite = JsonResumableWrite.of(null, ImmutableMap.of(), uploadUrl); + JsonResumableSession session = + new JsonResumableSession( + httpClientContext, RETRYING_DEPENDENCIES, RETRY_ALGORITHM, resumableWrite); + + ResumableOperationResult<@Nullable StorageObject> operationResult1 = + session.put(RewindableHttpContent.of(buf1), range1); + StorageObject call1 = operationResult1.getObject(); + assertThat(call1).isNull(); + assertThat(operationResult1.getPersistedSize()).isEqualTo(_512KiBL); + + ResumableOperationResult<@Nullable StorageObject> operationResult2 = + session.put(RewindableHttpContent.of(buf2), range3); + StorageObject call2 = operationResult2.getObject(); + assertThat(call2).isNull(); + assertThat(operationResult2.getPersistedSize()).isEqualTo(_768KiBL); + } + + List actual = + requests.stream().map(r -> r.headers().get(CONTENT_RANGE)).collect(Collectors.toList()); + + List expected = + ImmutableList.of(range1.getHeaderValue(), range2.getHeaderValue(), range3.getHeaderValue()); + + assertThat(actual).isEqualTo(expected); + } + + @Test + public void rewindOfContentIsRelativeToItsBeginOffsetOfTheOverallObject() throws Exception { + HttpContentRange range1 = HttpContentRange.of(ByteRangeSpec.explicit(0L, _512KiBL)); + HttpContentRange range2 = HttpContentRange.of(ByteRangeSpec.explicit(_512KiBL, _768KiBL)); + HttpContentRange range3 = HttpContentRange.query(); + + final AtomicBoolean fail = new AtomicBoolean(true); + final List requests = Collections.synchronizedList(new ArrayList<>()); + HttpRequestHandler handler = + req -> { + requests.add(req); + String contentRange = req.headers().get(CONTENT_RANGE); + DefaultFullHttpResponse resp; + if (range1.getHeaderValue().equals(contentRange) + || range3.getHeaderValue().equals(contentRange)) { + resp = new DefaultFullHttpResponse(req.protocolVersion(), RESUME_INCOMPLETE); + resp.headers().set(RANGE, ByteRangeSpec.explicit(0L, _512KiBL).getHttpRangeHeader()); + } else if (range2.getHeaderValue().equals(contentRange)) { + if (fail.getAndSet(false)) { + resp = new DefaultFullHttpResponse(req.protocolVersion(), SERVICE_UNAVAILABLE); + } else { + resp = new DefaultFullHttpResponse(req.protocolVersion(), RESUME_INCOMPLETE); + resp.headers() + .set(RANGE, ByteRangeSpec.explicit(_512KiBL, _768KiBL).getHttpRangeHeader()); + } + } else { + resp = new DefaultFullHttpResponse(req.protocolVersion(), RESUME_INCOMPLETE); + resp.headers() + .set(RANGE, ByteRangeSpec.explicit(_512KiBL, _768KiBL).getHttpRangeHeader()); + } + return resp; + }; + + ByteBuffer buf1 = DataGenerator.base64Characters().genByteBuffer(_512KiB); + ByteBuffer buf2 = DataGenerator.base64Characters().genByteBuffer(_256KiB); + + try (FakeHttpServer fakeHttpServer = FakeHttpServer.of(handler)) { + URI endpoint = fakeHttpServer.getEndpoint(); + String uploadUrl = String.format("%s/upload/%s", endpoint.toString(), UUID.randomUUID()); + + JsonResumableWrite resumableWrite = JsonResumableWrite.of(null, ImmutableMap.of(), uploadUrl); + JsonResumableSession session = + new JsonResumableSession( + httpClientContext, RETRYING_DEPENDENCIES, RETRY_ALGORITHM, resumableWrite); + + ResumableOperationResult<@Nullable StorageObject> operationResult1 = + session.put(RewindableHttpContent.of(buf1), range1); + StorageObject call1 = operationResult1.getObject(); + assertThat(call1).isNull(); + assertThat(operationResult1.getPersistedSize()).isEqualTo(_512KiBL); + + ResumableOperationResult<@Nullable StorageObject> operationResult2 = + session.put(RewindableHttpContent.of(buf2), range2); + StorageObject call2 = operationResult2.getObject(); + assertThat(call2).isNull(); + assertThat(operationResult2.getPersistedSize()).isEqualTo(_768KiBL); + } + + List actual = + requests.stream().map(r -> r.headers().get(CONTENT_RANGE)).collect(Collectors.toList()); + + List expected = + ImmutableList.of( + range1.getHeaderValue(), + range2.getHeaderValue(), + range3.getHeaderValue(), + range2.getHeaderValue()); + + assertThat(actual).isEqualTo(expected); + } } diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/JsonResumableSessionFailureScenarioTest.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/JsonResumableSessionFailureScenarioTest.java index 5f23a5aa1..7f5c7c7ac 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/JsonResumableSessionFailureScenarioTest.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/JsonResumableSessionFailureScenarioTest.java @@ -72,7 +72,7 @@ public void toStorageException_ioExceptionDuringContentResolutionAddedAsSuppress throw new Kaboom(); }); - assertThat(storageException.getCode()).isEqualTo(400); + assertThat(storageException.getCode()).isEqualTo(0); assertThat(storageException).hasCauseThat().isInstanceOf(Cause.class); assertThat(storageException.getSuppressed()).isNotEmpty(); assertThat(storageException.getSuppressed()[0]).isInstanceOf(StorageException.class); diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/PackagePrivateMethodWorkarounds.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/PackagePrivateMethodWorkarounds.java index da5f5310f..d6c5ad0af 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/PackagePrivateMethodWorkarounds.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/PackagePrivateMethodWorkarounds.java @@ -22,7 +22,6 @@ import com.google.cloud.WriteChannel; import com.google.cloud.storage.BucketInfo.BuilderImpl; import com.google.common.collect.ImmutableList; -import com.google.storage.v2.WriteObjectResponse; import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.function.Consumer; @@ -62,14 +61,13 @@ public static Blob blobCopyWithStorage(Blob b, Storage s) { public static Function> maybeGetBlobInfoFunction() { return (w) -> { - BlobWriteChannel blobWriteChannel; - if (w instanceof BlobWriteChannel) { - blobWriteChannel = (BlobWriteChannel) w; - return Optional.of(blobWriteChannel.getStorageObject()) + if (w instanceof BlobWriteChannelV2) { + BlobWriteChannelV2 blobWriteChannel = (BlobWriteChannelV2) w; + return Optional.ofNullable(blobWriteChannel.getResolvedObject()) .map(Conversions.apiary().blobInfo()::decode); } else if (w instanceof GrpcBlobWriteChannel) { GrpcBlobWriteChannel grpcBlobWriteChannel = (GrpcBlobWriteChannel) w; - return Optional.of(grpcBlobWriteChannel.getResults()) + return Optional.of(grpcBlobWriteChannel.getObject()) .map( f -> { try { @@ -77,9 +75,7 @@ public static Function> maybeGetBlobInfoFunctio } catch (InterruptedException | ExecutionException e) { throw new RuntimeException(e); } - }) - .map(WriteObjectResponse::getResource) - .map(Conversions.grpc().blobInfo()::decode); + }); } else { return Optional.empty(); } diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/RewindableByteBufferContentTest.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/RewindableByteBufferContentTest.java new file mode 100644 index 000000000..9d65a00dc --- /dev/null +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/RewindableByteBufferContentTest.java @@ -0,0 +1,160 @@ +/* + * Copyright 2023 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. + */ + +package com.google.cloud.storage; + +import static com.google.cloud.storage.TestUtils.assertAll; +import static com.google.cloud.storage.TestUtils.xxd; +import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertThrows; + +import com.google.cloud.storage.RewindableHttpContentPropertyTest.ErroringOutputStream; +import com.google.protobuf.ByteString; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.junit.Before; +import org.junit.Test; + +public final class RewindableByteBufferContentTest { + + private long total; + private ByteBuffer[] buffers; + private String fullXxd; + + @Before + public void setUp() throws Exception { + // full buffer + ByteBuffer bufFull = DataGenerator.base64Characters().genByteBuffer(16); + // limited buffer + ByteBuffer bufLimit = DataGenerator.base64Characters().genByteBuffer(16); + bufLimit.limit(15); + // offset buffer + ByteBuffer bufOffset = DataGenerator.base64Characters().genByteBuffer(16); + bufOffset.position(3); + // offset and limited buffer + ByteBuffer bufLimitAndOffset = DataGenerator.base64Characters().genByteBuffer(16); + bufLimitAndOffset.position(9).limit(12); + + total = + bufFull.remaining() + + bufLimit.remaining() + + bufOffset.remaining() + + bufLimitAndOffset.remaining(); + buffers = new ByteBuffer[] {bufFull, bufLimit, bufOffset, bufLimitAndOffset}; + fullXxd = xxd(false, buffers); + } + + @Test + public void getLength() { + RewindableHttpContent content = RewindableHttpContent.of(buffers); + + assertThat(content.getLength()).isEqualTo(total); + } + + @Test + public void writeTo() throws IOException { + + RewindableHttpContent content = RewindableHttpContent.of(buffers); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + content.writeTo(baos); + + String actual = xxd(baos.toByteArray()); + assertThat(actual).isEqualTo(fullXxd); + } + + @Test + public void rewind() throws IOException { + + RewindableHttpContent content = RewindableHttpContent.of(buffers); + + assertThrows( + IOException.class, + () -> { + try (ErroringOutputStream erroringOutputStream = new ErroringOutputStream(25)) { + content.writeTo(erroringOutputStream); + } + }); + content.rewindTo(0L); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + content.writeTo(baos); + + String actual = xxd(baos.toByteArray()); + assertThat(actual).isEqualTo(fullXxd); + } + + @Test + public void rewindTo() throws Exception { + RewindableHttpContent content = RewindableHttpContent.of(buffers); + + ByteString reduce = + Arrays.stream(buffers) + .map(ByteBuffer::duplicate) + .map(ByteStringStrategy.noCopy()) + .reduce(ByteString.empty(), ByteString::concat, (l, r) -> r); + + assertThat(content.getLength()).isEqualTo(total); + + int readOffset = 37; + ByteString substring = reduce.substring(readOffset); + ByteBuffer readOnlyByteBuffer = substring.asReadOnlyByteBuffer(); + String expected = xxd(false, readOnlyByteBuffer); + long value = total - readOffset; + content.rewindTo(readOffset); + assertThat(content.getLength()).isEqualTo(value); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + content.writeTo(baos); + + String actual = xxd(baos.toByteArray()); + assertAll( + () -> assertThat(baos.toByteArray()).hasLength(Math.toIntExact(value)), + () -> assertThat(actual).isEqualTo(expected)); + } + + @Test + public void rewind_dirtyAware() throws IOException { + + ByteBuffer buf = DataGenerator.base64Characters().genByteBuffer(10); + buf.position(3).limit(7); + + int position = buf.position(); + int limit = buf.limit(); + + RewindableHttpContent content = RewindableHttpContent.of(buf); + int hackPosition = 2; + // after content has initialized, mutate the position underneath it. We're doing this to detect + // if rewind is actually modifying things. It shouldn't until the content is dirtied by calling + // writeTo + buf.position(hackPosition); + + // invoke rewind, and expect it to not do anything + content.rewindTo(0L); + assertThat(buf.position()).isEqualTo(hackPosition); + assertThat(buf.limit()).isEqualTo(limit); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + content.writeTo(baos); + + assertThat(buf.position()).isEqualTo(limit); + assertThat(buf.limit()).isEqualTo(limit); + + content.rewindTo(0L); + assertThat(buf.position()).isEqualTo(position); + assertThat(buf.limit()).isEqualTo(limit); + } +} diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/RewindableHttpContentPropertyTest.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/RewindableHttpContentPropertyTest.java index 5d2353777..f8e18f286 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/RewindableHttpContentPropertyTest.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/RewindableHttpContentPropertyTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertThrows; import com.google.common.base.MoreObjects; +import com.google.protobuf.ByteString; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.OutputStream; @@ -30,6 +31,7 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.util.Arrays; +import java.util.stream.Collectors; import net.jqwik.api.Arbitraries; import net.jqwik.api.Arbitrary; import net.jqwik.api.Combinators; @@ -64,6 +66,29 @@ void path(@ForAll("PathScenario") PathScenario pathScenario) throws Exception { } } + @Property + void byteBuffers(@ForAll("ByteBuffersScenario") ByteBuffersScenario s) throws IOException { + RewindableHttpContent content = RewindableHttpContent.of(s.getBuffers()); + assertThat(content.getLength()).isEqualTo(s.getFullLength()); + assertThrows( + IOException.class, + () -> { + try (ErroringOutputStream erroringOutputStream = + new ErroringOutputStream(s.getErrorAtOffset())) { + content.writeTo(erroringOutputStream); + } + }); + content.rewindTo(s.getRewindOffset()); + assertThat(content.getLength()).isEqualTo(s.getPostRewindLength()); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + content.writeTo(baos); + + String actual = xxd(baos.toByteArray()); + + assertThat(actual).isEqualTo(s.getExpectedXxd()); + } + @Provide("PathScenario") static Arbitrary pathScenario() { return Arbitraries.lazyOf( @@ -85,6 +110,30 @@ static Arbitrary pathScenario() { .as(PathScenario::of))); } + @Provide("ByteBuffersScenario") + static Arbitrary byteBuffersScenarioArbitrary() { + return Arbitraries.lazyOf( + () -> + Arbitraries.oneOf( + byteBuffers(1, 10), + byteBuffers(10, 100), + byteBuffers(100, 1_000), + byteBuffers(1_000, 10_000), + byteBuffers(10_000, 100_000), + byteBuffers(100_000, 1_000_000))) + .flatMap( + buffers -> { + long totalAvailable = Arrays.stream(buffers).mapToLong(ByteBuffer::remaining).sum(); + + return Combinators.combine( + Arbitraries.longs().between(0, Math.max(0L, totalAvailable - 1)), + Arbitraries.longs().between(0, Math.max(0L, totalAvailable - 1)), + Arbitraries.just(buffers)) + .as(ByteBuffersScenario::of); + }) + .filter(bbs -> bbs.getFullLength() > 0); + } + @NonNull private static Arbitrary bytes(int minFileSize, int maxFileSize) { return Arbitraries.integers() @@ -93,6 +142,43 @@ private static Arbitrary bytes(int minFileSize, int maxFileSize) { .map(DataGenerator.base64Characters()::genBytes); } + @NonNull + private static Arbitrary byteBuffers(int perBufferMinSize, int perBufferMaxSize) { + return byteBuffer(perBufferMinSize, perBufferMaxSize) + .array(ByteBuffer[].class) + .ofMinSize(1) + .ofMaxSize(10); + } + + /** + * Generate a ByteBuffer with size between minSize, maxSize with a random position and random + * limit + */ + @NonNull + private static Arbitrary byteBuffer(int minSize, int maxSize) { + return Arbitraries.integers() + .between(minSize, maxSize) + .withDistribution(RandomDistribution.uniform()) + .withoutEdgeCases() + .map(DataGenerator.base64Characters()::genByteBuffer) + .flatMap( + buf -> + Arbitraries.integers() + .between(0, Math.max(0, buf.capacity() - 1)) + .withoutEdgeCases() + .flatMap( + limit -> + Arbitraries.integers() + .between(0, limit) + .withoutEdgeCases() + .flatMap( + position -> { + buf.limit(limit); + buf.position(position); + return Arbitraries.of(buf); + }))); + } + private static final class PathScenario implements AutoCloseable { private static final Path TMP_DIR = Paths.get(System.getProperty("java.io.tmpdir")); @@ -163,6 +249,81 @@ private static PathScenario of(int rewindOffset, int errorAtOffset, byte[] bytes } } + private static class ByteBuffersScenario { + + private final long rewindOffset; + private final long errorAtOffset; + private final ByteBuffer[] buffers; + private final long fullLength; + private final String expectedXxd; + + private ByteBuffersScenario( + long rewindOffset, + long errorAtOffset, + ByteBuffer[] buffers, + byte[] expectedBytes, + long fullLength) { + this.rewindOffset = rewindOffset; + this.errorAtOffset = errorAtOffset; + this.buffers = buffers; + this.fullLength = fullLength; + this.expectedXxd = xxd(expectedBytes); + } + + public long getRewindOffset() { + return rewindOffset; + } + + public long getErrorAtOffset() { + return errorAtOffset; + } + + public ByteBuffer[] getBuffers() { + // duplicate the buffer so we have stable toString + return Arrays.stream(buffers).map(ByteBuffer::duplicate).toArray(ByteBuffer[]::new); + } + + public String getExpectedXxd() { + return expectedXxd; + } + + public long getFullLength() { + return fullLength; + } + + public long getPostRewindLength() { + return fullLength - rewindOffset; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("\nexpectedXxd", "\n" + expectedXxd) + .add( + "\nbuffers", + Arrays.stream(buffers) + .map(Object::toString) + .collect(Collectors.joining("\n\t", "[\n\t", "\n]"))) + .add("\nrewindOffset", rewindOffset) + .add("\nerrorAtOffset", errorAtOffset) + .toString(); + } + + public static ByteBuffersScenario of( + long rewindOffset, long errorAtOffset, ByteBuffer[] buffers) { + + ByteString reduce = + Arrays.stream(buffers) + .map(ByteBuffer::duplicate) + .map(ByteStringStrategy.noCopy()) + .reduce(ByteString.empty(), ByteString::concat, (l, r) -> r); + + byte[] byteArray = reduce.substring(Math.toIntExact(rewindOffset)).toByteArray(); + return new ByteBuffersScenario( + rewindOffset, errorAtOffset, buffers, byteArray, reduce.size()); + } + } + static final class ErroringOutputStream extends OutputStream { private final long errorAt; private long totalWritten; @@ -189,7 +350,8 @@ public void write(byte[] b) throws IOException { } @Override - public void write(byte[] b, int off, int len) throws IOException { + public void write(@SuppressWarnings("NullableProblems") byte[] b, int off, int len) + throws IOException { int diff = len - off; if (totalWritten + diff >= errorAt) { throw new IOException("Reached errorAt limit"); diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/SerializationTest.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/SerializationTest.java index 9fc4b16f7..1e5eda2d4 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/SerializationTest.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/SerializationTest.java @@ -20,7 +20,6 @@ import static com.google.common.truth.Truth.assertThat; import static org.junit.Assert.assertEquals; -import com.google.api.gax.retrying.ResultRetryAlgorithm; import com.google.api.services.storage.model.StorageObject; import com.google.cloud.BaseSerializationTest; import com.google.cloud.NoCredentials; @@ -28,10 +27,11 @@ import com.google.cloud.ReadChannel; import com.google.cloud.Restorable; import com.google.cloud.RestorableState; +import com.google.cloud.WriteChannel; import com.google.cloud.storage.Acl.Project.ProjectRole; -import com.google.cloud.storage.BlobReadChannel.StateImpl; import com.google.cloud.storage.BlobReadChannelV2.BlobReadChannelContext; import com.google.cloud.storage.BlobReadChannelV2.BlobReadChannelV2State; +import com.google.cloud.storage.BlobWriteChannelV2.BlobWriteChannelV2State; import com.google.cloud.storage.Storage.BucketField; import com.google.cloud.storage.Storage.PredefinedAcl; import com.google.cloud.storage.UnifiedOpts.Opt; @@ -205,16 +205,18 @@ protected Serializable[] serializableObjects() { @SuppressWarnings("resource") protected Restorable[] restorableObjects() { HttpStorageOptions options = HttpStorageOptions.newBuilder().setProjectId("p2").build(); - ResultRetryAlgorithm algorithm = - options.getRetryAlgorithmManager().getForResumableUploadSessionWrite(EMPTY_RPC_OPTIONS); ReadChannel readerV2 = new BlobReadChannelV2( new StorageObject().setBucket("b").setName("n"), EMPTY_RPC_OPTIONS, BlobReadChannelContext.from(options)); - BlobWriteChannel writer = - new BlobWriteChannel( - options, BlobInfo.newBuilder(BlobId.of("b", "n")).build(), "upload-id", algorithm); + WriteChannel writer = + new BlobWriteChannelV2( + BlobReadChannelContext.from(options), + JsonResumableWrite.of( + Conversions.apiary().blobInfo().encode(BlobInfo.newBuilder("b", "n").build()), + ImmutableMap.of(), + "upload-id")); return new Restorable[] {readerV2, writer}; } @@ -227,7 +229,7 @@ public void restoreOfV1BlobReadChannelShouldReturnV2Channel() try (InputStream is = SerializationTest.class .getClassLoader() - .getResourceAsStream("com/google/cloud/storage/blobWriteChannel.ser.properties")) { + .getResourceAsStream("com/google/cloud/storage/blobReadChannel.ser.properties")) { properties.load(is); } String b64bytes = properties.getProperty("b64bytes"); @@ -239,8 +241,8 @@ public void restoreOfV1BlobReadChannelShouldReturnV2Channel() Object o = ois.readObject(); assertThat(o).isInstanceOf(RestorableState.class); RestorableState restorableState = (RestorableState) o; - assertThat(o).isInstanceOf(StateImpl.class); - StateImpl state = (StateImpl) restorableState; + assertThat(o).isInstanceOf(BlobReadChannel.StateImpl.class); + BlobReadChannel.StateImpl state = (BlobReadChannel.StateImpl) restorableState; ReadChannel restore = state.restore(); assertThat(restore).isInstanceOf(BlobReadChannelV2.class); RestorableState capture = restore.capture(); @@ -248,6 +250,36 @@ public void restoreOfV1BlobReadChannelShouldReturnV2Channel() } } + @SuppressWarnings({"deprecation", "rawtypes"}) + @Test + public void restoreOfV1BlobWriteChannelShouldReturnV2Channel() + throws IOException, ClassNotFoundException { + + Properties properties = new Properties(); + try (InputStream is = + SerializationTest.class + .getClassLoader() + .getResourceAsStream("com/google/cloud/storage/blobWriteChannel.ser.properties")) { + properties.load(is); + } + String b64bytes = properties.getProperty("b64bytes"); + assertThat(b64bytes).isNotEmpty(); + + byte[] decode = Base64.getDecoder().decode(b64bytes); + try (ByteArrayInputStream bais = new ByteArrayInputStream(decode); + ObjectInputStream ois = new ObjectInputStream(bais)) { + Object o = ois.readObject(); + assertThat(o).isInstanceOf(RestorableState.class); + RestorableState restorableState = (RestorableState) o; + assertThat(o).isInstanceOf(BlobWriteChannel.StateImpl.class); + BlobWriteChannel.StateImpl state = (BlobWriteChannel.StateImpl) restorableState; + WriteChannel restore = state.restore(); + assertThat(restore).isInstanceOf(BlobWriteChannelV2.class); + RestorableState capture = restore.capture(); + assertThat(capture).isInstanceOf(BlobWriteChannelV2State.class); + } + } + /** * Here we override the super classes implementation to remove the "assertNotSame". * diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/StorageImplMockitoTest.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/StorageImplMockitoTest.java index 8395c6d0a..1d1453402 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/StorageImplMockitoTest.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/StorageImplMockitoTest.java @@ -18,7 +18,6 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; @@ -32,7 +31,6 @@ import com.google.api.services.storage.model.StorageObject; import com.google.cloud.ServiceOptions; import com.google.cloud.Tuple; -import com.google.cloud.WriteChannel; import com.google.cloud.storage.Storage.BlobTargetOption; import com.google.cloud.storage.spi.StorageRpcFactory; import com.google.cloud.storage.spi.v1.StorageRpc; @@ -42,7 +40,6 @@ import com.google.common.io.BaseEncoding; import java.io.ByteArrayInputStream; import java.io.IOException; -import java.io.InputStream; import java.math.BigInteger; import java.nio.file.Files; import java.nio.file.Path; @@ -983,130 +980,6 @@ private BlobInfo initializeUpload( return blobInfo; } - @Test - public void testCreateFromStream() throws Exception { - byte[] dataToSend = {1, 2, 3, 4, 5}; - ByteArrayInputStream stream = new ByteArrayInputStream(dataToSend); - - BlobInfo blobInfo = initializeUpload(dataToSend); - Blob blob = storage.createFrom(blobInfo, stream); - assertEquals(expectedUpdated, blob); - } - - @Test - public void testCreateFromWithOptions() throws Exception { - byte[] dataToSend = {1, 2, 3, 4, 5, 6}; - ByteArrayInputStream stream = new ByteArrayInputStream(dataToSend); - - BlobInfo blobInfo = initializeUpload(dataToSend, DEFAULT_BUFFER_SIZE, KMS_KEY_NAME_OPTIONS); - Blob blob = - storage.createFrom(blobInfo, stream, Storage.BlobWriteOption.kmsKeyName(KMS_KEY_NAME)); - assertEquals(expectedUpdated, blob); - } - - @Test - public void testCreateFromWithBufferSize() throws Exception { - byte[] dataToSend = {1, 2, 3, 4, 5, 6}; - ByteArrayInputStream stream = new ByteArrayInputStream(dataToSend); - int bufferSize = MIN_BUFFER_SIZE * 2; - - BlobInfo blobInfo = initializeUpload(dataToSend, bufferSize); - Blob blob = storage.createFrom(blobInfo, stream, bufferSize); - assertEquals(expectedUpdated, blob); - } - - @Test - public void testCreateFromWithBufferSizeAndOptions() throws Exception { - byte[] dataToSend = {1, 2, 3, 4, 5, 6}; - ByteArrayInputStream stream = new ByteArrayInputStream(dataToSend); - int bufferSize = MIN_BUFFER_SIZE * 2; - - BlobInfo blobInfo = initializeUpload(dataToSend, bufferSize, KMS_KEY_NAME_OPTIONS); - Blob blob = - storage.createFrom( - blobInfo, stream, bufferSize, Storage.BlobWriteOption.kmsKeyName(KMS_KEY_NAME)); - assertEquals(expectedUpdated, blob); - } - - @Test - public void testCreateFromWithSmallBufferSize() throws Exception { - byte[] dataToSend = new byte[100_000]; - ByteArrayInputStream stream = new ByteArrayInputStream(dataToSend); - int smallBufferSize = 100; - - BlobInfo blobInfo = initializeUpload(dataToSend, MIN_BUFFER_SIZE); - Blob blob = storage.createFrom(blobInfo, stream, smallBufferSize); - assertEquals(expectedUpdated, blob); - } - - @Test - public void testCreateFromWithException() throws Exception { - initializeService(); - String uploadId = "id-exception"; - byte[] bytes = new byte[10]; - byte[] buffer = new byte[MIN_BUFFER_SIZE]; - System.arraycopy(bytes, 0, buffer, 0, bytes.length); - BlobInfo info = BLOB_INFO1.toBuilder().setMd5(null).setCrc32c(null).build(); - doReturn(uploadId) - .doThrow(UNEXPECTED_CALL_EXCEPTION) - .when(storageRpcMock) - .open(Conversions.apiary().blobInfo().encode(info), EMPTY_RPC_OPTIONS); - - Exception runtimeException = new RuntimeException("message"); - doThrow(runtimeException) - .when(storageRpcMock) - .writeWithResponse(uploadId, buffer, 0, 0L, bytes.length, true); - - InputStream input = new ByteArrayInputStream(bytes); - try { - storage.createFrom(info, input, MIN_BUFFER_SIZE); - fail(); - } catch (StorageException e) { - assertSame(runtimeException, e.getCause()); - } - } - - @Test - public void testCreateFromMultipleParts() throws Exception { - initializeService(); - String uploadId = "id-multiple-parts"; - int extraBytes = 10; - int totalSize = MIN_BUFFER_SIZE + extraBytes; - byte[] dataToSend = new byte[totalSize]; - dataToSend[0] = 42; - dataToSend[MIN_BUFFER_SIZE + 1] = 43; - - StorageObject storageObject = new StorageObject(); - storageObject.setBucket(BLOB_INFO1.getBucket()); - storageObject.setName(BLOB_INFO1.getName()); - storageObject.setSize(BigInteger.valueOf(totalSize)); - - BlobInfo info = BLOB_INFO1.toBuilder().setMd5(null).setCrc32c(null).build(); - doReturn(uploadId) - .doThrow(UNEXPECTED_CALL_EXCEPTION) - .when(storageRpcMock) - .open(Conversions.apiary().blobInfo().encode(info), EMPTY_RPC_OPTIONS); - - byte[] buffer1 = new byte[MIN_BUFFER_SIZE]; - System.arraycopy(dataToSend, 0, buffer1, 0, MIN_BUFFER_SIZE); - doReturn(null) - .doThrow(UNEXPECTED_CALL_EXCEPTION) - .when(storageRpcMock) - .writeWithResponse(uploadId, buffer1, 0, 0L, MIN_BUFFER_SIZE, false); - - byte[] buffer2 = new byte[MIN_BUFFER_SIZE]; - System.arraycopy(dataToSend, MIN_BUFFER_SIZE, buffer2, 0, extraBytes); - doReturn(storageObject) - .doThrow(UNEXPECTED_CALL_EXCEPTION) - .when(storageRpcMock) - .writeWithResponse(uploadId, buffer2, 0, (long) MIN_BUFFER_SIZE, extraBytes, true); - - InputStream input = new ByteArrayInputStream(dataToSend); - Blob blob = storage.createFrom(info, input, MIN_BUFFER_SIZE); - BlobInfo info1 = Conversions.apiary().blobInfo().decode(storageObject); - assertEquals(info1.asBlob(storage), blob); - } - @Test public void testListBuckets() { String cursor = "cursor"; @@ -1330,85 +1203,6 @@ public void testListBlobsWithException() { } } - @Test - public void testWriter() { - // verify that md5 and crc32c are cleared if present when calling create - doReturn("upload-id") - .doThrow(UNEXPECTED_CALL_EXCEPTION) - .when(storageRpcMock) - .open(Conversions.apiary().blobInfo().encode(BLOB_INFO_WITHOUT_HASHES), EMPTY_RPC_OPTIONS); - initializeService(); - WriteChannel channel = storage.writer(BLOB_INFO_WITH_HASHES); - assertNotNull(channel); - assertTrue(channel.isOpen()); - } - - @Test - public void testWriterWithOptions() { - BlobInfo info = BLOB_INFO1.toBuilder().setMd5(CONTENT_MD5).setCrc32c(CONTENT_CRC32C).build(); - doReturn("upload-id") - .doThrow(UNEXPECTED_CALL_EXCEPTION) - .when(storageRpcMock) - .open(Conversions.apiary().blobInfo().encode(info), BLOB_TARGET_OPTIONS_CREATE); - initializeService(); - WriteChannel channel = - storage.writer( - info, - BLOB_WRITE_METAGENERATION, - BLOB_WRITE_NOT_EXIST, - BLOB_WRITE_PREDEFINED_ACL, - BLOB_WRITE_CRC2C, - BLOB_WRITE_MD5_HASH); - assertNotNull(channel); - assertTrue(channel.isOpen()); - } - - @Test - public void testWriterWithEncryptionKey() { - BlobInfo info = BLOB_INFO1.toBuilder().setMd5(null).setCrc32c(null).build(); - doReturn("upload-id-1", "upload-id-2") - .doThrow(UNEXPECTED_CALL_EXCEPTION) - .when(storageRpcMock) - .open(Conversions.apiary().blobInfo().encode(info), ENCRYPTION_KEY_OPTIONS); - initializeService(); - WriteChannel channel = storage.writer(info, Storage.BlobWriteOption.encryptionKey(KEY)); - assertNotNull(channel); - assertTrue(channel.isOpen()); - channel = storage.writer(info, Storage.BlobWriteOption.encryptionKey(BASE64_KEY)); - assertNotNull(channel); - assertTrue(channel.isOpen()); - } - - @Test - public void testWriterWithKmsKeyName() { - BlobInfo info = BLOB_INFO1.toBuilder().setMd5(null).setCrc32c(null).build(); - doReturn("upload-id-1", "upload-id-2") - .doThrow(UNEXPECTED_CALL_EXCEPTION) - .when(storageRpcMock) - .open(Conversions.apiary().blobInfo().encode(info), KMS_KEY_NAME_OPTIONS); - initializeService(); - WriteChannel channel = storage.writer(info, Storage.BlobWriteOption.kmsKeyName(KMS_KEY_NAME)); - assertNotNull(channel); - assertTrue(channel.isOpen()); - channel = storage.writer(info, Storage.BlobWriteOption.kmsKeyName(KMS_KEY_NAME)); - assertNotNull(channel); - assertTrue(channel.isOpen()); - } - - @Test - public void testWriterFailure() { - doThrow(STORAGE_FAILURE) - .when(storageRpcMock) - .open(Conversions.apiary().blobInfo().encode(BLOB_INFO_WITHOUT_HASHES), EMPTY_RPC_OPTIONS); - initializeService(); - try { - storage.writer(BLOB_INFO_WITH_HASHES); - fail(); - } catch (StorageException e) { - assertSame(STORAGE_FAILURE, e.getCause()); - } - } - @Test public void testCreateNotification() { doReturn(Conversions.apiary().notificationInfo().encode(NOTIFICATION_INFO_01)) diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/TestUtils.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/TestUtils.java index 875a3b0d8..af92080a7 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/TestUtils.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/TestUtils.java @@ -226,11 +226,27 @@ public static String xxd(byte[] bytes) { } public static String xxd(ByteBuffer bytes) { + return xxd(true, bytes); + } + + public static String xxd(boolean flip, ByteBuffer bytes) { ByteBuffer dup = bytes.duplicate(); - dup.flip(); + if (flip) dup.flip(); return ByteBufUtil.prettyHexDump(Unpooled.wrappedBuffer(dup)); } + public static String xxd(boolean flip, ByteBuffer[] buffers) { + ByteBuffer[] dups = + Arrays.stream(buffers) + .map(ByteBuffer::duplicate) + .peek( + byteBuffer -> { + if (flip) byteBuffer.flip(); + }) + .toArray(ByteBuffer[]::new); + return ByteBufUtil.prettyHexDump(Unpooled.wrappedBuffer(dups)); + } + public static void assertAll(ThrowingRunnable... trs) throws Exception { List x = Arrays.stream(trs) diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/TmpFile.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/TmpFile.java index a8c846c6c..eef1b087d 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/TmpFile.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/TmpFile.java @@ -16,6 +16,7 @@ package com.google.cloud.storage; +import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableSet; import java.io.IOException; import java.nio.channels.SeekableByteChannel; @@ -54,6 +55,11 @@ public void close() throws IOException { Files.delete(path); } + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("path", path).toString(); + } + /** * Create a temporary file, which will be deleted when close is called on the returned {@link * TmpFile} diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/conformance/retry/ITRetryConformanceTest.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/conformance/retry/ITRetryConformanceTest.java index d42755ef0..612879c4f 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/conformance/retry/ITRetryConformanceTest.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/conformance/retry/ITRetryConformanceTest.java @@ -397,6 +397,11 @@ static BiPredicate scenarioIdIs(int scenarioId) return (m, trc) -> trc.getScenarioId() == scenarioId; } + static BiPredicate mappingIdIn(Integer... mappingIds) { + ImmutableSet ids = ImmutableSet.copyOf(mappingIds); + return (m, trc) -> ids.contains(trc.getMappingId()); + } + static final class Builder { private String retryTestsJsonResourcePath; private RpcMethodMappings mappings; diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/it/ITBlobWriteChannelTest.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/it/ITBlobWriteChannelTest.java index e81f63e3c..9b4ab3668 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/it/ITBlobWriteChannelTest.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/it/ITBlobWriteChannelTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import com.google.api.client.json.JsonParser; @@ -38,26 +39,21 @@ import com.google.cloud.storage.PackagePrivateMethodWorkarounds; import com.google.cloud.storage.Storage; import com.google.cloud.storage.Storage.BlobWriteOption; -import com.google.cloud.storage.StorageException; import com.google.cloud.storage.StorageOptions; +import com.google.cloud.storage.TransportCompatibility.Transport; import com.google.cloud.storage.it.runner.StorageITRunner; import com.google.cloud.storage.it.runner.annotations.Backend; import com.google.cloud.storage.it.runner.annotations.Inject; import com.google.cloud.storage.it.runner.annotations.SingleBackend; +import com.google.cloud.storage.it.runner.annotations.StorageFixture; import com.google.cloud.storage.it.runner.registry.Generator; import com.google.cloud.storage.it.runner.registry.TestBench; import com.google.cloud.storage.it.runner.registry.TestBench.RetryTestResource; -import com.google.cloud.storage.spi.StorageRpcFactory; -import com.google.cloud.storage.spi.v1.StorageRpc; -import com.google.cloud.storage.spi.v1.StorageRpc.Option; import com.google.common.collect.ImmutableMap; -import com.google.common.reflect.AbstractInvocationHandler; -import com.google.common.reflect.Reflection; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Optional; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.logging.Logger; import org.junit.Test; import org.junit.runner.RunWith; @@ -84,6 +80,11 @@ public final class ITBlobWriteChannelTest { @Inject public TestBench testBench; + @Inject + @StorageFixture(Transport.HTTP) + public Storage storage; + + @Inject public BucketInfo bucket; @Inject public Generator generator; /** @@ -110,24 +111,6 @@ public void testJsonEOF_10B() throws IOException { doJsonUnexpectedEOFTest(contentSize, cappedByteCount); } - @Test - public void blobWriteChannel_handlesRecoveryOnLastChunkWhenGenerationIsPresent_multipleChunks() - throws IOException { - int _2MiB = 256 * 1024; - int contentSize = 292_617; - - blobWriteChannel_handlesRecoveryOnLastChunkWhenGenerationIsPresent(_2MiB, contentSize); - } - - @Test - public void blobWriteChannel_handlesRecoveryOnLastChunkWhenGenerationIsPresent_singleChunk() - throws IOException { - int _4MiB = 256 * 1024 * 2; - int contentSize = 292_617; - - blobWriteChannel_handlesRecoveryOnLastChunkWhenGenerationIsPresent(_4MiB, contentSize); - } - @Test public void testWriteChannelExistingBlob() throws IOException { HttpStorageOptions baseStorageOptions = @@ -156,6 +139,20 @@ public void testWriteChannelExistingBlob() throws IOException { assertTrue(storage.delete(bucketInfo.getName(), blobInfo.getName())); } + @Test + public void changeChunkSizeAfterWrite() throws IOException { + BlobInfo info = BlobInfo.newBuilder(bucket, generator.randomObjectName()).build(); + System.out.println("info = " + info); + + int _512KiB = 512 * 1024; + byte[] bytes = DataGenerator.base64Characters().genBytes(_512KiB + 13); + try (WriteChannel writer = storage.writer(info, BlobWriteOption.doesNotExist())) { + writer.setChunkSize(2 * 1024 * 1024); + writer.write(ByteBuffer.wrap(bytes, 0, _512KiB)); + assertThrows(IllegalStateException.class, () -> writer.setChunkSize(768 * 1024)); + } + } + private void doJsonUnexpectedEOFTest(int contentSize, int cappedByteCount) throws IOException { String blobPath = String.format("%s/%s/blob", generator.randomObjectName(), NOW_STRING); @@ -176,47 +173,11 @@ private void doJsonUnexpectedEOFTest(int contentSize, int cappedByteCount) throw .setCredentials(NoCredentials.getInstance()) .setHost(testBench.getBaseUri()) .setProjectId("project-id") - .build(); - StorageRpc noHeader = (StorageRpc) baseOptions.getRpc(); - StorageRpc yesHeader = - (StorageRpc) - baseOptions - .toBuilder() - .setHeaderProvider( - FixedHeaderProvider.create(ImmutableMap.of("x-retry-test-id", retryTest.id))) - .build() - .getRpc(); - - StorageOptions storageOptions = - baseOptions - .toBuilder() - .setServiceRpcFactory( - options -> - Reflection.newProxy( - StorageRpc.class, - (proxy, method, args) -> { - try { - if ("writeWithResponse".equals(method.getName())) { - boolean lastChunk = (boolean) args[5]; - LOGGER.fine( - String.format( - "writeWithResponse called. (lastChunk = %b)", lastChunk)); - if (lastChunk) { - return method.invoke(yesHeader, args); - } - } - return method.invoke(noHeader, args); - } catch (Exception e) { - if (e.getCause() != null) { - throw e.getCause(); - } else { - throw e; - } - } - })) + .setHeaderProvider( + FixedHeaderProvider.create(ImmutableMap.of("x-retry-test-id", retryTest.id))) .build(); - Storage testStorage = storageOptions.getService(); + Storage testStorage = baseOptions.getService(); testStorage.create(bucketInfo); @@ -234,7 +195,7 @@ private void doJsonUnexpectedEOFTest(int contentSize, int cappedByteCount) throw Optional optionalStorageObject = PackagePrivateMethodWorkarounds.maybeGetBlobInfoFunction().apply(w); - assertTrue(optionalStorageObject.isPresent()); + assertThat(optionalStorageObject.isPresent()).isTrue(); BlobInfo internalInfo = optionalStorageObject.get(); assertThat(internalInfo.getName()).isEqualTo(blobInfoGen0.getName()); @@ -248,100 +209,4 @@ private void doJsonUnexpectedEOFTest(int contentSize, int cappedByteCount) throw ByteBuffer actual = ByteBuffer.wrap(actualData.toByteArray()); assertEquals(expected, actual); } - - private void blobWriteChannel_handlesRecoveryOnLastChunkWhenGenerationIsPresent( - int chunkSize, int contentSize) throws IOException { - Instant now = Clock.systemUTC().instant(); - DateTimeFormatter formatter = - DateTimeFormatter.ISO_LOCAL_DATE_TIME.withZone(ZoneId.from(ZoneOffset.UTC)); - String nowString = formatter.format(now); - BucketInfo bucketInfo = BucketInfo.of(generator.randomBucketName()); - String blobPath = String.format("%s/%s/blob", generator.randomObjectName(), nowString); - BlobId blobId = BlobId.of(bucketInfo.getName(), blobPath); - BlobInfo blobInfo = BlobInfo.newBuilder(blobId).build(); - - ByteBuffer contentGen1 = DataGenerator.base64Characters().genByteBuffer(contentSize); - ByteBuffer contentGen2 = DataGenerator.base64Characters().genByteBuffer(contentSize); - ByteBuffer contentGen2Expected = contentGen2.duplicate(); - HttpStorageOptions baseStorageOptions = - StorageOptions.http() - .setCredentials(NoCredentials.getInstance()) - .setHost(testBench.getBaseUri()) - .setProjectId("test-project-id") - .build(); - Storage storage = baseStorageOptions.getService(); - storage.create(bucketInfo); - WriteChannel ww = storage.writer(blobInfo); - ww.setChunkSize(chunkSize); - ww.write(contentGen1); - ww.close(); - - Blob blobGen1 = storage.get(blobId); - - final AtomicBoolean exceptionThrown = new AtomicBoolean(false); - - Storage testStorage = - baseStorageOptions - .toBuilder() - .setServiceRpcFactory( - new StorageRpcFactory() { - /** - * Here we're creating a proxy of StorageRpc where we can delegate all calls to - * the normal implementation, except in the case of {@link - * StorageRpc#writeWithResponse(String, byte[], int, long, int, boolean)} where - * {@code lastChunk == true}. We allow the call to execute, but instead of - * returning the result we throw an IOException to simulate a prematurely close - * connection. This behavior is to ensure appropriate handling of a completed - * upload where the ACK wasn't received. In particular, if an upload is initiated - * against an object where an {@link Option#IF_GENERATION_MATCH} simply calling - * get on an object can result in a 404 because the object that is created while - * the BlobWriteChannel is executing will be a new generation. - */ - @Override - public StorageRpc create(final StorageOptions options) { - return Reflection.newProxy( - StorageRpc.class, - new AbstractInvocationHandler() { - final StorageRpc delegate = (StorageRpc) baseStorageOptions.getRpc(); - - @Override - protected Object handleInvocation( - Object proxy, java.lang.reflect.Method method, Object[] args) - throws Throwable { - if ("writeWithResponse".equals(method.getName())) { - Object result = method.invoke(delegate, args); - boolean lastChunk = (boolean) args[5]; - // if we're on the lastChunk simulate a connection failure which - // happens after the request was processed but before response could - // be received by the client. - if (lastChunk) { - exceptionThrown.set(true); - throw StorageException.translate( - new IOException("simulated Connection closed prematurely")); - } else { - return result; - } - } - return method.invoke(delegate, args); - } - }); - } - }) - .build() - .getService(); - try (WriteChannel w = testStorage.writer(blobGen1, BlobWriteOption.generationMatch())) { - w.setChunkSize(chunkSize); - - w.write(contentGen2); - } - - assertTrue("Expected an exception to be thrown for the last chunk", exceptionThrown.get()); - - Blob blobGen2 = storage.get(blobId); - assertEquals(contentSize, (long) blobGen2.getSize()); - assertNotEquals(blobInfo.getGeneration(), blobGen2.getGeneration()); - ByteArrayOutputStream actualData = new ByteArrayOutputStream(); - blobGen2.downloadTo(actualData); - assertEquals(contentGen2Expected, ByteBuffer.wrap(actualData.toByteArray())); - } } diff --git a/google-cloud-storage/src/test/java/com/google/cloud/storage/it/TemporaryBucket.java b/google-cloud-storage/src/test/java/com/google/cloud/storage/it/TemporaryBucket.java index 0f6d255b3..54ccfd92d 100644 --- a/google-cloud-storage/src/test/java/com/google/cloud/storage/it/TemporaryBucket.java +++ b/google-cloud-storage/src/test/java/com/google/cloud/storage/it/TemporaryBucket.java @@ -25,7 +25,7 @@ import com.google.common.base.Preconditions; import java.time.Duration; -final class TemporaryBucket implements AutoCloseable { +public final class TemporaryBucket implements AutoCloseable { private final BucketInfo bucket; private final Storage storage; @@ -44,7 +44,7 @@ private TemporaryBucket( } /** Return the BucketInfo from the created temporary bucket. */ - BucketInfo getBucket() { + public BucketInfo getBucket() { return bucket; } @@ -55,11 +55,11 @@ public void close() throws Exception { } } - static Builder newBuilder() { + public static Builder newBuilder() { return new Builder(); } - static final class Builder { + public static final class Builder { private CleanupStrategy cleanupStrategy; private Duration cleanupTimeoutDuration; @@ -71,27 +71,27 @@ private Builder() { this.cleanupTimeoutDuration = Duration.ofMinutes(1); } - Builder setCleanupStrategy(CleanupStrategy cleanupStrategy) { + public Builder setCleanupStrategy(CleanupStrategy cleanupStrategy) { this.cleanupStrategy = cleanupStrategy; return this; } - Builder setCleanupTimeoutDuration(Duration cleanupTimeoutDuration) { + public Builder setCleanupTimeoutDuration(Duration cleanupTimeoutDuration) { this.cleanupTimeoutDuration = cleanupTimeoutDuration; return this; } - Builder setBucketInfo(BucketInfo bucketInfo) { + public Builder setBucketInfo(BucketInfo bucketInfo) { this.bucketInfo = bucketInfo; return this; } - Builder setStorage(Storage storage) { + public Builder setStorage(Storage storage) { this.storage = storage; return this; } - TemporaryBucket build() { + public TemporaryBucket build() { Preconditions.checkArgument( cleanupStrategy != CleanupStrategy.ONLY_ON_SUCCESS, "Unable to detect success."); Storage s = requireNonNull(storage, "storage must be non null"); diff --git a/google-cloud-storage/src/test/resources/com/google/cloud/storage/blobReadChannel.ser.properties b/google-cloud-storage/src/test/resources/com/google/cloud/storage/blobReadChannel.ser.properties new file mode 100644 index 000000000..c9d3dc5ff --- /dev/null +++ b/google-cloud-storage/src/test/resources/com/google/cloud/storage/blobReadChannel.ser.properties @@ -0,0 +1,70 @@ +# +# Copyright 2022 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. +# + +# Base 64 Encoded bytes of a BlobReadChannel circa v2.16.0 +# Generated using the following snippet: +# +# Storage s = StorageOptions.http() +# .setProjectId("proj") +# .setCredentials(NoCredentials.getInstance()) +# .build() +# .getService(); +# +# ReadChannel reader = s.reader(BlobId.of("buck", "obj", 1L)); +# RestorableState capture = reader.capture(); +# +# ByteArrayOutputStream baos = new ByteArrayOutputStream(); +# try (ObjectOutputStream oos = new ObjectOutputStream(baos)) { +# oos.writeObject(capture); +# } +# +# byte[] bytes = baos.toByteArray(); +# String b64Ser = Base64.getEncoder().encodeToString(bytes); +# +# System.out.println("b64Ser = " + b64Ser); +# +b64bytes=\ + rO0ABXNyADJjb20uZ29vZ2xlLmNsb3VkLnN0b3JhZ2UuQmxvYlJlYWRDaGFubmVsJFN0YXRlSW1wbGwJWjOFWbi1AgAJSQAJY2h1bmtTaXplWgALZW5kT2ZTdHJlYW1a\ + AAZpc09wZW5KAAVsaW1pdEoACHBvc2l0aW9uTAAEYmxvYnQAIUxjb20vZ29vZ2xlL2Nsb3VkL3N0b3JhZ2UvQmxvYklkO0wACGxhc3RFdGFndAASTGphdmEvbGFuZy9T\ + dHJpbmc7TAAOcmVxdWVzdE9wdGlvbnN0AA9MamF2YS91dGlsL01hcDtMAA5zZXJ2aWNlT3B0aW9uc3QALUxjb20vZ29vZ2xlL2Nsb3VkL3N0b3JhZ2UvSHR0cFN0b3Jh\ + Z2VPcHRpb25zO3hwACAAAAABf/////////8AAAAAAAAAAHNyAB9jb20uZ29vZ2xlLmNsb3VkLnN0b3JhZ2UuQmxvYklkcdHeVjWP2d0CAANMAAZidWNrZXRxAH4AAkwA\ + CmdlbmVyYXRpb250ABBMamF2YS9sYW5nL0xvbmc7TAAEbmFtZXEAfgACeHB0AARidWNrc3IADmphdmEubGFuZy5Mb25nO4vkkMyPI98CAAFKAAV2YWx1ZXhyABBqYXZh\ + LmxhbmcuTnVtYmVyhqyVHQuU4IsCAAB4cAAAAAAAAAABdAADb2JqcHNyADVjb20uZ29vZ2xlLmNvbW1vbi5jb2xsZWN0LkltbXV0YWJsZU1hcCRTZXJpYWxpemVkRm9y\ + bQAAAAAAAAAAAgACTAAEa2V5c3QAEkxqYXZhL2xhbmcvT2JqZWN0O0wABnZhbHVlc3EAfgAPeHB1cgATW0xqYXZhLmxhbmcuT2JqZWN0O5DOWJ8QcylsAgAAeHAAAAAA\ + dXEAfgARAAAAAHNyACtjb20uZ29vZ2xlLmNsb3VkLnN0b3JhZ2UuSHR0cFN0b3JhZ2VPcHRpb25ztmk+4Fw7cvMCAAFMABVyZXRyeUFsZ29yaXRobU1hbmFnZXJ0ADRM\ + Y29tL2dvb2dsZS9jbG91ZC9zdG9yYWdlL0h0dHBSZXRyeUFsZ29yaXRobU1hbmFnZXI7eHIAJ2NvbS5nb29nbGUuY2xvdWQuc3RvcmFnZS5TdG9yYWdlT3B0aW9uc5q/\ + 8jOW5d5PAgAAeHIAH2NvbS5nb29nbGUuY2xvdWQuU2VydmljZU9wdGlvbnN/qQsz9VFyfgIAC0wADmNsaWVudExpYlRva2VucQB+AAJMAAVjbG9ja3QAHkxjb20vZ29v\ + Z2xlL2FwaS9jb3JlL0FwaUNsb2NrO0wAC2NyZWRlbnRpYWxzdAAdTGNvbS9nb29nbGUvYXV0aC9DcmVkZW50aWFscztMAA5oZWFkZXJQcm92aWRlcnQAJ0xjb20vZ29v\ + Z2xlL2FwaS9nYXgvcnBjL0hlYWRlclByb3ZpZGVyO0wABGhvc3RxAH4AAkwACXByb2plY3RJZHEAfgACTAAOcXVvdGFQcm9qZWN0SWRxAH4AAkwADXJldHJ5U2V0dGlu\ + Z3N0ACtMY29tL2dvb2dsZS9hcGkvZ2F4L3JldHJ5aW5nL1JldHJ5U2V0dGluZ3M7TAAXc2VydmljZUZhY3RvcnlDbGFzc05hbWVxAH4AAkwAGnNlcnZpY2VScGNGYWN0\ + b3J5Q2xhc3NOYW1lcQB+AAJMABB0cmFuc3BvcnRPcHRpb25zdAAjTGNvbS9nb29nbGUvY2xvdWQvVHJhbnNwb3J0T3B0aW9uczt4cHQABGdjY2xzcgAmY29tLmdvb2ds\ + ZS5hcGkuY29yZS5DdXJyZW50TWlsbGlzQ2xvY2usd0sHJ9YTCwIAAHhwc3IAHmNvbS5nb29nbGUuY2xvdWQuTm9DcmVkZW50aWFsc6kR5wOeLAxAAgAAeHIAKGNvbS5n\ + b29nbGUuYXV0aC5vYXV0aDIuT0F1dGgyQ3JlZGVudGlhbHM/PX166aVRVwIABEwAEGV4cGlyYXRpb25NYXJnaW50ABRMamF2YS90aW1lL0R1cmF0aW9uO0wABGxvY2tx\ + AH4AD0wADXJlZnJlc2hNYXJnaW5xAH4AI0wABXZhbHVldAA1TGNvbS9nb29nbGUvYXV0aC9vYXV0aDIvT0F1dGgyQ3JlZGVudGlhbHMkT0F1dGhWYWx1ZTt4cgAbY29t\ + Lmdvb2dsZS5hdXRoLkNyZWRlbnRpYWxzCzii14w9kIECAAB4cHNyAA1qYXZhLnRpbWUuU2VylV2EuhsiSLIMAAB4cHcNAQAAAAAAAAEsAAAAAHh1cgACW0Ks8xf4BghU\ + 4AIAAHhwAAAAAHNxAH4AJ3cNAQAAAAAAAAFoAAAAAHhwc3IAJ2NvbS5nb29nbGUuYXBpLmdheC5ycGMuTm9IZWFkZXJQcm92aWRlcmWjEqhqxXthAgAAeHB0AB5odHRw\ + czovL3N0b3JhZ2UuZ29vZ2xlYXBpcy5jb210AARwcm9qcHNyADNjb20uZ29vZ2xlLmFwaS5nYXgucmV0cnlpbmcuQXV0b1ZhbHVlX1JldHJ5U2V0dGluZ3Nym/9/a0d0\ + swIACVoACGppdHRlcmVkSQALbWF4QXR0ZW1wdHNEABRyZXRyeURlbGF5TXVsdGlwbGllckQAFHJwY1RpbWVvdXRNdWx0aXBsaWVyTAARaW5pdGlhbFJldHJ5RGVsYXl0\ + ABpMb3JnL3RocmVldGVuL2JwL0R1cmF0aW9uO0wAEWluaXRpYWxScGNUaW1lb3V0cQB+ADFMAA1tYXhSZXRyeURlbGF5cQB+ADFMAA1tYXhScGNUaW1lb3V0cQB+ADFM\ + AAx0b3RhbFRpbWVvdXRxAH4AMXhyACljb20uZ29vZ2xlLmFwaS5nYXgucmV0cnlpbmcuUmV0cnlTZXR0aW5nc3Kb/39rR3SzAgAAeHABAAAABkAAAAAAAAAAP/AAAAAA\ + AABzcgATb3JnLnRocmVldGVuLmJwLlNlcpVdhLobIkiyDAAAeHB3DQEAAAAAAAAAAQAAAAB4c3EAfgA0dw0BAAAAAAAAADIAAAAAeHNxAH4ANHcNAQAAAAAAAAAgAAAA\ + AHhzcQB+ADR3DQEAAAAAAAAAMgAAAAB4c3EAfgA0dw0BAAAAAAAAADIAAAAAeHQAPmNvbS5nb29nbGUuY2xvdWQuc3RvcmFnZS5IdHRwU3RvcmFnZU9wdGlvbnMkSHR0\ + cFN0b3JhZ2VGYWN0b3J5dABBY29tLmdvb2dsZS5jbG91ZC5zdG9yYWdlLkh0dHBTdG9yYWdlT3B0aW9ucyRIdHRwU3RvcmFnZVJwY0ZhY3RvcnlzcgAqY29tLmdvb2ds\ + ZS5jbG91ZC5odHRwLkh0dHBUcmFuc3BvcnRPcHRpb25zbX9UTb2H/yICAANJAA5jb25uZWN0VGltZW91dEkAC3JlYWRUaW1lb3V0TAAdaHR0cFRyYW5zcG9ydEZhY3Rv\ + cnlDbGFzc05hbWVxAH4AAnhw//////////90AEZjb20uZ29vZ2xlLmNsb3VkLmh0dHAuSHR0cFRyYW5zcG9ydE9wdGlvbnMkRGVmYXVsdEh0dHBUcmFuc3BvcnRGYWN0\ + b3J5c3IAMmNvbS5nb29nbGUuY2xvdWQuc3RvcmFnZS5IdHRwUmV0cnlBbGdvcml0aG1NYW5hZ2Vy0i1ymVA0mEUCAAFMAA1yZXRyeVN0cmF0ZWd5dAAvTGNvbS9nb29n\ + bGUvY2xvdWQvc3RvcmFnZS9TdG9yYWdlUmV0cnlTdHJhdGVneTt4cHNyADRjb20uZ29vZ2xlLmNsb3VkLnN0b3JhZ2UuRGVmYXVsdFN0b3JhZ2VSZXRyeVN0cmF0ZWd5\ + bgaLnarjlYkCAAB4cA== diff --git a/google-cloud-storage/src/test/resources/com/google/cloud/storage/blobWriteChannel.ser.properties b/google-cloud-storage/src/test/resources/com/google/cloud/storage/blobWriteChannel.ser.properties index c9d3dc5ff..d60cb1749 100644 --- a/google-cloud-storage/src/test/resources/com/google/cloud/storage/blobWriteChannel.ser.properties +++ b/google-cloud-storage/src/test/resources/com/google/cloud/storage/blobWriteChannel.ser.properties @@ -1,5 +1,5 @@ # -# Copyright 2022 Google LLC +# Copyright 2023 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -23,8 +23,8 @@ # .build() # .getService(); # -# ReadChannel reader = s.reader(BlobId.of("buck", "obj", 1L)); -# RestorableState capture = reader.capture(); +# WriteChannel reader = s.writer(BlobInfo.newBuilder("buck", "obj").build(), BlobWriteOption.doesNotExist()); +# RestorableState capture = reader.capture(); # # ByteArrayOutputStream baos = new ByteArrayOutputStream(); # try (ObjectOutputStream oos = new ObjectOutputStream(baos)) { @@ -37,34 +37,44 @@ # System.out.println("b64Ser = " + b64Ser); # b64bytes=\ - rO0ABXNyADJjb20uZ29vZ2xlLmNsb3VkLnN0b3JhZ2UuQmxvYlJlYWRDaGFubmVsJFN0YXRlSW1wbGwJWjOFWbi1AgAJSQAJY2h1bmtTaXplWgALZW5kT2ZTdHJlYW1a\ - AAZpc09wZW5KAAVsaW1pdEoACHBvc2l0aW9uTAAEYmxvYnQAIUxjb20vZ29vZ2xlL2Nsb3VkL3N0b3JhZ2UvQmxvYklkO0wACGxhc3RFdGFndAASTGphdmEvbGFuZy9T\ - dHJpbmc7TAAOcmVxdWVzdE9wdGlvbnN0AA9MamF2YS91dGlsL01hcDtMAA5zZXJ2aWNlT3B0aW9uc3QALUxjb20vZ29vZ2xlL2Nsb3VkL3N0b3JhZ2UvSHR0cFN0b3Jh\ - Z2VPcHRpb25zO3hwACAAAAABf/////////8AAAAAAAAAAHNyAB9jb20uZ29vZ2xlLmNsb3VkLnN0b3JhZ2UuQmxvYklkcdHeVjWP2d0CAANMAAZidWNrZXRxAH4AAkwA\ - CmdlbmVyYXRpb250ABBMamF2YS9sYW5nL0xvbmc7TAAEbmFtZXEAfgACeHB0AARidWNrc3IADmphdmEubGFuZy5Mb25nO4vkkMyPI98CAAFKAAV2YWx1ZXhyABBqYXZh\ - LmxhbmcuTnVtYmVyhqyVHQuU4IsCAAB4cAAAAAAAAAABdAADb2JqcHNyADVjb20uZ29vZ2xlLmNvbW1vbi5jb2xsZWN0LkltbXV0YWJsZU1hcCRTZXJpYWxpemVkRm9y\ - bQAAAAAAAAAAAgACTAAEa2V5c3QAEkxqYXZhL2xhbmcvT2JqZWN0O0wABnZhbHVlc3EAfgAPeHB1cgATW0xqYXZhLmxhbmcuT2JqZWN0O5DOWJ8QcylsAgAAeHAAAAAA\ - dXEAfgARAAAAAHNyACtjb20uZ29vZ2xlLmNsb3VkLnN0b3JhZ2UuSHR0cFN0b3JhZ2VPcHRpb25ztmk+4Fw7cvMCAAFMABVyZXRyeUFsZ29yaXRobU1hbmFnZXJ0ADRM\ - Y29tL2dvb2dsZS9jbG91ZC9zdG9yYWdlL0h0dHBSZXRyeUFsZ29yaXRobU1hbmFnZXI7eHIAJ2NvbS5nb29nbGUuY2xvdWQuc3RvcmFnZS5TdG9yYWdlT3B0aW9uc5q/\ - 8jOW5d5PAgAAeHIAH2NvbS5nb29nbGUuY2xvdWQuU2VydmljZU9wdGlvbnN/qQsz9VFyfgIAC0wADmNsaWVudExpYlRva2VucQB+AAJMAAVjbG9ja3QAHkxjb20vZ29v\ - Z2xlL2FwaS9jb3JlL0FwaUNsb2NrO0wAC2NyZWRlbnRpYWxzdAAdTGNvbS9nb29nbGUvYXV0aC9DcmVkZW50aWFscztMAA5oZWFkZXJQcm92aWRlcnQAJ0xjb20vZ29v\ - Z2xlL2FwaS9nYXgvcnBjL0hlYWRlclByb3ZpZGVyO0wABGhvc3RxAH4AAkwACXByb2plY3RJZHEAfgACTAAOcXVvdGFQcm9qZWN0SWRxAH4AAkwADXJldHJ5U2V0dGlu\ - Z3N0ACtMY29tL2dvb2dsZS9hcGkvZ2F4L3JldHJ5aW5nL1JldHJ5U2V0dGluZ3M7TAAXc2VydmljZUZhY3RvcnlDbGFzc05hbWVxAH4AAkwAGnNlcnZpY2VScGNGYWN0\ - b3J5Q2xhc3NOYW1lcQB+AAJMABB0cmFuc3BvcnRPcHRpb25zdAAjTGNvbS9nb29nbGUvY2xvdWQvVHJhbnNwb3J0T3B0aW9uczt4cHQABGdjY2xzcgAmY29tLmdvb2ds\ - ZS5hcGkuY29yZS5DdXJyZW50TWlsbGlzQ2xvY2usd0sHJ9YTCwIAAHhwc3IAHmNvbS5nb29nbGUuY2xvdWQuTm9DcmVkZW50aWFsc6kR5wOeLAxAAgAAeHIAKGNvbS5n\ - b29nbGUuYXV0aC5vYXV0aDIuT0F1dGgyQ3JlZGVudGlhbHM/PX166aVRVwIABEwAEGV4cGlyYXRpb25NYXJnaW50ABRMamF2YS90aW1lL0R1cmF0aW9uO0wABGxvY2tx\ - AH4AD0wADXJlZnJlc2hNYXJnaW5xAH4AI0wABXZhbHVldAA1TGNvbS9nb29nbGUvYXV0aC9vYXV0aDIvT0F1dGgyQ3JlZGVudGlhbHMkT0F1dGhWYWx1ZTt4cgAbY29t\ - Lmdvb2dsZS5hdXRoLkNyZWRlbnRpYWxzCzii14w9kIECAAB4cHNyAA1qYXZhLnRpbWUuU2VylV2EuhsiSLIMAAB4cHcNAQAAAAAAAAEsAAAAAHh1cgACW0Ks8xf4BghU\ - 4AIAAHhwAAAAAHNxAH4AJ3cNAQAAAAAAAAFoAAAAAHhwc3IAJ2NvbS5nb29nbGUuYXBpLmdheC5ycGMuTm9IZWFkZXJQcm92aWRlcmWjEqhqxXthAgAAeHB0AB5odHRw\ - czovL3N0b3JhZ2UuZ29vZ2xlYXBpcy5jb210AARwcm9qcHNyADNjb20uZ29vZ2xlLmFwaS5nYXgucmV0cnlpbmcuQXV0b1ZhbHVlX1JldHJ5U2V0dGluZ3Nym/9/a0d0\ - swIACVoACGppdHRlcmVkSQALbWF4QXR0ZW1wdHNEABRyZXRyeURlbGF5TXVsdGlwbGllckQAFHJwY1RpbWVvdXRNdWx0aXBsaWVyTAARaW5pdGlhbFJldHJ5RGVsYXl0\ - ABpMb3JnL3RocmVldGVuL2JwL0R1cmF0aW9uO0wAEWluaXRpYWxScGNUaW1lb3V0cQB+ADFMAA1tYXhSZXRyeURlbGF5cQB+ADFMAA1tYXhScGNUaW1lb3V0cQB+ADFM\ - AAx0b3RhbFRpbWVvdXRxAH4AMXhyACljb20uZ29vZ2xlLmFwaS5nYXgucmV0cnlpbmcuUmV0cnlTZXR0aW5nc3Kb/39rR3SzAgAAeHABAAAABkAAAAAAAAAAP/AAAAAA\ - AABzcgATb3JnLnRocmVldGVuLmJwLlNlcpVdhLobIkiyDAAAeHB3DQEAAAAAAAAAAQAAAAB4c3EAfgA0dw0BAAAAAAAAADIAAAAAeHNxAH4ANHcNAQAAAAAAAAAgAAAA\ - AHhzcQB+ADR3DQEAAAAAAAAAMgAAAAB4c3EAfgA0dw0BAAAAAAAAADIAAAAAeHQAPmNvbS5nb29nbGUuY2xvdWQuc3RvcmFnZS5IdHRwU3RvcmFnZU9wdGlvbnMkSHR0\ - cFN0b3JhZ2VGYWN0b3J5dABBY29tLmdvb2dsZS5jbG91ZC5zdG9yYWdlLkh0dHBTdG9yYWdlT3B0aW9ucyRIdHRwU3RvcmFnZVJwY0ZhY3RvcnlzcgAqY29tLmdvb2ds\ - ZS5jbG91ZC5odHRwLkh0dHBUcmFuc3BvcnRPcHRpb25zbX9UTb2H/yICAANJAA5jb25uZWN0VGltZW91dEkAC3JlYWRUaW1lb3V0TAAdaHR0cFRyYW5zcG9ydEZhY3Rv\ - cnlDbGFzc05hbWVxAH4AAnhw//////////90AEZjb20uZ29vZ2xlLmNsb3VkLmh0dHAuSHR0cFRyYW5zcG9ydE9wdGlvbnMkRGVmYXVsdEh0dHBUcmFuc3BvcnRGYWN0\ - b3J5c3IAMmNvbS5nb29nbGUuY2xvdWQuc3RvcmFnZS5IdHRwUmV0cnlBbGdvcml0aG1NYW5hZ2Vy0i1ymVA0mEUCAAFMAA1yZXRyeVN0cmF0ZWd5dAAvTGNvbS9nb29n\ - bGUvY2xvdWQvc3RvcmFnZS9TdG9yYWdlUmV0cnlTdHJhdGVneTt4cHNyADRjb20uZ29vZ2xlLmNsb3VkLnN0b3JhZ2UuRGVmYXVsdFN0b3JhZ2VSZXRyeVN0cmF0ZWd5\ - bgaLnarjlYkCAAB4cA== + rO0ABXNyADNjb20uZ29vZ2xlLmNsb3VkLnN0b3JhZ2UuQmxvYldyaXRlQ2hhbm5lbCRTdGF0ZUltcGyjA3jVYuVZZQIAAUwAEWFsZ29yaXRobUZvcldyaXRldAAyTGNv\ + bS9nb29nbGUvYXBpL2dheC9yZXRyeWluZy9SZXN1bHRSZXRyeUFsZ29yaXRobTt4cgArY29tLmdvb2dsZS5jbG91ZC5CYXNlV3JpdGVDaGFubmVsJEJhc2VTdGF0ZXaH\ + 8w86CHBzAgAHSQAJY2h1bmtTaXplWgAGaXNPcGVuSgAIcG9zaXRpb25bAAZidWZmZXJ0AAJbQkwABmVudGl0eXQAFkxqYXZhL2lvL1NlcmlhbGl6YWJsZTtMAA5zZXJ2\ + aWNlT3B0aW9uc3QAIUxjb20vZ29vZ2xlL2Nsb3VkL1NlcnZpY2VPcHRpb25zO0wACHVwbG9hZElkdAASTGphdmEvbGFuZy9TdHJpbmc7eHAA8AAAAQAAAAAAAAAAdXIA\ + AltCrPMX+AYIVOACAAB4cAAAAABwc3IAK2NvbS5nb29nbGUuY2xvdWQuc3RvcmFnZS5IdHRwU3RvcmFnZU9wdGlvbnO2aT7gXDty8wIAAUwAFXJldHJ5QWxnb3JpdGht\ + TWFuYWdlcnQANExjb20vZ29vZ2xlL2Nsb3VkL3N0b3JhZ2UvSHR0cFJldHJ5QWxnb3JpdGhtTWFuYWdlcjt4cgAnY29tLmdvb2dsZS5jbG91ZC5zdG9yYWdlLlN0b3Jh\ + Z2VPcHRpb25zmr/yM5bl3k8CAAB4cgAfY29tLmdvb2dsZS5jbG91ZC5TZXJ2aWNlT3B0aW9uc3+pCzP1UXJ+AgALTAAOY2xpZW50TGliVG9rZW5xAH4ABkwABWNsb2Nr\ + dAAeTGNvbS9nb29nbGUvYXBpL2NvcmUvQXBpQ2xvY2s7TAALY3JlZGVudGlhbHN0AB1MY29tL2dvb2dsZS9hdXRoL0NyZWRlbnRpYWxzO0wADmhlYWRlclByb3ZpZGVy\ + dAAnTGNvbS9nb29nbGUvYXBpL2dheC9ycGMvSGVhZGVyUHJvdmlkZXI7TAAEaG9zdHEAfgAGTAAJcHJvamVjdElkcQB+AAZMAA5xdW90YVByb2plY3RJZHEAfgAGTAAN\ + cmV0cnlTZXR0aW5nc3QAK0xjb20vZ29vZ2xlL2FwaS9nYXgvcmV0cnlpbmcvUmV0cnlTZXR0aW5ncztMABdzZXJ2aWNlRmFjdG9yeUNsYXNzTmFtZXEAfgAGTAAac2Vy\ + dmljZVJwY0ZhY3RvcnlDbGFzc05hbWVxAH4ABkwAEHRyYW5zcG9ydE9wdGlvbnN0ACNMY29tL2dvb2dsZS9jbG91ZC9UcmFuc3BvcnRPcHRpb25zO3hwdAAEZ2NjbHNy\ + ACZjb20uZ29vZ2xlLmFwaS5jb3JlLkN1cnJlbnRNaWxsaXNDbG9ja6x3Swcn1hMLAgAAeHBzcgAeY29tLmdvb2dsZS5jbG91ZC5Ob0NyZWRlbnRpYWxzqRHnA54sDEAC\ + AAB4cgAoY29tLmdvb2dsZS5hdXRoLm9hdXRoMi5PQXV0aDJDcmVkZW50aWFscz89fXrppVFXAgAETAAQZXhwaXJhdGlvbk1hcmdpbnQAFExqYXZhL3RpbWUvRHVyYXRp\ + b247TAAEbG9ja3QAEkxqYXZhL2xhbmcvT2JqZWN0O0wADXJlZnJlc2hNYXJnaW5xAH4AGUwABXZhbHVldAA1TGNvbS9nb29nbGUvYXV0aC9vYXV0aDIvT0F1dGgyQ3Jl\ + ZGVudGlhbHMkT0F1dGhWYWx1ZTt4cgAbY29tLmdvb2dsZS5hdXRoLkNyZWRlbnRpYWxzCzii14w9kIECAAB4cHNyAA1qYXZhLnRpbWUuU2VylV2EuhsiSLIMAAB4cHcN\ + AQAAAAAAAAEsAAAAAHh1cQB+AAgAAAAAc3EAfgAedw0BAAAAAAAAAWgAAAAAeHBzcgAnY29tLmdvb2dsZS5hcGkuZ2F4LnJwYy5Ob0hlYWRlclByb3ZpZGVyZaMSqGrF\ + e2ECAAB4cHQAFWh0dHA6Ly9sb2NhbGhvc3Q6OTAwMHQABHByb2pwc3IAM2NvbS5nb29nbGUuYXBpLmdheC5yZXRyeWluZy5BdXRvVmFsdWVfUmV0cnlTZXR0aW5nc3Kb\ + /39rR3SzAgAJWgAIaml0dGVyZWRJAAttYXhBdHRlbXB0c0QAFHJldHJ5RGVsYXlNdWx0aXBsaWVyRAAUcnBjVGltZW91dE11bHRpcGxpZXJMABFpbml0aWFsUmV0cnlE\ + ZWxheXQAGkxvcmcvdGhyZWV0ZW4vYnAvRHVyYXRpb247TAARaW5pdGlhbFJwY1RpbWVvdXRxAH4AJ0wADW1heFJldHJ5RGVsYXlxAH4AJ0wADW1heFJwY1RpbWVvdXRx\ + AH4AJ0wADHRvdGFsVGltZW91dHEAfgAneHIAKWNvbS5nb29nbGUuYXBpLmdheC5yZXRyeWluZy5SZXRyeVNldHRpbmdzcpv/f2tHdLMCAAB4cAEAAAAGQAAAAAAAAAA/\ + 8AAAAAAAAHNyABNvcmcudGhyZWV0ZW4uYnAuU2VylV2EuhsiSLIMAAB4cHcNAQAAAAAAAAABAAAAAHhzcQB+ACp3DQEAAAAAAAAAMgAAAAB4c3EAfgAqdw0BAAAAAAAA\ + ACAAAAAAeHNxAH4AKncNAQAAAAAAAAAyAAAAAHhzcQB+ACp3DQEAAAAAAAAAMgAAAAB4dAA+Y29tLmdvb2dsZS5jbG91ZC5zdG9yYWdlLkh0dHBTdG9yYWdlT3B0aW9u\ + cyRIdHRwU3RvcmFnZUZhY3Rvcnl0AEFjb20uZ29vZ2xlLmNsb3VkLnN0b3JhZ2UuSHR0cFN0b3JhZ2VPcHRpb25zJEh0dHBTdG9yYWdlUnBjRmFjdG9yeXNyACpjb20u\ + Z29vZ2xlLmNsb3VkLmh0dHAuSHR0cFRyYW5zcG9ydE9wdGlvbnNtf1RNvYf/IgIAA0kADmNvbm5lY3RUaW1lb3V0SQALcmVhZFRpbWVvdXRMAB1odHRwVHJhbnNwb3J0\ + RmFjdG9yeUNsYXNzTmFtZXEAfgAGeHD//////////3QARmNvbS5nb29nbGUuY2xvdWQuaHR0cC5IdHRwVHJhbnNwb3J0T3B0aW9ucyREZWZhdWx0SHR0cFRyYW5zcG9y\ + dEZhY3RvcnlzcgAyY29tLmdvb2dsZS5jbG91ZC5zdG9yYWdlLkh0dHBSZXRyeUFsZ29yaXRobU1hbmFnZXLSLXKZUDSYRQIAAUwADXJldHJ5U3RyYXRlZ3l0AC9MY29t\ + L2dvb2dsZS9jbG91ZC9zdG9yYWdlL1N0b3JhZ2VSZXRyeVN0cmF0ZWd5O3hwc3IANGNvbS5nb29nbGUuY2xvdWQuc3RvcmFnZS5EZWZhdWx0U3RvcmFnZVJldHJ5U3Ry\ + YXRlZ3luBoudquOViQIAAHhwdACQaHR0cDovL2xvY2FsaG9zdDo5MDAwL3VwbG9hZC9zdG9yYWdlL3YxL2IvYnVjay9vP3VwbG9hZFR5cGU9cmVzdW1hYmxlJnVwbG9h\ + ZF9pZD0xNzcyNzI1NDM5ZDEyZWUzNjNmZmRlNmNiZmNlYjEzMGYzZTIxMWJiM2NjMzBlNjFhNGQ2N2I2MTU0OTUxMjIxc3IAIWNvbS5nb29nbGUuY2xvdWQuRXhjZXB0\ + aW9uSGFuZGxlct3Z0AGsJj+JAgAETAAMaW50ZXJjZXB0b3JzdAApTGNvbS9nb29nbGUvY29tbW9uL2NvbGxlY3QvSW1tdXRhYmxlTGlzdDtMABZub25SZXRyaWFibGVF\ + eGNlcHRpb25zdAAoTGNvbS9nb29nbGUvY29tbW9uL2NvbGxlY3QvSW1tdXRhYmxlU2V0O0wAE3JldHJpYWJsZUV4Y2VwdGlvbnNxAH4APUwACXJldHJ5SW5mb3QAD0xq\ + YXZhL3V0aWwvU2V0O3hwc3IANmNvbS5nb29nbGUuY29tbW9uLmNvbGxlY3QuSW1tdXRhYmxlTGlzdCRTZXJpYWxpemVkRm9ybQAAAAAAAAAAAgABWwAIZWxlbWVudHN0\ + ABNbTGphdmEvbGFuZy9PYmplY3Q7eHB1cgATW0xqYXZhLmxhbmcuT2JqZWN0O5DOWJ8QcylsAgAAeHAAAAACc3IAWWNvbS5nb29nbGUuY2xvdWQuc3RvcmFnZS5EZWZh\ + dWx0U3RvcmFnZVJldHJ5U3RyYXRlZ3kkRW1wdHlKc29uUGFyc2luZ0V4Y2VwdGlvbkludGVyY2VwdG9yz+LSc1EB+RsCAAB4cHNyAERjb20uZ29vZ2xlLmNsb3VkLnN0\ + b3JhZ2UuRGVmYXVsdFN0b3JhZ2VSZXRyeVN0cmF0ZWd5JEludGVyY2VwdG9ySW1wbElTPf0EVOdoAgACWgAKaWRlbXBvdGVudEwAD3JldHJ5YWJsZUVycm9yc3EAfgA9\ + eHABc3IANWNvbS5nb29nbGUuY29tbW9uLmNvbGxlY3QuSW1tdXRhYmxlU2V0JFNlcmlhbGl6ZWRGb3JtAAAAAAAAAAACAAFbAAhlbGVtZW50c3EAfgBBeHB1cQB+AEMA\ + AAAIc3IAK2NvbS5nb29nbGUuY2xvdWQuQmFzZVNlcnZpY2VFeGNlcHRpb24kRXJyb3LIN4LqhDNMpwIAA1oACHJlamVjdGVkTAAEY29kZXQAE0xqYXZhL2xhbmcvSW50\ + ZWdlcjtMAAZyZWFzb25xAH4ABnhwAHNyABFqYXZhLmxhbmcuSW50ZWdlchLioKT3gYc4AgABSQAFdmFsdWV4cgAQamF2YS5sYW5nLk51bWJlcoaslR0LlOCLAgAAeHAA\ + AAH4cHNxAH4ATABzcQB+AE8AAAH3cHNxAH4ATABzcQB+AE8AAAH2cHNxAH4ATABzcQB+AE8AAAH0cHNxAH4ATABzcQB+AE8AAAGtcHNxAH4ATABzcQB+AE8AAAGYcHNx\ + AH4ATABwdAANaW50ZXJuYWxFcnJvcnNxAH4ATABwdAAbY29ubmVjdGlvbkNsb3NlZFByZW1hdHVyZWx5c3EAfgBJdXEAfgBDAAAAAHEAfgBgc3IAEWphdmEudXRpbC5I\ + YXNoU2V0ukSFlZa4tzQDAAB4cHcMAAAAED9AAAAAAAAAeA==