diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 248e17f02c5a8..3520e12c0c05d 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -108,7 +108,7 @@
3.0.5
3.6.1
- 27.0-jre
+ 33.1.0-jre
5.1.0
1.78.1
@@ -2142,6 +2142,11 @@
failsafe
2.4.4
+
+ com.google.cloud
+ google-cloud-storage
+ 2.52.0
+
diff --git a/hadoop-tools/hadoop-gcp/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-gcp/dev-support/findbugs-exclude.xml
new file mode 100644
index 0000000000000..80be329bd6d16
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/dev-support/findbugs-exclude.xml
@@ -0,0 +1,29 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/hadoop-tools/hadoop-gcp/pom.xml b/hadoop-tools/hadoop-gcp/pom.xml
new file mode 100644
index 0000000000000..2da2881ab7935
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/pom.xml
@@ -0,0 +1,514 @@
+
+
+
+ 4.0.0
+
+ org.apache.hadoop
+ hadoop-project
+ 3.5.0-SNAPSHOT
+ ../../hadoop-project
+
+ hadoop-gcp
+ 3.5.0-SNAPSHOT
+ Apache Hadoop Google Cloud Platform support
+
+ This module contains code to support integration with Google Cloud Platform.
+ It also declares the dependencies needed to work with Google Cloud Storage.
+
+ jar
+
+
+ UTF-8
+ true
+ ${project.build.directory}/test
+
+
+
+
+ tests-off
+
+
+ src/test/resources/auth-keys.xml
+
+
+
+ true
+
+
+
+ tests-on
+
+
+ src/test/resources/auth-keys.xml
+
+
+
+ false
+
+
+
+ parallel-tests
+
+
+ parallel-tests
+
+
+
+
+
+ org.apache.hadoop
+ hadoop-maven-plugins
+
+
+ parallel-tests-createdir
+
+ parallel-tests-createdir
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-surefire-plugin
+
+ ${testsThreadCount}
+ false
+ false
+ ${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true
+
+ ${testsThreadCount}
+ ${test.build.data}/${surefire.forkNumber}
+ ${test.build.dir}/${surefire.forkNumber}
+ ${hadoop.tmp.dir}/${surefire.forkNumber}
+ job-${job.id}-fork-000${surefire.forkNumber}
+
+
+
+
+ org.apache.maven.plugins
+ maven-failsafe-plugin
+
+
+ default-integration-test
+
+ integration-test
+ verify
+
+
+ ${testsThreadCount}
+ false
+ ${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true
+ ${fs.gs.scale.test.timeout}
+ false
+
+
+ true
+ ${test.build.data}/${surefire.forkNumber}
+ ${test.build.dir}/${surefire.forkNumber}
+ ${hadoop.tmp.dir}/${surefire.forkNumber}
+
+
+
+
+
+ job-${job.id}-fork-000${surefire.forkNumber}
+ ${test.integration.timeout}
+
+
+ **/ITest*.java
+
+
+
+
+ sequential-integration-tests
+
+ integration-test
+ verify
+
+
+ ${fs.gs.scale.test.timeout}
+ false
+
+
+ false
+ job-${job.id}
+
+
+
+
+ **/ITest*.java
+
+
+
+
+
+
+
+
+
+ sequential-tests
+
+
+ !parallel-tests
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-failsafe-plugin
+
+
+
+ integration-test
+ verify
+
+
+ false
+
+ job-${job.id}
+
+
+
+
+
+
+
+
+
+
+
+
+
+ maven-shade-plugin
+
+
+ package
+
+ shade
+
+
+
+
+
+
+
+
+
+
+ com.google.auth:*
+
+ **
+
+
+
+ io.grpc:*
+
+ **
+
+
+
+ io.opencensus:*
+
+ **
+
+
+
+ *:*
+
+ *.json
+ google/**
+ grpc/**
+
+
+
+
+
+ com.google.api
+ com.google.api-client
+ com.google.api.grpc
+ com.google.apis
+ com.google.auth
+ com.google.cloud
+ com.google.cloud.bigdataoss
+ com.google.cloud.grpc
+ com.google.cloud.http
+ com.google.flogger
+ com.google.code.gson
+ com.google.guava
+ com.google.http-client
+ com.google.oauth-client
+ com.google.protobuf
+ com.google.re2j
+ com.google.storage.v2
+ com.lmax
+ io.grpc
+ io.opencensus
+ io.opentelemetry
+ io.opentelemetry.api
+ io.opentelemetry.contrib
+ io.opentelemetry.semconv
+ io.perfmark
+ org.apache.httpcomponents
+ org.threeten:threetenbp
+
+
+ true
+
+
+ com
+ com.google.cloud.hadoop.repackaged.ossgcs.com
+
+ com.google.api.**
+ com.google.api.gax.**
+ com.google.auth.**
+ com.google.cloud.*
+ com.google.cloud.audit.**
+ com.google.cloud.grpc.**
+ com.google.cloud.hadoop.gcsio.**
+ com.google.cloud.hadoop.util.**
+ com.google.cloud.http.**
+ com.google.cloud.monitoring.**
+ com.google.cloud.opentelemetry.**
+ com.google.cloud.spi.**
+ com.google.cloud.storage.**
+ com.google.common.**
+ com.google.geo.**
+ com.google.gson.**
+ com.google.google.storage.**
+ com.google.iam.**
+ com.google.logging.**
+ com.google.longrunning.**
+ com.google.monitoring.**
+ com.google.protobuf.**
+ com.google.re2j.**
+ com.google.rpc.**
+ com.google.storage.**
+ com.google.thirdparty.**
+ com.google.type.**
+ com.lmax.disruptor.**
+
+
+ com.google.cloud.hadoop.util.AccessTokenProvider
+ com.google.cloud.hadoop.util.AccessTokenProvider$AccessToken
+ com.google.cloud.hadoop.util.AccessTokenProvider$AccessTokenType
+ com.google.cloud.hadoop.util.AccessBoundary
+ com.google.cloud.hadoop.util.AccessBoundary$Action
+ com.google.cloud.hadoop.util.AutoValue_AccessBoundary
+
+
+
+ org
+ com.google.cloud.hadoop.repackaged.ossgcs.org
+
+ org.apache.http.**
+ org.threeten.**
+
+
+
+
+ io.grpc.netty.shaded
+
+ com.google.cloud.hadoop.repackaged.ossgcs.io.grpc.netty.shaded
+
+
+
+ io
+ com.google.cloud.hadoop.repackaged.ossgcs.io
+
+ io.grpc.**
+ io.opencensus.**
+ io.perfmark.**
+
+
+
+ META-INF/native/io_grpc_netty_shaded_
+
+ META-INF/native/com_google_cloud_hadoop_repackaged_gcs_io_grpc_netty_shaded_
+
+
+
+ META-INF/native/libio_grpc_netty_shaded_
+
+ META-INF/native/libcom_google_cloud_hadoop_repackaged_gcs_io_grpc_netty_shaded_
+
+
+
+ true
+
+
+
+
+
+
+ com.github.spotbugs
+ spotbugs-maven-plugin
+
+ true
+ ${basedir}/dev-support/findbugs-exclude.xml
+
+ Max
+
+
+
+ org.apache.maven.plugins
+ maven-checkstyle-plugin
+
+ src/config/checkstyle-suppressions.xml
+
+
+
+ org.apache.maven.plugins
+ maven-surefire-plugin
+
+ 3600
+
+ ${test.integration.timeout}
+
+
+
+
+ org.apache.maven.plugins
+ maven-enforcer-plugin
+
+
+ banned-illegal-imports
+ process-sources
+
+ enforce
+
+
+
+
+ false
+ Restrict mapreduce imports to committer code
+
+
+
+
+ org.apache.hadoop.mapreduce.**
+ org.apache.hadoop.mapred.**
+
+
+
+ false
+ Restrict encryption client imports to encryption client factory
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ com.google.protobuf
+ protobuf-java
+ 3.25.5
+
+
+
+
+
+
+ org.apache.hadoop
+ hadoop-common
+ provided
+
+
+ javax.servlet
+ servlet-api
+
+
+ javax.enterprise
+ cdi-api
+
+
+
+ com.google.protobuf
+ protobuf-java
+
+
+
+
+ org.apache.hadoop
+ hadoop-common
+ test
+ test-jar
+
+
+ org.assertj
+ assertj-core
+ test
+
+
+ junit
+ junit
+ test
+
+
+ org.junit.platform
+ junit-platform-launcher
+ test
+
+
+ org.junit.vintage
+ junit-vintage-engine
+ test
+
+
+ com.google.cloud
+ google-cloud-storage
+
+
+ org.junit.jupiter
+ junit-jupiter-api
+ test
+
+
+ org.junit.jupiter
+ junit-jupiter-engine
+ test
+
+
+ org.junit.jupiter
+ junit-jupiter-params
+ test
+
+
+
+
diff --git a/hadoop-tools/hadoop-gcp/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-gcp/src/config/checkstyle-suppressions.xml
new file mode 100644
index 0000000000000..8c765bc97f92a
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/config/checkstyle-suppressions.xml
@@ -0,0 +1,26 @@
+
+
+
+
+
+
+
+
+
+
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/ApiErrorExtractor.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/ApiErrorExtractor.java
new file mode 100644
index 0000000000000..4fef41b1971ee
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/ApiErrorExtractor.java
@@ -0,0 +1,327 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import com.google.api.client.googleapis.json.GoogleJsonError;
+import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.client.http.HttpResponseException;
+import com.google.api.client.http.HttpStatusCodes;
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Iterables;
+import java.io.IOException;
+import java.util.List;
+import javax.annotation.Nullable;
+
+/**
+ * Translates exceptions from API calls into higher-level meaning, while allowing injectability for
+ * testing how API errors are handled.
+ */
+class ApiErrorExtractor {
+
+ /** Singleton instance of the ApiErrorExtractor. */
+ public static final ApiErrorExtractor INSTANCE = new ApiErrorExtractor();
+
+ public static final int STATUS_CODE_RANGE_NOT_SATISFIABLE = 416;
+
+ public static final String GLOBAL_DOMAIN = "global";
+ public static final String USAGE_LIMITS_DOMAIN = "usageLimits";
+
+ public static final String RATE_LIMITED_REASON = "rateLimitExceeded";
+ public static final String USER_RATE_LIMITED_REASON = "userRateLimitExceeded";
+
+ public static final String QUOTA_EXCEEDED_REASON = "quotaExceeded";
+
+ // These come with "The account for ... has been disabled" message.
+ public static final String ACCOUNT_DISABLED_REASON = "accountDisabled";
+
+ // These come with "Project marked for deletion" message.
+ public static final String ACCESS_NOT_CONFIGURED_REASON = "accessNotConfigured";
+
+ // These are 400 error codes with "resource 'xyz' is not ready" message.
+ // These sometimes happens when create operation is still in-flight but resource
+ // representation is already available via get call.
+ public static final String RESOURCE_NOT_READY_REASON = "resourceNotReady";
+
+ // HTTP 413 with message "Value for field 'foo' is too large".
+ public static final String FIELD_SIZE_TOO_LARGE_REASON = "fieldSizeTooLarge";
+
+ // HTTP 400 message for 'USER_PROJECT_MISSING' error.
+ public static final String USER_PROJECT_MISSING_MESSAGE =
+ "Bucket is a requester pays bucket but no user project provided.";
+
+ // The debugInfo field present on Errors collection in GoogleJsonException
+ // as an unknown key.
+ private static final String DEBUG_INFO_FIELD = "debugInfo";
+
+ /**
+ * Determines if the given exception indicates intermittent request failure or failure caused by
+ * user error.
+ */
+ public boolean requestFailure(IOException e) {
+ HttpResponseException httpException = getHttpResponseException(e);
+ return httpException != null
+ && (accessDenied(httpException)
+ || badRequest(httpException)
+ || internalServerError(httpException)
+ || rateLimited(httpException)
+ || IoExceptionHelper.isSocketError(httpException)
+ || unauthorized(httpException));
+ }
+
+ /**
+ * Determines if the given exception indicates 'access denied'. Recursively checks getCause() if
+ * outer exception isn't an instance of the correct class.
+ *
+ *
Warning: this method only checks for access denied status code, however this may include
+ * potentially recoverable reason codes such as rate limiting. For alternative, see {@link
+ * #accessDeniedNonRecoverable(IOException)}.
+ */
+ public boolean accessDenied(IOException e) {
+ return recursiveCheckForCode(e, HttpStatusCodes.STATUS_CODE_FORBIDDEN);
+ }
+
+ /** Determines if the given exception indicates bad request. */
+ public boolean badRequest(IOException e) {
+ return recursiveCheckForCode(e, HttpStatusCodes.STATUS_CODE_BAD_REQUEST);
+ }
+
+ /**
+ * Determines if the given exception indicates the request was unauthenticated. This can be caused
+ * by attaching invalid credentials to a request.
+ */
+ public boolean unauthorized(IOException e) {
+ return recursiveCheckForCode(e, HttpStatusCodes.STATUS_CODE_UNAUTHORIZED);
+ }
+
+ /**
+ * Determines if the exception is a non-recoverable access denied code (such as account closed or
+ * marked for deletion).
+ */
+ public boolean accessDeniedNonRecoverable(IOException e) {
+ ErrorInfo errorInfo = getErrorInfo(e);
+ String reason = errorInfo != null ? errorInfo.getReason() : null;
+ return ACCOUNT_DISABLED_REASON.equals(reason) || ACCESS_NOT_CONFIGURED_REASON.equals(reason);
+ }
+
+ /** Determines if the exception is a client error. */
+ public boolean clientError(IOException e) {
+ HttpResponseException httpException = getHttpResponseException(e);
+ return httpException != null && getHttpStatusCode(httpException) / 100 == 4;
+ }
+
+ /** Determines if the exception is an internal server error. */
+ public boolean internalServerError(IOException e) {
+ HttpResponseException httpException = getHttpResponseException(e);
+ return httpException != null && getHttpStatusCode(httpException) / 100 == 5;
+ }
+
+ /**
+ * Determines if the given exception indicates 'item already exists'. Recursively checks
+ * getCause() if outer exception isn't an instance of the correct class.
+ */
+ public boolean itemAlreadyExists(IOException e) {
+ return recursiveCheckForCode(e, HttpStatusCodes.STATUS_CODE_CONFLICT);
+ }
+
+ /**
+ * Determines if the given exception indicates 'item not found'. Recursively checks getCause() if
+ * outer exception isn't an instance of the correct class.
+ */
+ public boolean itemNotFound(IOException e) {
+ return recursiveCheckForCode(e, HttpStatusCodes.STATUS_CODE_NOT_FOUND);
+ }
+
+ /**
+ * Determines if the given exception indicates 'field size too large'. Recursively checks
+ * getCause() if outer exception isn't an instance of the correct class.
+ */
+ public boolean fieldSizeTooLarge(IOException e) {
+ ErrorInfo errorInfo = getErrorInfo(e);
+ return errorInfo != null && FIELD_SIZE_TOO_LARGE_REASON.equals(errorInfo.getReason());
+ }
+
+ /**
+ * Determines if the given exception indicates 'resource not ready'. Recursively checks getCause()
+ * if outer exception isn't an instance of the correct class.
+ */
+ public boolean resourceNotReady(IOException e) {
+ ErrorInfo errorInfo = getErrorInfo(e);
+ return errorInfo != null && RESOURCE_NOT_READY_REASON.equals(errorInfo.getReason());
+ }
+
+ /**
+ * Determines if the given IOException indicates 'precondition not met' Recursively checks
+ * getCause() if outer exception isn't an instance of the correct class.
+ */
+ public boolean preconditionNotMet(IOException e) {
+ return recursiveCheckForCode(e, HttpStatusCodes.STATUS_CODE_PRECONDITION_FAILED);
+ }
+
+ /**
+ * Determines if the given exception indicates 'range not satisfiable'. Recursively checks
+ * getCause() if outer exception isn't an instance of the correct class.
+ */
+ public boolean rangeNotSatisfiable(IOException e) {
+ return recursiveCheckForCode(e, STATUS_CODE_RANGE_NOT_SATISFIABLE);
+ }
+
+ /**
+ * Determines if a given Throwable is caused by a rate limit being applied. Recursively checks
+ * getCause() if outer exception isn't an instance of the correct class.
+ *
+ * @param e The Throwable to check.
+ * @return True if the Throwable is a result of rate limiting being applied.
+ */
+ public boolean rateLimited(IOException e) {
+ ErrorInfo errorInfo = getErrorInfo(e);
+ if (errorInfo != null) {
+ String domain = errorInfo.getDomain();
+ boolean isRateLimitedOrGlobalDomain =
+ USAGE_LIMITS_DOMAIN.equals(domain) || GLOBAL_DOMAIN.equals(domain);
+ String reason = errorInfo.getReason();
+ boolean isRateLimitedReason =
+ RATE_LIMITED_REASON.equals(reason) || USER_RATE_LIMITED_REASON.equals(reason);
+ return isRateLimitedOrGlobalDomain && isRateLimitedReason;
+ }
+ return false;
+ }
+
+ /**
+ * Determines if a given Throwable is caused by Quota Exceeded. Recursively checks getCause() if
+ * outer exception isn't an instance of the correct class.
+ */
+ public boolean quotaExceeded(IOException e) {
+ ErrorInfo errorInfo = getErrorInfo(e);
+ return errorInfo != null && QUOTA_EXCEEDED_REASON.equals(errorInfo.getReason());
+ }
+
+ /**
+ * Determines if the given exception indicates that 'userProject' is missing in request.
+ * Recursively checks getCause() if outer exception isn't an instance of the correct class.
+ */
+ public boolean userProjectMissing(IOException e) {
+ GoogleJsonError jsonError = getJsonError(e);
+ return jsonError != null
+ && jsonError.getCode() == HttpStatusCodes.STATUS_CODE_BAD_REQUEST
+ && USER_PROJECT_MISSING_MESSAGE.equals(jsonError.getMessage());
+ }
+
+ /** Extracts the error message. */
+ public String getErrorMessage(IOException e) {
+ // Prefer to use message from GJRE.
+ GoogleJsonError jsonError = getJsonError(e);
+ return jsonError == null ? e.getMessage() : jsonError.getMessage();
+ }
+
+ /**
+ * Converts the exception to a user-presentable error message. Specifically, extracts message
+ * field for HTTP 4xx codes, and creates a generic "Internal Server Error" for HTTP 5xx codes.
+ *
+ * @param e the exception
+ * @param action the description of the action being performed at the time of error.
+ * @see #toUserPresentableMessage(IOException, String)
+ */
+ public IOException toUserPresentableException(IOException e, String action) throws IOException {
+ throw new IOException(toUserPresentableMessage(e, action), e);
+ }
+
+ /**
+ * Converts the exception to a user-presentable error message. Specifically, extracts message
+ * field for HTTP 4xx codes, and creates a generic "Internal Server Error" for HTTP 5xx codes.
+ */
+ public String toUserPresentableMessage(IOException e, @Nullable String action) {
+ String message = "Internal server error";
+ if (clientError(e)) {
+ message = getErrorMessage(e);
+ }
+ return action == null
+ ? message
+ : String.format("Encountered an error while %s: %s", action, message);
+ }
+
+ /** See {@link #toUserPresentableMessage(IOException, String)}. */
+ public String toUserPresentableMessage(IOException e) {
+ return toUserPresentableMessage(e, null);
+ }
+
+ @Nullable
+ public String getDebugInfo(IOException e) {
+ ErrorInfo errorInfo = getErrorInfo(e);
+ return errorInfo != null ? (String) errorInfo.getUnknownKeys().get(DEBUG_INFO_FIELD) : null;
+ }
+
+ /**
+ * Returns HTTP status code from the given exception.
+ *
+ *
Note: GoogleJsonResponseException.getStatusCode() method is marked final therefore it cannot
+ * be mocked using Mockito. We use this helper so that we can override it in tests.
+ */
+ protected int getHttpStatusCode(HttpResponseException e) {
+ return e.getStatusCode();
+ }
+
+ /**
+ * Get the first ErrorInfo from an IOException if it is an instance of
+ * GoogleJsonResponseException, otherwise return null.
+ */
+ @Nullable
+ protected ErrorInfo getErrorInfo(IOException e) {
+ GoogleJsonError jsonError = getJsonError(e);
+ List errors = jsonError != null ? jsonError.getErrors() : ImmutableList.of();
+ return errors != null ? Iterables.getFirst(errors, null) : null;
+ }
+
+ /** If the exception is a GoogleJsonResponseException, get the error details, else return null. */
+ @Nullable
+ protected GoogleJsonError getJsonError(IOException e) {
+ GoogleJsonResponseException jsonException = getJsonResponseException(e);
+ return jsonException == null ? null : jsonException.getDetails();
+ }
+
+ /** Recursively checks getCause() if outer exception isn't an instance of the correct class. */
+ protected boolean recursiveCheckForCode(IOException e, int code) {
+ HttpResponseException httpException = getHttpResponseException(e);
+ return httpException != null && getHttpStatusCode(httpException) == code;
+ }
+
+ @Nullable
+ public static GoogleJsonResponseException getJsonResponseException(Throwable throwable) {
+ Throwable cause = throwable;
+ while (cause != null) {
+ if (cause instanceof GoogleJsonResponseException) {
+ return (GoogleJsonResponseException) cause;
+ }
+ cause = cause.getCause();
+ }
+ return null;
+ }
+
+ @Nullable
+ public static HttpResponseException getHttpResponseException(Throwable throwable) {
+ Throwable cause = throwable;
+ while (cause != null) {
+ if (cause instanceof HttpResponseException) {
+ return (HttpResponseException) cause;
+ }
+ cause = cause.getCause();
+ }
+ return null;
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/Constants.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/Constants.java
new file mode 100644
index 0000000000000..34434b2859a06
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/Constants.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+final class Constants {
+ private Constants() {}
+
+ // URI scheme for GCS.
+ static final String SCHEME = "gs";
+ static final String PATH_DELIMITER = "/";
+
+ static final String GCS_CONFIG_PREFIX = "fs.gs";
+
+ static final String BASE_KEY_PREFIX = "google.cloud";
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/CreateBucketOptions.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/CreateBucketOptions.java
new file mode 100644
index 0000000000000..46cd2a7efbd34
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/CreateBucketOptions.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import java.time.Duration;
+
+final class CreateBucketOptions {
+ // TODO: Make sure the defaults have the setting matching the existing connector.
+ static final CreateBucketOptions DEFAULT = new Builder().build();
+ private final String location;
+ private final String storageClass;
+ private final Duration ttl;
+ private final String projectId;
+
+ private CreateBucketOptions(Builder builder) {
+ this.location = builder.location;
+ this.storageClass = builder.storageClass;
+ this.ttl = builder.ttl;
+ this.projectId = builder.projectId;
+ }
+
+ public String getLocation() {
+ return location;
+ }
+
+ public String getStorageClass() {
+ return storageClass;
+ }
+
+ public Duration getTtl() { // Changed return type to Duration
+ return ttl;
+ }
+
+ static class Builder {
+ private String location;
+ private String storageClass;
+ private Duration ttl;
+ private String projectId;
+
+ public Builder withLocation(String loc) {
+ this.location = loc;
+ return this;
+ }
+
+ public Builder withStorageClass(String sc) {
+ this.storageClass = sc;
+ return this;
+ }
+
+ public Builder withTtl(Duration ttlDuration) {
+ this.ttl = ttlDuration;
+ return this;
+ }
+
+ public Builder withProjectId(String pid) {
+ this.projectId = pid;
+ return this;
+ }
+
+ public CreateBucketOptions build() {
+ return new CreateBucketOptions(this);
+ }
+ }
+}
+
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/CreateObjectOptions.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/CreateObjectOptions.java
new file mode 100644
index 0000000000000..26c91fcae7bd2
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/CreateObjectOptions.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/** Options that can be specified when creating a file in the {@link GoogleCloudStorage}. */
+
+final class CreateObjectOptions {
+ static final CreateObjectOptions DEFAULT_OVERWRITE = builder().setOverwriteExisting(true).build();
+
+ private final String contentEncoding;
+ private final String contentType;
+ private final boolean ensureEmptyObjectsMetadataMatch;
+ private final String kmsKeyName;
+ private final ImmutableMap metadata;
+ private final boolean overwriteExisting;
+
+ private CreateObjectOptions(Builder builder) {
+ this.contentEncoding = builder.contentEncoding;
+ this.contentType = builder.contentType;
+ this.ensureEmptyObjectsMetadataMatch = builder.ensureEmptyObjectsMetadataMatch;
+ this.kmsKeyName = builder.kmsKeyName;
+ this.metadata = ImmutableMap.copyOf(builder.metadata);
+ this.overwriteExisting = builder.overwriteExisting;
+ }
+
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ public String getContentEncoding() {
+ return contentEncoding;
+ }
+
+ public String getContentType() {
+ return contentType;
+ }
+
+ public boolean isEnsureEmptyObjectsMetadataMatch() {
+ return ensureEmptyObjectsMetadataMatch;
+ }
+
+ public String getKmsKeyName() {
+ return kmsKeyName;
+ }
+
+ public Map getMetadata() {
+ return metadata;
+ }
+
+ public boolean isOverwriteExisting() {
+ return overwriteExisting;
+ }
+
+ public Builder toBuilder() {
+ return builder().setContentEncoding(this.contentEncoding).setContentType(this.contentType)
+ .setEnsureEmptyObjectsMetadataMatch(this.ensureEmptyObjectsMetadataMatch)
+ .setKmsKeyName(this.kmsKeyName).setMetadata(this.metadata)
+ .setOverwriteExisting(this.overwriteExisting);
+ }
+
+ static final class Builder {
+ private String contentEncoding;
+ private String contentType;
+ private boolean ensureEmptyObjectsMetadataMatch = false;
+ private String kmsKeyName;
+ private Map metadata = new HashMap<>();
+ private boolean overwriteExisting = false;
+
+ private Builder() {
+ }
+
+ public Builder setContentEncoding(String ce) {
+ this.contentEncoding = ce;
+ return this;
+ }
+
+ public Builder setContentType(String ct) {
+ this.contentType = ct;
+ return this;
+ }
+
+ public Builder setEnsureEmptyObjectsMetadataMatch(boolean val) {
+ this.ensureEmptyObjectsMetadataMatch = val;
+ return this;
+ }
+
+ public Builder setKmsKeyName(String key) {
+ this.kmsKeyName = key;
+ return this;
+ }
+
+ public Builder setMetadata(Map m) {
+ this.metadata = m;
+ return this;
+ }
+
+ public Builder setOverwriteExisting(boolean overwrite) {
+ this.overwriteExisting = overwrite;
+ return this;
+ }
+
+ public CreateObjectOptions build() {
+ return new CreateObjectOptions(this);
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/CreateOptions.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/CreateOptions.java
new file mode 100644
index 0000000000000..c9b44a1a481b1
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/CreateOptions.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
+
+import java.util.Map;
+import javax.annotation.Nullable;
+
+/**
+ * Options that can be specified when creating a file in the {@link GoogleCloudStorageFileSystem}.
+ */
+final class CreateOptions {
+ private final ImmutableMap attributes;
+ private final String contentType;
+ private final long overwriteGenerationId;
+ private final WriteMode mode;
+
+ private CreateOptions(CreateOperationOptionsBuilder builder) {
+ this.attributes = ImmutableMap.copyOf(builder.attributes);
+ this.contentType = builder.contentType;
+ this.overwriteGenerationId = builder.overwriteGenerationId;
+ this.mode = builder.writeMode;
+ }
+
+ boolean isOverwriteExisting() {
+ return this.mode == WriteMode.OVERWRITE;
+ }
+
+ enum WriteMode {
+ /**
+ * Creates a new file for write and fails if file already exists.
+ */
+ CREATE_NEW,
+ /**
+ * Creates a new file for write or overwrites an existing file if it already exists.
+ */
+ OVERWRITE
+ }
+
+ static CreateOperationOptionsBuilder builder() {
+ return new CreateOperationOptionsBuilder();
+ }
+
+ /**
+ * Extended attributes to set when creating a file.
+ */
+ ImmutableMap getAttributes() {
+ return attributes;
+ }
+
+ /**
+ * Content-type to set when creating a file.
+ */
+ @Nullable
+ String getContentType() {
+ return contentType;
+ }
+
+ /**
+ * Whether to overwrite an existing file with the same name.
+ */
+ WriteMode getWriteMode() {
+ return mode;
+ }
+
+ /**
+ * Generation of existing object to overwrite. Ignored if set to {@link
+ * StorageResourceId#UNKNOWN_GENERATION_ID}, but otherwise this is used instead of {@code
+ * overwriteExisting}, where 0 indicates no existing object, and otherwise an existing object will
+ * only be overwritten by the newly created file if its generation matches this provided
+ * generationId.
+ */
+ long getOverwriteGenerationId() {
+ return overwriteGenerationId;
+ }
+
+ static class CreateOperationOptionsBuilder {
+ private Map attributes = ImmutableMap.of();
+ private String contentType = "application/octet-stream";
+ private long overwriteGenerationId = StorageResourceId.UNKNOWN_GENERATION_ID;
+ private WriteMode writeMode = WriteMode.CREATE_NEW;
+
+ CreateOperationOptionsBuilder setWriteMode(WriteMode mode) {
+ this.writeMode = mode;
+ return this;
+ }
+
+ CreateOptions build() {
+ CreateOptions options = new CreateOptions(this);
+
+ checkArgument(!options.getAttributes().containsKey("Content-Type"),
+ "The Content-Type attribute must be set via the contentType option");
+ if (options.getWriteMode() != WriteMode.OVERWRITE) {
+ checkArgument(options.getOverwriteGenerationId() == StorageResourceId.UNKNOWN_GENERATION_ID,
+ "overwriteGenerationId is set to %s but it can be set only in OVERWRITE mode",
+ options.getOverwriteGenerationId());
+ }
+
+ return options;
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/ErrorTypeExtractor.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/ErrorTypeExtractor.java
new file mode 100644
index 0000000000000..547d855d1d649
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/ErrorTypeExtractor.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import javax.annotation.Nullable;
+
+import io.grpc.Status;
+import io.grpc.StatusRuntimeException;
+
+/**
+ * Implementation for {@link ErrorTypeExtractor} for exception specifically thrown from gRPC path.
+ */
+final class ErrorTypeExtractor {
+
+ static boolean bucketAlreadyExists(Exception e) {
+ ErrorType errorType = getErrorType(e);
+ if (errorType == ErrorType.ALREADY_EXISTS) {
+ return true;
+ } else if (errorType == ErrorType.FAILED_PRECONDITION) {
+ // The gRPC API currently throws a FAILED_PRECONDITION status code instead of ALREADY_EXISTS,
+ // so we handle both these conditions in the interim.
+ StatusRuntimeException statusRuntimeException = getStatusRuntimeException(e);
+ return statusRuntimeException != null
+ && BUCKET_ALREADY_EXISTS_MESSAGE.equals(statusRuntimeException.getMessage());
+ }
+ return false;
+ }
+
+ @Nullable
+ static private StatusRuntimeException getStatusRuntimeException(Exception e) {
+ Throwable cause = e;
+ // Keeping a counter to break early from the loop to avoid infinite loop condition due to
+ // cyclic exception chains.
+ int currentExceptionDepth = 0, maxChainDepth = 1000;
+ while (cause != null && currentExceptionDepth < maxChainDepth) {
+ if (cause instanceof StatusRuntimeException) {
+ return (StatusRuntimeException) cause;
+ }
+ cause = cause.getCause();
+ currentExceptionDepth++;
+ }
+ return null;
+ }
+
+ enum ErrorType {
+ NOT_FOUND, OUT_OF_RANGE, ALREADY_EXISTS, FAILED_PRECONDITION, INTERNAL, RESOURCE_EXHAUSTED,
+ UNAVAILABLE, UNKNOWN
+ }
+
+ // public static final ErrorTypeExtractor INSTANCE = new ErrorTypeExtractor();
+
+ private static final String BUCKET_ALREADY_EXISTS_MESSAGE =
+ "FAILED_PRECONDITION: Your previous request to create the named bucket succeeded and you "
+ + "already own it.";
+
+ private ErrorTypeExtractor() {
+ }
+
+ static ErrorType getErrorType(Exception error) {
+ switch (Status.fromThrowable(error).getCode()) {
+ case NOT_FOUND:
+ return ErrorType.NOT_FOUND;
+ case OUT_OF_RANGE:
+ return ErrorType.OUT_OF_RANGE;
+ case ALREADY_EXISTS:
+ return ErrorType.ALREADY_EXISTS;
+ case FAILED_PRECONDITION:
+ return ErrorType.FAILED_PRECONDITION;
+ case RESOURCE_EXHAUSTED:
+ return ErrorType.RESOURCE_EXHAUSTED;
+ case INTERNAL:
+ return ErrorType.INTERNAL;
+ case UNAVAILABLE:
+ return ErrorType.UNAVAILABLE;
+ default:
+ return ErrorType.UNKNOWN;
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/FileInfo.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/FileInfo.java
new file mode 100644
index 0000000000000..3b9d9f475ae91
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/FileInfo.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import java.net.URI;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Contains information about a file or a directory.
+ *
+ * Note: This class wraps GoogleCloudStorageItemInfo, adds file system specific information and
+ * hides bucket/object specific information.
+ */
+final class FileInfo {
+
+ // Info about the root path.
+ static final FileInfo ROOT_INFO =
+ new FileInfo(GoogleCloudStorageFileSystem.GCSROOT, GoogleCloudStorageItemInfo.ROOT_INFO);
+
+ // Path of this file or directory.
+ private final URI path;
+
+ // Information about the underlying GCS item.
+ private final GoogleCloudStorageItemInfo itemInfo;
+
+ /**
+ * Constructs an instance of FileInfo.
+ *
+ * @param itemInfo Information about the underlying item.
+ */
+ private FileInfo(URI path, GoogleCloudStorageItemInfo itemInfo) {
+ this.itemInfo = itemInfo;
+
+ // Construct the path once.
+ this.path = path;
+ }
+
+ /**
+ * Gets the path of this file or directory.
+ */
+ URI getPath() {
+ return path;
+ }
+
+ /**
+ * Indicates whether this item is a directory.
+ */
+ boolean isDirectory() {
+ return itemInfo.isDirectory();
+ }
+
+ /**
+ * Indicates whether this item is an inferred directory.
+ */
+ boolean isInferredDirectory() {
+ return itemInfo.isInferredDirectory();
+ }
+
+ /**
+ * Indicates whether this instance has information about the unique, shared root of the underlying
+ * storage system.
+ */
+ boolean isGlobalRoot() {
+ return itemInfo.isGlobalRoot();
+ }
+
+ /**
+ * Gets creation time of this item.
+ *
+ *
Time is expressed as milliseconds since January 1, 1970 UTC.
+ */
+ long getCreationTime() {
+ return itemInfo.getCreationTime();
+ }
+
+ /**
+ * Gets the size of this file or directory.
+ *
+ *
For files, size is in number of bytes. For directories size is 0. For items that do not
+ * exist, size is -1.
+ */
+ long getSize() {
+ return itemInfo.getSize();
+ }
+
+ /**
+ * Gets the modification time of this file if one is set, otherwise the value of {@link
+ * #getCreationTime()} is returned.
+ *
+ *
Time is expressed as milliseconds since January 1, 1970 UTC.
+ */
+ long getModificationTime() {
+ return itemInfo.getModificationTime();
+ }
+
+ /**
+ * Retrieve file attributes for this file.
+ *
+ * @return A map of file attributes
+ */
+ Map getAttributes() {
+ return itemInfo.getMetadata();
+ }
+
+ /**
+ * Indicates whether this file or directory exists.
+ */
+ boolean exists() {
+ return itemInfo.exists();
+ }
+
+ /**
+ * Returns CRC32C checksum of the file or {@code null}.
+ */
+ byte[] getCrc32cChecksum() {
+ VerificationAttributes verificationAttributes = itemInfo.getVerificationAttributes();
+ return verificationAttributes == null ? null : verificationAttributes.getCrc32c();
+ }
+
+ /**
+ * Returns MD5 checksum of the file or {@code null}.
+ */
+ byte[] getMd5Checksum() {
+ VerificationAttributes verificationAttributes = itemInfo.getVerificationAttributes();
+ return verificationAttributes == null ? null : verificationAttributes.getMd5hash();
+ }
+
+ /**
+ * Gets information about the underlying item.
+ */
+ GoogleCloudStorageItemInfo getItemInfo() {
+ return itemInfo;
+ }
+
+ /**
+ * Gets string representation of this instance.
+ */
+ @Override
+ public String toString() {
+ return getPath() + (exists() ?
+ ": created on: " + Instant.ofEpochMilli(getCreationTime()) :
+ ": exists: no");
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof FileInfo)) {
+ return false;
+ }
+ FileInfo fileInfo = (FileInfo) o;
+ return Objects.equals(path, fileInfo.path) && Objects.equals(itemInfo, fileInfo.itemInfo);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(path, itemInfo);
+ }
+
+ /**
+ * Handy factory method for constructing a FileInfo from a GoogleCloudStorageItemInfo while
+ * potentially returning a singleton instead of really constructing an object for cases like ROOT.
+ */
+ static FileInfo fromItemInfo(GoogleCloudStorageItemInfo itemInfo) {
+ if (itemInfo.isRoot()) {
+ return ROOT_INFO;
+ }
+ URI path = UriPaths.fromResourceId(itemInfo.getResourceId(), /* allowEmptyObjectName= */ true);
+ return new FileInfo(path, itemInfo);
+ }
+
+ /**
+ * Handy factory method for constructing a list of FileInfo from a list of
+ * GoogleCloudStorageItemInfo.
+ */
+ static List fromItemInfos(List itemInfos) {
+ List fileInfos = new ArrayList<>(itemInfos.size());
+ for (GoogleCloudStorageItemInfo itemInfo : itemInfos) {
+ fileInfos.add(fromItemInfo(itemInfo));
+ }
+ return fileInfos;
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleCloudStorage.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleCloudStorage.java
new file mode 100644
index 0000000000000..d68eca6a8a5f3
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleCloudStorage.java
@@ -0,0 +1,632 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.*;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Strings.isNullOrEmpty;
+import static java.lang.Math.toIntExact;
+
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.ExponentialBackOff;
+import com.google.api.client.util.Sleeper;
+import com.google.api.gax.paging.Page;
+import com.google.cloud.storage.*;
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Maps;
+import org.apache.hadoop.thirdparty.com.google.common.io.BaseEncoding;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.FileAlreadyExistsException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A wrapper around Google cloud storage
+ * client.
+ */
+class GoogleCloudStorage {
+ static final Logger LOG = LoggerFactory.getLogger(GoogleHadoopFileSystem.class);
+ static final List BLOB_FIELDS =
+ ImmutableList.of(
+ Storage.BlobField.BUCKET, Storage.BlobField.CONTENT_ENCODING,
+ Storage.BlobField.CONTENT_TYPE, Storage.BlobField.CRC32C, Storage.BlobField.GENERATION,
+ Storage.BlobField.METADATA, Storage.BlobField.MD5HASH, Storage.BlobField.METAGENERATION,
+ Storage.BlobField.NAME, Storage.BlobField.SIZE, Storage.BlobField.TIME_CREATED,
+ Storage.BlobField.UPDATED);
+
+ static final CreateObjectOptions EMPTY_OBJECT_CREATE_OPTIONS =
+ CreateObjectOptions.DEFAULT_OVERWRITE.toBuilder()
+ .setEnsureEmptyObjectsMetadataMatch(false)
+ .build();
+
+ private final Storage storage;
+ private final GoogleHadoopFileSystemConfiguration configuration;
+
+ /**
+ * Having an instance of gscImpl to redirect calls to Json client while new client implementation
+ * is in WIP.
+ */
+ GoogleCloudStorage(GoogleHadoopFileSystemConfiguration configuration) throws IOException {
+ // TODO: Set credentials
+ this.storage = createStorage(configuration.getProjectId());
+ this.configuration = configuration;
+ }
+
+ private static Storage createStorage(String projectId) {
+ if (projectId != null) {
+ return StorageOptions.newBuilder().setProjectId(projectId).build().getService();
+ }
+
+ return StorageOptions.newBuilder().build().getService();
+ }
+
+ WritableByteChannel create(final StorageResourceId resourceId, final CreateOptions options)
+ throws IOException {
+ LOG.trace("create({})", resourceId);
+
+ checkArgument(resourceId.isStorageObject(), "Expected full StorageObject id, got %s",
+ resourceId);
+ // Update resourceId if generationId is missing
+ StorageResourceId resourceIdWithGeneration = resourceId;
+ if (!resourceId.hasGenerationId()) {
+ resourceIdWithGeneration =
+ new StorageResourceId(resourceId.getBucketName(), resourceId.getObjectName(),
+ getWriteGeneration(resourceId, options.isOverwriteExisting()));
+ }
+
+ return new GoogleCloudStorageClientWriteChannel(storage, resourceIdWithGeneration, options);
+ }
+
+ /**
+ * Gets the object generation for a write operation
+ *
+ * making getItemInfo call even if overwrite is disabled to fail fast in case file is existing.
+ *
+ * @param resourceId object for which generation info is requested
+ * @param overwrite whether existing object should be overwritten
+ * @return the generation of the object
+ * @throws IOException if the object already exists and cannot be overwritten
+ */
+ private long getWriteGeneration(StorageResourceId resourceId, boolean overwrite)
+ throws IOException {
+ LOG.trace("getWriteGeneration({}, {})", resourceId, overwrite);
+ GoogleCloudStorageItemInfo info = getItemInfo(resourceId);
+ if (!info.exists()) {
+ return 0L;
+ }
+ if (info.exists() && overwrite) {
+ long generation = info.getContentGeneration();
+ checkState(generation != 0, "Generation should not be 0 for an existing item");
+ return generation;
+ }
+
+ throw new FileAlreadyExistsException(String.format("Object %s already exists.", resourceId));
+ }
+
+ void close() {
+ try {
+ storage.close();
+ } catch (Exception e) {
+ LOG.warn("Error occurred while closing the storage client", e);
+ }
+ }
+
+ GoogleCloudStorageItemInfo getItemInfo(StorageResourceId resourceId) throws IOException {
+ LOG.trace("getItemInfo({})", resourceId);
+
+ // Handle ROOT case first.
+ if (resourceId.isRoot()) {
+ return GoogleCloudStorageItemInfo.ROOT_INFO;
+ }
+ GoogleCloudStorageItemInfo itemInfo = null;
+
+ if (resourceId.isBucket()) {
+ Bucket bucket = getBucket(resourceId.getBucketName());
+ if (bucket != null) {
+ itemInfo = createItemInfoForBucket(resourceId, bucket);
+ } else {
+ LOG.debug("getBucket({}): not found", resourceId.getBucketName());
+ }
+ } else {
+ Blob blob = getBlob(resourceId);
+ if (blob != null) {
+ itemInfo = createItemInfoForBlob(resourceId, blob);
+ } else {
+ LOG.debug("getObject({}): not found", resourceId);
+ }
+ }
+
+ if (itemInfo == null) {
+ itemInfo = GoogleCloudStorageItemInfo.createNotFound(resourceId);
+ }
+ LOG.debug("getItemInfo: {}", itemInfo);
+ return itemInfo;
+ }
+
+ /**
+ * Gets the bucket with the given name.
+ *
+ * @param bucketName name of the bucket to get
+ * @return the bucket with the given name or null if bucket not found
+ * @throws IOException if the bucket exists but cannot be accessed
+ */
+ @Nullable
+ private Bucket getBucket(String bucketName) throws IOException {
+ LOG.debug("getBucket({})", bucketName);
+ checkArgument(!isNullOrEmpty(bucketName), "bucketName must not be null or empty");
+ try {
+ return storage.get(bucketName);
+ } catch (StorageException e) {
+ if (ErrorTypeExtractor.getErrorType(e) == ErrorTypeExtractor.ErrorType.NOT_FOUND) {
+ return null;
+ }
+ throw new IOException("Error accessing Bucket " + bucketName, e);
+ }
+ }
+
+ private static GoogleCloudStorageItemInfo createItemInfoForBlob(StorageResourceId resourceId,
+ Blob blob) {
+ checkArgument(resourceId != null, "resourceId must not be null");
+ checkArgument(blob != null, "object must not be null");
+ checkArgument(resourceId.isStorageObject(),
+ "resourceId must be a StorageObject. resourceId: %s", resourceId);
+ checkArgument(resourceId.getBucketName().equals(blob.getBucket()),
+ "resourceId.getBucketName() must equal object.getBucket(): '%s' vs '%s'",
+ resourceId.getBucketName(), blob.getBucket());
+ checkArgument(resourceId.getObjectName().equals(blob.getName()),
+ "resourceId.getObjectName() must equal object.getName(): '%s' vs '%s'",
+ resourceId.getObjectName(), blob.getName());
+
+ Map decodedMetadata =
+ blob.getMetadata() == null ? null : decodeMetadata(blob.getMetadata());
+
+ byte[] md5Hash = null;
+ byte[] crc32c = null;
+
+ if (!isNullOrEmpty(blob.getCrc32c())) {
+ crc32c = BaseEncoding.base64().decode(blob.getCrc32c());
+ }
+
+ if (!isNullOrEmpty(blob.getMd5())) {
+ md5Hash = BaseEncoding.base64().decode(blob.getMd5());
+ }
+
+ return GoogleCloudStorageItemInfo.createObject(resourceId,
+ blob.getCreateTimeOffsetDateTime() == null ?
+ 0 :
+ blob.getCreateTimeOffsetDateTime().toInstant().toEpochMilli(),
+ blob.getUpdateTimeOffsetDateTime() == null ?
+ 0 :
+ blob.getUpdateTimeOffsetDateTime().toInstant().toEpochMilli(),
+ blob.getSize() == null ? 0 : blob.getSize(), blob.getContentType(),
+ blob.getContentEncoding(), decodedMetadata,
+ blob.getGeneration() == null ? 0 : blob.getGeneration(),
+ blob.getMetageneration() == null ? 0 : blob.getMetageneration(),
+ new VerificationAttributes(md5Hash, crc32c));
+ }
+
+ static Map decodeMetadata(Map metadata) {
+ return Maps.transformValues(metadata, GoogleCloudStorage::decodeMetadataValues);
+ }
+
+ @Nullable
+ private static byte[] decodeMetadataValues(String value) {
+ try {
+ return BaseEncoding.base64().decode(value);
+ } catch (IllegalArgumentException iae) {
+ LOG.error("Failed to parse base64 encoded attribute value {}", value, iae);
+ return null;
+ }
+ }
+
+ /**
+ * Gets the object with the given resourceId.
+ *
+ * @param resourceId identifies a StorageObject
+ * @return the object with the given name or null if object not found
+ * @throws IOException if the object exists but cannot be accessed
+ */
+ @Nullable
+ Blob getBlob(StorageResourceId resourceId) throws IOException {
+ checkArgument(resourceId.isStorageObject(), "Expected full StorageObject id, got %s",
+ resourceId);
+ String bucketName = resourceId.getBucketName();
+ String objectName = resourceId.getObjectName();
+ Blob blob;
+ try {
+ blob = storage.get(BlobId.of(bucketName, objectName),
+ Storage.BlobGetOption.fields(BLOB_FIELDS.toArray(new Storage.BlobField[0])));
+ } catch (StorageException e) {
+ throw new IOException("Error accessing " + resourceId, e);
+ }
+ return blob;
+ }
+
+ private static GoogleCloudStorageItemInfo createItemInfoForBucket(StorageResourceId resourceId,
+ Bucket bucket) {
+ checkArgument(resourceId != null, "resourceId must not be null");
+ checkArgument(bucket != null, "bucket must not be null");
+ checkArgument(resourceId.isBucket(), "resourceId must be a Bucket. resourceId: %s", resourceId);
+ checkArgument(resourceId.getBucketName().equals(bucket.getName()),
+ "resourceId.getBucketName() must equal bucket.getName(): '%s' vs '%s'",
+ resourceId.getBucketName(), bucket.getName());
+
+ return GoogleCloudStorageItemInfo.createBucket(resourceId,
+ bucket.asBucketInfo().getCreateTimeOffsetDateTime().toInstant().toEpochMilli(),
+ bucket.asBucketInfo().getUpdateTimeOffsetDateTime().toInstant().toEpochMilli(),
+ bucket.getLocation(),
+ bucket.getStorageClass() == null ? null : bucket.getStorageClass().name());
+ }
+
+ List listObjectInfo(
+ String bucketName,
+ String objectNamePrefix,
+ ListObjectOptions listOptions) throws IOException {
+ try {
+ long maxResults = listOptions.getMaxResults() > 0 ?
+ listOptions.getMaxResults() + (listOptions.isIncludePrefix() ? 0 : 1) :
+ listOptions.getMaxResults();
+
+ Storage.BlobListOption[] blobListOptions =
+ getBlobListOptions(objectNamePrefix, listOptions, maxResults);
+ Page blobs = storage.list(bucketName, blobListOptions);
+ ListOperationResult result = new ListOperationResult(maxResults);
+ for (Blob blob : blobs.iterateAll()) {
+ result.add(blob);
+ }
+
+ return result.getItems();
+ } catch (StorageException e) {
+ throw new IOException(
+ String.format("listing object '%s' failed.", BlobId.of(bucketName, objectNamePrefix)),
+ e);
+ }
+ }
+
+ private Storage.BlobListOption[] getBlobListOptions(
+ String objectNamePrefix, ListObjectOptions listOptions, long maxResults) {
+ List options = new ArrayList<>();
+
+ options.add(Storage.BlobListOption.fields(BLOB_FIELDS.toArray(new Storage.BlobField[0])));
+ options.add(Storage.BlobListOption.prefix(objectNamePrefix));
+ // TODO: set max results as a BlobListOption
+ if ("/".equals(listOptions.getDelimiter())) {
+ options.add(Storage.BlobListOption.currentDirectory());
+ }
+
+ if (listOptions.getDelimiter() != null) {
+ options.add(Storage.BlobListOption.includeTrailingDelimiter());
+ }
+
+ return options.toArray(new Storage.BlobListOption[0]);
+ }
+
+ private GoogleCloudStorageItemInfo createItemInfoForBlob(Blob blob) {
+ long generationId = blob.getGeneration() == null ? 0L : blob.getGeneration();
+ StorageResourceId resourceId =
+ new StorageResourceId(blob.getBucket(), blob.getName(), generationId);
+ return createItemInfoForBlob(resourceId, blob);
+ }
+
+ void createBucket(String bucketName, CreateBucketOptions options) throws IOException {
+ LOG.trace("createBucket({})", bucketName);
+ checkArgument(!isNullOrEmpty(bucketName), "bucketName must not be null or empty");
+ checkNotNull(options, "options must not be null");
+
+ BucketInfo.Builder bucketInfoBuilder =
+ BucketInfo.newBuilder(bucketName).setLocation(options.getLocation());
+
+ if (options.getStorageClass() != null) {
+ bucketInfoBuilder.setStorageClass(
+ StorageClass.valueOfStrict(options.getStorageClass().toUpperCase()));
+ }
+ if (options.getTtl() != null) {
+ bucketInfoBuilder.setLifecycleRules(
+ Collections.singletonList(
+ new BucketInfo.LifecycleRule(
+ BucketInfo.LifecycleRule.LifecycleAction.newDeleteAction(),
+ BucketInfo.LifecycleRule.LifecycleCondition.newBuilder()
+ .setAge(toIntExact(options.getTtl().toDays()))
+ .build())));
+ }
+ try {
+ storage.create(bucketInfoBuilder.build());
+ } catch (StorageException e) {
+ if (ErrorTypeExtractor.bucketAlreadyExists(e)) {
+ throw (FileAlreadyExistsException)
+ new FileAlreadyExistsException(String.format("Bucket '%s' already exists.", bucketName))
+ .initCause(e);
+ }
+ throw new IOException(e);
+ }
+ }
+
+ void createEmptyObject(StorageResourceId resourceId) throws IOException {
+ LOG.trace("createEmptyObject({})", resourceId);
+ checkArgument(
+ resourceId.isStorageObject(), "Expected full StorageObject id, got %s", resourceId);
+ createEmptyObject(resourceId, EMPTY_OBJECT_CREATE_OPTIONS);
+ }
+
+ void createEmptyObject(StorageResourceId resourceId, CreateObjectOptions options)
+ throws IOException {
+ checkArgument(
+ resourceId.isStorageObject(), "Expected full StorageObject id, got %s", resourceId);
+
+ try {
+ createEmptyObjectInternal(resourceId, options);
+ } catch (StorageException e) {
+ if (canIgnoreExceptionForEmptyObject(e, resourceId, options)) {
+ LOG.info(
+ "Ignoring exception of type {}; verified object already exists with desired state.",
+ e.getClass().getSimpleName());
+ LOG.trace("Ignored exception while creating empty object: {}", resourceId, e);
+ } else {
+ if (ErrorTypeExtractor.getErrorType(e) == ErrorTypeExtractor.ErrorType.ALREADY_EXISTS) {
+ throw (FileAlreadyExistsException)
+ new FileAlreadyExistsException(
+ String.format("Object '%s' already exists.", resourceId)
+ ).initCause(e);
+ }
+ throw new IOException(e);
+ }
+ }
+ }
+
+ /**
+ * Helper to check whether an empty object already exists with the expected metadata specified in
+ * {@code options}, to be used to determine whether it's safe to ignore an exception that was
+ * thrown when trying to create the object, {@code exceptionOnCreate}.
+ */
+ private boolean canIgnoreExceptionForEmptyObject(
+ StorageException exceptionOnCreate, StorageResourceId resourceId, CreateObjectOptions options)
+ throws IOException {
+ ErrorTypeExtractor.ErrorType errorType = ErrorTypeExtractor.getErrorType(exceptionOnCreate);
+ if (shouldBackoff(resourceId, errorType)) {
+ GoogleCloudStorageItemInfo existingInfo;
+ Duration maxWaitTime = Duration.ofSeconds(3); // TODO: make this configurable
+
+ BackOff backOff =
+ !maxWaitTime.isZero() && !maxWaitTime.isNegative()
+ ? new ExponentialBackOff.Builder()
+ .setMaxElapsedTimeMillis(toIntExact(maxWaitTime.toMillis()))
+ .setMaxIntervalMillis(500)
+ .setInitialIntervalMillis(100)
+ .setMultiplier(1.5)
+ .setRandomizationFactor(0.15)
+ .build()
+ : BackOff.STOP_BACKOFF;
+ long nextSleep = 0L;
+ do {
+ if (nextSleep > 0) {
+ try {
+ Sleeper.DEFAULT.sleep(nextSleep);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ nextSleep = BackOff.STOP;
+ }
+ }
+ existingInfo = getItemInfo(resourceId);
+ nextSleep = nextSleep == BackOff.STOP ? BackOff.STOP : backOff.nextBackOffMillis();
+ } while (!existingInfo.exists() && nextSleep != BackOff.STOP);
+
+ // Compare existence, size, and metadata; for 429 errors creating an empty object,
+ // we don't care about metaGeneration/contentGeneration as long as the metadata
+ // matches, since we don't know for sure whether our low-level request succeeded
+ // first or some other client succeeded first.
+ if (existingInfo.exists() && existingInfo.getSize() == 0) {
+ if (options.isEnsureEmptyObjectsMetadataMatch()) {
+ return existingInfo.metadataEquals(options.getMetadata());
+ }
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private static boolean shouldBackoff(StorageResourceId resourceId,
+ ErrorTypeExtractor.ErrorType errorType) {
+ return errorType == ErrorTypeExtractor.ErrorType.RESOURCE_EXHAUSTED
+ || errorType == ErrorTypeExtractor.ErrorType.INTERNAL ||
+ (resourceId.isDirectory() && errorType == ErrorTypeExtractor.ErrorType.FAILED_PRECONDITION);
+ }
+
+ private void createEmptyObjectInternal(
+ StorageResourceId resourceId, CreateObjectOptions createObjectOptions) throws IOException {
+ Map rewrittenMetadata = encodeMetadata(createObjectOptions.getMetadata());
+
+ List blobTargetOptions = new ArrayList<>();
+ blobTargetOptions.add(Storage.BlobTargetOption.disableGzipContent());
+ if (resourceId.hasGenerationId()) {
+ blobTargetOptions.add(Storage.BlobTargetOption.generationMatch(resourceId.getGenerationId()));
+ } else if (resourceId.isDirectory() || !createObjectOptions.isOverwriteExisting()) {
+ blobTargetOptions.add(Storage.BlobTargetOption.doesNotExist());
+ }
+
+ try {
+ // TODO: Set encryption key and related properties
+ storage.create(
+ BlobInfo.newBuilder(BlobId.of(resourceId.getBucketName(), resourceId.getObjectName()))
+ .setMetadata(rewrittenMetadata)
+ .setContentEncoding(createObjectOptions.getContentEncoding())
+ .setContentType(createObjectOptions.getContentType())
+ .build(),
+ blobTargetOptions.toArray(new Storage.BlobTargetOption[0]));
+ } catch (StorageException e) {
+ throw new IOException(String.format("Creating empty object %s failed.", resourceId), e);
+ }
+ }
+
+ private static Map encodeMetadata(Map metadata) {
+ return Maps.transformValues(metadata, GoogleCloudStorage::encodeMetadataValues);
+ }
+
+ private static String encodeMetadataValues(byte[] bytes) {
+ return bytes == null ? null : BaseEncoding.base64().encode(bytes);
+ }
+
+ List listDirectoryRecursive(String bucketName, String objectName)
+ throws IOException {
+ // TODO: Take delimiter from config
+ // TODO: Set specific fields
+
+ try {
+ Page blobs = storage.list(
+ bucketName,
+ Storage.BlobListOption.prefix(objectName));
+
+ List result = new ArrayList<>();
+ for (Blob blob : blobs.iterateAll()) {
+ result.add(createItemInfoForBlob(blob));
+ }
+
+ return result;
+ } catch (StorageException e) {
+ throw new IOException(
+ String.format("Listing '%s' failed", BlobId.of(bucketName, objectName)), e);
+ }
+ }
+
+ void deleteObjects(List fullObjectNames) throws IOException {
+ LOG.trace("deleteObjects({})", fullObjectNames);
+
+ if (fullObjectNames.isEmpty()) {
+ return;
+ }
+
+ // Validate that all the elements represent StorageObjects.
+ for (StorageResourceId toDelete : fullObjectNames) {
+ checkArgument(
+ toDelete.isStorageObject(),
+ "Expected full StorageObject names only, got: %s",
+ toDelete);
+ }
+
+ // TODO: Do this concurrently
+ // TODO: There is duplication. fix it
+ for (StorageResourceId toDelete : fullObjectNames) {
+ try {
+ LOG.trace("Deleting Object ({})", toDelete);
+ if (toDelete.hasGenerationId() && toDelete.getGenerationId() != 0) {
+ storage.delete(
+ BlobId.of(toDelete.getBucketName(), toDelete.getObjectName()),
+ Storage.BlobSourceOption.generationMatch(toDelete.getGenerationId()));
+ } else {
+ // TODO: Remove delete without generationId
+ storage.delete(BlobId.of(toDelete.getBucketName(), toDelete.getObjectName()));
+
+ LOG.trace("Deleting Object without generationId ({})", toDelete);
+ }
+ } catch (StorageException e) {
+ throw new IOException(String.format("Deleting resource %s failed.", toDelete), e);
+ }
+ }
+ }
+
+ List listBucketInfo() throws IOException {
+ List allBuckets = listBucketsInternal();
+ List bucketInfos = new ArrayList<>(allBuckets.size());
+ for (Bucket bucket : allBuckets) {
+ bucketInfos.add(createItemInfoForBucket(new StorageResourceId(bucket.getName()), bucket));
+ }
+ return bucketInfos;
+ }
+
+
+ private List listBucketsInternal() throws IOException {
+ checkNotNull(configuration.getProjectId(), "projectId must not be null");
+ List allBuckets = new ArrayList<>();
+ try {
+ Page buckets =
+ storage.list(
+ Storage.BucketListOption.pageSize(configuration.getMaxListItemsPerCall()),
+ Storage.BucketListOption.fields(
+ Storage.BucketField.LOCATION,
+ Storage.BucketField.STORAGE_CLASS,
+ Storage.BucketField.TIME_CREATED,
+ Storage.BucketField.UPDATED));
+
+ // Loop to fetch all the items.
+ for (Bucket bucket : buckets.iterateAll()) {
+ allBuckets.add(bucket);
+ }
+ } catch (StorageException e) {
+ throw new IOException(e);
+ }
+ return allBuckets;
+ }
+
+ // Helper class to capture the results of list operation.
+ private class ListOperationResult {
+ private final Map prefixes = new HashMap<>();
+ private final List objects = new ArrayList<>();
+
+ private final Set objectsSet = new HashSet<>();
+
+ private final long maxResults;
+
+ ListOperationResult(long maxResults) {
+ this.maxResults = maxResults;
+ }
+
+ void add(Blob blob) {
+ String path = blob.getBlobId().toGsUtilUri();
+ if (blob.getGeneration() != null) {
+ prefixes.remove(path);
+ objects.add(blob);
+
+ objectsSet.add(path);
+ } else if (!objectsSet.contains(path)) {
+ prefixes.put(path, blob);
+ }
+ }
+
+ List getItems() {
+ List result = new ArrayList<>(prefixes.size() + objects.size());
+
+ for (Blob blob : objects) {
+ result.add(createItemInfoForBlob(blob));
+
+ if (result.size() == maxResults) {
+ return result;
+ }
+ }
+
+ for (Blob blob : prefixes.values()) {
+ if (result.size() == maxResults) {
+ return result;
+ }
+
+ result.add(createItemInfoForBlob(blob));
+ }
+
+ return result;
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleCloudStorageClientWriteChannel.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleCloudStorageClientWriteChannel.java
new file mode 100644
index 0000000000000..7956b6f0a8276
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleCloudStorageClientWriteChannel.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import com.google.cloud.storage.BlobId;
+import com.google.cloud.storage.BlobInfo;
+import com.google.cloud.storage.BlobWriteSession;
+import com.google.cloud.storage.Storage;
+import com.google.cloud.storage.Storage.BlobWriteOption;
+import com.google.cloud.storage.StorageException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Implements WritableByteChannel to provide write access to GCS via java-storage client.
+ */
+class GoogleCloudStorageClientWriteChannel implements WritableByteChannel {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(GoogleCloudStorageClientWriteChannel.class);
+
+ private final StorageResourceId resourceId;
+ private WritableByteChannel writableByteChannel;
+
+ GoogleCloudStorageClientWriteChannel(final Storage storage,
+ final StorageResourceId resourceId, final CreateOptions createOptions) throws IOException {
+ this.resourceId = resourceId;
+ BlobWriteSession blobWriteSession = getBlobWriteSession(storage, resourceId, createOptions);
+ try {
+ this.writableByteChannel = blobWriteSession.open();
+ } catch (StorageException e) {
+ throw new IOException(e);
+ }
+ }
+
+ private static BlobInfo getBlobInfo(final StorageResourceId resourceId,
+ final CreateOptions createOptions) {
+ BlobInfo blobInfo = BlobInfo.newBuilder(
+ BlobId.of(resourceId.getBucketName(), resourceId.getObjectName(),
+ resourceId.getGenerationId())).setContentType(createOptions.getContentType())
+ // .setMetadata(encodeMetadata(createOptions.getMetadata())) // TODO:
+ .build();
+ return blobInfo;
+ }
+
+ private static BlobWriteSession getBlobWriteSession(final Storage storage,
+ final StorageResourceId resourceId, final CreateOptions createOptions) {
+ return storage.blobWriteSession(getBlobInfo(resourceId, createOptions),
+ generateWriteOptions(createOptions));
+ }
+
+ private static BlobWriteOption[] generateWriteOptions(final CreateOptions createOptions) {
+ List blobWriteOptions = new ArrayList<>();
+
+ blobWriteOptions.add(BlobWriteOption.disableGzipContent());
+ blobWriteOptions.add(BlobWriteOption.generationMatch());
+
+ //TODO: Enable KMS and checksum
+ return blobWriteOptions.toArray(new BlobWriteOption[blobWriteOptions.size()]);
+ }
+
+ @Override
+ public boolean isOpen() {
+ return writableByteChannel != null && writableByteChannel.isOpen();
+ }
+
+ @Override
+ public void close() throws IOException {
+ try {
+ if (!isOpen()) {
+ return;
+ }
+
+ writableByteChannel.close();
+ } catch (Exception e) {
+ throw new IOException(
+ String.format("Upload failed for '%s'. reason=%s", resourceId, e.getMessage()), e);
+ } finally {
+ writableByteChannel = null;
+ }
+ }
+
+ private int writeInternal(final ByteBuffer byteBuffer) throws IOException {
+ int bytesWritten = writableByteChannel.write(byteBuffer);
+ LOG.trace("{} bytes were written out of provided buffer of capacity {}", bytesWritten,
+ byteBuffer.limit());
+ return bytesWritten;
+ }
+
+ @Override
+ public int write(final ByteBuffer src) throws IOException {
+ return writeInternal(src);
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleCloudStorageFileSystem.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleCloudStorageFileSystem.java
new file mode 100644
index 0000000000000..aa1617e4da687
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleCloudStorageFileSystem.java
@@ -0,0 +1,374 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.*;
+import static java.util.Comparator.comparing;
+import static org.apache.hadoop.fs.gs.Constants.PATH_DELIMITER;
+import static org.apache.hadoop.fs.gs.Constants.SCHEME;
+
+import com.google.auth.Credentials;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.DirectoryNotEmptyException;
+import java.nio.file.FileAlreadyExistsException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Provides FS semantics over GCS based on Objects API.
+ */
+class GoogleCloudStorageFileSystem {
+ private static final Logger LOG = LoggerFactory.getLogger(StorageResourceId.class);
+ // Comparator used for sorting paths.
+ //
+ // For some bulk operations, we need to operate on parent directories before
+ // we operate on their children. To achieve this, we sort paths such that
+ // shorter paths appear before longer paths. Also, we sort lexicographically
+ // within paths of the same length (this is not strictly required but helps when
+ // debugging/testing).
+ @VisibleForTesting
+ static final Comparator PATH_COMPARATOR =
+ comparing(
+ URI::toString,
+ (as, bs) ->
+ (as.length() == bs.length())
+ ? as.compareTo(bs)
+ : Integer.compare(as.length(), bs.length()));
+
+ static final Comparator FILE_INFO_PATH_COMPARATOR =
+ comparing(FileInfo::getPath, PATH_COMPARATOR);
+
+ private static final ListObjectOptions GET_FILE_INFO_LIST_OPTIONS =
+ ListObjectOptions.DEFAULT.builder().setIncludePrefix(true).setMaxResults(1).build();
+
+ private static final ListObjectOptions LIST_FILE_INFO_LIST_OPTIONS =
+ ListObjectOptions.DEFAULT.builder().setIncludePrefix(true).build();
+
+ // URI of the root path.
+ static final URI GCSROOT = URI.create(SCHEME + ":/");
+
+ // GCS access instance.
+ private GoogleCloudStorage gcs;
+
+ private static GoogleCloudStorage createCloudStorage(
+ final GoogleHadoopFileSystemConfiguration configuration, final Credentials credentials)
+ throws IOException {
+ checkNotNull(configuration, "configuration must not be null");
+
+ return new GoogleCloudStorage(configuration);
+ }
+
+ GoogleCloudStorageFileSystem(final GoogleHadoopFileSystemConfiguration configuration,
+ final Credentials credentials) throws IOException {
+ gcs = createCloudStorage(configuration, credentials);
+ }
+
+ WritableByteChannel create(final URI path, final CreateOptions createOptions)
+ throws IOException {
+ LOG.trace("create(path: {}, createOptions: {})", path, createOptions);
+ checkNotNull(path, "path could not be null");
+ StorageResourceId resourceId =
+ StorageResourceId.fromUriPath(path, /* allowEmptyObjectName=*/ true);
+
+ if (resourceId.isDirectory()) {
+ throw new IOException(
+ String.format("Cannot create a file whose name looks like a directory: '%s'",
+ resourceId));
+ }
+
+ if (createOptions.getOverwriteGenerationId() != StorageResourceId.UNKNOWN_GENERATION_ID) {
+ resourceId = new StorageResourceId(resourceId.getBucketName(), resourceId.getObjectName(),
+ createOptions.getOverwriteGenerationId());
+ }
+
+ return gcs.create(resourceId, createOptions);
+ }
+
+ void close() {
+ if (gcs == null) {
+ return;
+ }
+ LOG.trace("close()");
+ try {
+ gcs.close();
+ } finally {
+ gcs = null;
+ }
+ }
+
+ public FileInfo getFileInfo(URI path) throws IOException {
+ checkArgument(path != null, "path must not be null");
+ // Validate the given path. true == allow empty object name.
+ // One should be able to get info about top level directory (== bucket),
+ // therefore we allow object name to be empty.
+ StorageResourceId resourceId = StorageResourceId.fromUriPath(path, true);
+ FileInfo fileInfo =
+ FileInfo.fromItemInfo(
+ getFileInfoInternal(resourceId, /* inferImplicitDirectories= */ true));
+ LOG.trace("getFileInfo(path: {}): {}", path, fileInfo);
+ return fileInfo;
+ }
+
+ private GoogleCloudStorageItemInfo getFileInfoInternal(
+ StorageResourceId resourceId,
+ boolean inferImplicitDirectories)
+ throws IOException {
+ if (resourceId.isRoot() || resourceId.isBucket()) {
+ return gcs.getItemInfo(resourceId);
+ }
+
+ StorageResourceId dirId = resourceId.toDirectoryId();
+ if (!resourceId.isDirectory()) {
+ GoogleCloudStorageItemInfo itemInfo = gcs.getItemInfo(resourceId);
+ if (itemInfo.exists()) {
+ return itemInfo;
+ }
+
+ if (inferImplicitDirectories) {
+ // TODO: Set max result
+ List listDirResult = gcs.listObjectInfo(
+ resourceId.getBucketName(),
+ resourceId.getObjectName(),
+ GET_FILE_INFO_LIST_OPTIONS);
+ LOG.trace("List for getMetadata returned {}. {}", listDirResult.size(), listDirResult);
+ if (!listDirResult.isEmpty()) {
+ LOG.trace("Get metadata for directory returned non empty {}", listDirResult);
+ return GoogleCloudStorageItemInfo.createInferredDirectory(resourceId.toDirectoryId());
+ }
+ }
+ }
+
+ List listDirInfo = ImmutableList.of(gcs.getItemInfo(dirId));
+ if (listDirInfo.isEmpty()) {
+ return GoogleCloudStorageItemInfo.createNotFound(resourceId);
+ }
+ checkState(listDirInfo.size() <= 2, "listed more than 2 objects: '%s'", listDirInfo);
+ GoogleCloudStorageItemInfo dirInfo = Iterables.get(listDirInfo, /* position= */ 0);
+ checkState(
+ dirInfo.getResourceId().equals(dirId) || !inferImplicitDirectories,
+ "listed wrong object '%s', but should be '%s'",
+ dirInfo.getResourceId(),
+ resourceId);
+ return dirInfo.getResourceId().equals(dirId) && dirInfo.exists()
+ ? dirInfo
+ : GoogleCloudStorageItemInfo.createNotFound(resourceId);
+ }
+
+ public void mkdirs(URI path) throws IOException {
+ LOG.trace("mkdirs(path: {})", path);
+ checkNotNull(path, "path should not be null");
+
+ /* allowEmptyObjectName= */
+ StorageResourceId resourceId =
+ StorageResourceId.fromUriPath(path, /* allowEmptyObjectName= */ true);
+ if (resourceId.isRoot()) {
+ // GCS_ROOT directory always exists, no need to go through the rest of the method.
+ return;
+ }
+
+ // In case path is a bucket we just attempt to create it without additional checks
+ if (resourceId.isBucket()) {
+ try {
+ gcs.createBucket(resourceId.getBucketName(), CreateBucketOptions.DEFAULT);
+ } catch (FileAlreadyExistsException e) {
+ // This means that bucket already exist, and we do not need to do anything.
+ LOG.trace("mkdirs: {} already exists, ignoring creation failure", resourceId, e);
+ }
+ return;
+ }
+
+ resourceId = resourceId.toDirectoryId();
+
+ // TODO: Before creating a leaf directory we need to check if there are no conflicting files
+ // TODO: with the same name as any subdirectory
+
+ // Create only a leaf directory because subdirectories will be inferred
+ // if leaf directory exists
+ try {
+ gcs.createEmptyObject(resourceId);
+ } catch (FileAlreadyExistsException e) {
+ // This means that directory object already exist, and we do not need to do anything.
+ LOG.trace("mkdirs: {} already exists, ignoring creation failure", resourceId, e);
+ }
+ }
+
+ void delete(URI path, boolean recursive) throws IOException {
+ checkNotNull(path, "path should not be null");
+ checkArgument(!path.equals(GCSROOT), "Cannot delete root path (%s)", path);
+
+ FileInfo fileInfo = getFileInfo(path);
+ if (!fileInfo.exists()) {
+ throw new FileNotFoundException("Item not found: " + path);
+ }
+
+ List itemsToDelete;
+ // Delete sub-items if it is a directory.
+ if (fileInfo.isDirectory()) {
+ itemsToDelete =
+ recursive
+ ? listRecursive(fileInfo.getPath()) // TODO: Get only one result
+ : listDirectory(fileInfo.getPath());
+
+ if (!itemsToDelete.isEmpty() && !recursive) {
+ throw new DirectoryNotEmptyException("Cannot delete a non-empty directory. : " + path);
+ }
+ } else {
+ itemsToDelete = new ArrayList<>();
+ }
+
+ List bucketsToDelete = new ArrayList<>();
+ (fileInfo.getItemInfo().isBucket() ? bucketsToDelete : itemsToDelete).add(fileInfo);
+
+ deleteObjects(itemsToDelete, bucketsToDelete);
+
+ StorageResourceId parentId =
+ StorageResourceId.fromUriPath(UriPaths.getParentPath(path), true);
+ GoogleCloudStorageItemInfo parentInfo =
+ getFileInfoInternal(parentId, /* inferImplicitDirectories= */ false);
+
+ StorageResourceId resourceId = parentInfo.getResourceId();
+ if (parentInfo.exists()
+ || resourceId.isRoot()
+ || resourceId.isBucket()
+ || PATH_DELIMITER.equals(resourceId.getObjectName())) {
+ return;
+ }
+
+ // TODO: Keep the repair parent step behind a flag
+ gcs.createEmptyObject(parentId);
+ }
+
+ private List listRecursive(URI prefix) throws IOException {
+ StorageResourceId prefixId = getPrefixId(prefix);
+ List itemInfos =
+ gcs.listDirectoryRecursive(prefixId.getBucketName(), prefixId.getObjectName());
+ List fileInfos = FileInfo.fromItemInfos(itemInfos);
+ fileInfos.sort(FILE_INFO_PATH_COMPARATOR);
+ return fileInfos;
+ }
+
+ private List listDirectory(URI prefix) throws IOException {
+ StorageResourceId prefixId = getPrefixId(prefix);
+ List itemInfos = gcs.listObjectInfo(
+ prefixId.getBucketName(),
+ prefixId.getObjectName(),
+ ListObjectOptions.DEFAULT_FLAT_LIST);
+
+ List fileInfos = FileInfo.fromItemInfos(itemInfos);
+ fileInfos.sort(FILE_INFO_PATH_COMPARATOR);
+ return fileInfos;
+ }
+
+ private StorageResourceId getPrefixId(URI prefix) {
+ checkNotNull(prefix, "prefix could not be null");
+
+ StorageResourceId prefixId = StorageResourceId.fromUriPath(prefix, true);
+ checkArgument(!prefixId.isRoot(), "prefix must not be global root, got '%s'", prefix);
+
+ return prefixId;
+ }
+
+ private void deleteObjects(
+ List itemsToDelete, List bucketsToDelete)
+ throws IOException {
+ LOG.trace("deleteInternalWithFolders; fileSize={} bucketSize={}",
+ itemsToDelete.size(), bucketsToDelete.size());
+ deleteObjects(itemsToDelete);
+ deleteBucket(bucketsToDelete);
+ }
+
+ private void deleteObjects(List itemsToDelete) throws IOException {
+ // Delete children before their parents.
+ //
+ // Note: we modify the input list, which is ok for current usage.
+ // We should make a copy in case that changes in future.
+ itemsToDelete.sort(FILE_INFO_PATH_COMPARATOR.reversed());
+
+ if (!itemsToDelete.isEmpty()) {
+ List objectsToDelete = new ArrayList<>(itemsToDelete.size());
+ for (FileInfo fileInfo : itemsToDelete) {
+ if (!fileInfo.isInferredDirectory()) {
+ objectsToDelete.add(
+ new StorageResourceId(
+ fileInfo.getItemInfo().getBucketName(),
+ fileInfo.getItemInfo().getObjectName(),
+ fileInfo.getItemInfo().getContentGeneration()));
+ }
+ }
+
+ gcs.deleteObjects(objectsToDelete);
+ }
+ }
+
+ private void deleteBucket(List bucketsToDelete) throws IOException {
+ if (bucketsToDelete == null || bucketsToDelete.isEmpty()) {
+ return;
+ }
+
+ // TODO: Add support for deleting bucket
+ throw new UnsupportedOperationException("deleteBucket is not supported.");
+ }
+
+ public List listFileInfo(URI path, ListFileOptions listOptions) throws IOException {
+ checkNotNull(path, "path can not be null");
+ LOG.trace("listStatus(path: {})", path);
+
+ StorageResourceId pathId =
+ StorageResourceId.fromUriPath(path, /* allowEmptyObjectName= */ true);
+
+ if (!pathId.isDirectory()) {
+ GoogleCloudStorageItemInfo pathInfo = gcs.getItemInfo(pathId);
+ if (pathInfo.exists()) {
+ List listedInfo = new ArrayList<>();
+ listedInfo.add(FileInfo.fromItemInfo(pathInfo));
+
+ return listedInfo;
+ }
+ }
+
+ StorageResourceId dirId = pathId.toDirectoryId();
+ List dirItemInfos = dirId.isRoot() ?
+ gcs.listBucketInfo() :
+ gcs.listObjectInfo(
+ dirId.getBucketName(), dirId.getObjectName(), LIST_FILE_INFO_LIST_OPTIONS);
+
+ if (pathId.isStorageObject() && dirItemInfos.isEmpty()) {
+ throw new FileNotFoundException("Item not found: " + path);
+ }
+
+ if (!dirItemInfos.isEmpty() && Objects.equals(dirItemInfos.get(0).getResourceId(), dirId)) {
+ dirItemInfos.remove(0);
+ }
+
+ List fileInfos = FileInfo.fromItemInfos(dirItemInfos);
+ fileInfos.sort(FILE_INFO_PATH_COMPARATOR);
+ return fileInfos;
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleCloudStorageItemInfo.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleCloudStorageItemInfo.java
new file mode 100644
index 0000000000000..83169b8d9213d
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleCloudStorageItemInfo.java
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
+
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Contains information about an item in Google Cloud Storage.
+ */
+final class GoogleCloudStorageItemInfo {
+ // Info about the root of GCS namespace.
+ public static final GoogleCloudStorageItemInfo ROOT_INFO =
+ new GoogleCloudStorageItemInfo(StorageResourceId.ROOT,
+ /* creationTime= */ 0,
+ /* modificationTime= */ 0,
+ /* size= */ 0,
+ /* location= */ null,
+ /* storageClass= */ null,
+ /* contentType= */ null,
+ /* contentEncoding= */ null,
+ /* metadata= */ null,
+ /* contentGeneration= */ 0,
+ /* metaGeneration= */ 0,
+ /* verificationAttributes= */ null);
+
+ /**
+ * Factory method for creating a GoogleCloudStorageItemInfo for a bucket.
+ *
+ * @param resourceId Resource ID that identifies a bucket
+ * @param creationTime Time when a bucket was created (milliseconds since January 1, 1970
+ * UTC).
+ * @param modificationTime Time when a bucket was last modified (milliseconds since January 1,
+ * 1970 UTC).
+ * @param location Location of a bucket.
+ * @param storageClass Storage class of a bucket.
+ */
+ static GoogleCloudStorageItemInfo createBucket(StorageResourceId resourceId,
+ long creationTime, long modificationTime, String location, String storageClass) {
+ checkNotNull(resourceId, "resourceId must not be null");
+ checkArgument(resourceId.isBucket(), "expected bucket but got '%s'", resourceId);
+ return new GoogleCloudStorageItemInfo(resourceId, creationTime, modificationTime,
+ /* size= */ 0, location, storageClass,
+ /* contentType= */ null,
+ /* contentEncoding= */ null,
+ /* metadata= */ null,
+ /* contentGeneration= */ 0,
+ /* metaGeneration= */ 0,
+ /* verificationAttributes= */ null);
+ }
+
+ /**
+ * Factory method for creating a GoogleCloudStorageItemInfo for an object.
+ *
+ * @param resourceId identifies either root, a Bucket, or a StorageObject
+ * @param creationTime Time when object was created (milliseconds since January 1, 1970
+ * UTC).
+ * @param size Size of the given object (number of bytes) or -1 if the object
+ * does not exist.
+ * @param metadata User-supplied object metadata for this object.
+ */
+ static GoogleCloudStorageItemInfo createObject(StorageResourceId resourceId,
+ long creationTime, long modificationTime, long size, String contentType,
+ String contentEncoding, Map metadata, long contentGeneration,
+ long metaGeneration, VerificationAttributes verificationAttributes) {
+ checkNotNull(resourceId, "resourceId must not be null");
+ checkArgument(
+ !resourceId.isRoot(),
+ "expected object or directory but got '%s'", resourceId);
+ checkArgument(
+ !resourceId.isBucket(),
+ "expected object or directory but got '%s'", resourceId);
+ return new GoogleCloudStorageItemInfo(resourceId, creationTime, modificationTime, size,
+ /* location= */ null,
+ /* storageClass= */ null, contentType, contentEncoding, metadata, contentGeneration,
+ metaGeneration, verificationAttributes);
+ }
+
+ /**
+ * Factory method for creating a "found" GoogleCloudStorageItemInfo for an inferred directory.
+ *
+ * @param resourceId Resource ID that identifies an inferred directory
+ */
+ static GoogleCloudStorageItemInfo createInferredDirectory(StorageResourceId resourceId) {
+ return new GoogleCloudStorageItemInfo(resourceId,
+ /* creationTime= */ 0,
+ /* modificationTime= */ 0,
+ /* size= */ 0,
+ /* location= */ null,
+ /* storageClass= */ null,
+ /* contentType= */ null,
+ /* contentEncoding= */ null,
+ /* metadata= */ null,
+ /* contentGeneration= */ 0,
+ /* metaGeneration= */ 0,
+ /* verificationAttributes= */ null);
+ }
+
+ /**
+ * Factory method for creating a "not found" GoogleCloudStorageItemInfo for a bucket or an object.
+ *
+ * @param resourceId Resource ID that identifies an inferred directory
+ */
+ static GoogleCloudStorageItemInfo createNotFound(StorageResourceId resourceId) {
+ return new GoogleCloudStorageItemInfo(resourceId,
+ /* creationTime= */ 0,
+ /* modificationTime= */ 0,
+ /* size= */ -1,
+ /* location= */ null,
+ /* storageClass= */ null,
+ /* contentType= */ null,
+ /* contentEncoding= */ null,
+ /* metadata= */ null,
+ /* contentGeneration= */ 0,
+ /* metaGeneration= */ 0,
+ /* verificationAttributes= */ null);
+ }
+
+ // The Bucket and maybe StorageObject names of the GCS "item" referenced by this object. Not
+ // null.
+ private final StorageResourceId resourceId;
+
+ // Creation time of this item.
+ // Time is expressed as milliseconds since January 1, 1970 UTC.
+ private final long creationTime;
+
+ // Modification time of this item.
+ // Time is expressed as milliseconds since January 1, 1970 UTC.
+ private final long modificationTime;
+
+ // Size of an object (number of bytes).
+ // Size is -1 for items that do not exist.
+ private final long size;
+
+ // Location of this item.
+ private final String location;
+
+ // Storage class of this item.
+ private final String storageClass;
+
+ // Content-Type of this item
+ private final String contentType;
+
+ private final String contentEncoding;
+
+ // User-supplied metadata.
+ private final Map metadata;
+
+ private final long contentGeneration;
+
+ private final long metaGeneration;
+
+ private final VerificationAttributes verificationAttributes;
+
+ private GoogleCloudStorageItemInfo(StorageResourceId resourceId, long creationTime,
+ long modificationTime, long size, String location, String storageClass, String contentType,
+ String contentEncoding, Map metadata, long contentGeneration,
+ long metaGeneration, VerificationAttributes verificationAttributes) {
+ this.resourceId = checkNotNull(resourceId, "resourceId must not be null");
+ this.creationTime = creationTime;
+ this.modificationTime = modificationTime;
+ this.size = size;
+ this.location = location;
+ this.storageClass = storageClass;
+ this.contentType = contentType;
+ this.contentEncoding = contentEncoding;
+ this.metadata = (metadata == null) ? ImmutableMap.of() : metadata;
+ this.contentGeneration = contentGeneration;
+ this.metaGeneration = metaGeneration;
+ this.verificationAttributes = verificationAttributes;
+ }
+
+ /**
+ * Gets bucket name of this item.
+ */
+ String getBucketName() {
+ return resourceId.getBucketName();
+ }
+
+ /**
+ * Gets object name of this item.
+ */
+ String getObjectName() {
+ return resourceId.getObjectName();
+ }
+
+ /**
+ * Gets the resourceId that holds the (possibly null) bucketName and objectName of this object.
+ */
+ StorageResourceId getResourceId() {
+ return resourceId;
+ }
+
+ /**
+ * Gets creation time of this item.
+ *
+ * Time is expressed as milliseconds since January 1, 1970 UTC.
+ */
+ long getCreationTime() {
+ return creationTime;
+ }
+
+ /**
+ * Gets modification time of this item.
+ *
+ *
Time is expressed as milliseconds since January 1, 1970 UTC.
+ */
+ long getModificationTime() {
+ return modificationTime;
+ }
+
+ /**
+ * Gets size of this item (number of bytes). Returns -1 if the object does not exist.
+ */
+ long getSize() {
+ return size;
+ }
+
+ /**
+ * Gets location of this item.
+ *
+ *
Note: Location is only supported for buckets. The value is always null for objects.
+ */
+ String getLocation() {
+ return location;
+ }
+
+ /**
+ * Gets storage class of this item.
+ *
+ *
Note: Storage-class is only supported for buckets. The value is always null for objects.
+ */
+ String getStorageClass() {
+ return storageClass;
+ }
+
+ /**
+ * Gets the content-type of this item, or null if unknown or inapplicable.
+ *
+ *
Note: content-type is only supported for objects, and will always be null for buckets.
+ */
+ String getContentType() {
+ return contentType;
+ }
+
+ /**
+ * Gets the content-encoding of this item, or null if unknown or inapplicable.
+ *
+ *
Note: content-encoding is only supported for objects, and will always be null for buckets.
+ */
+ String getContentEncoding() {
+ return contentEncoding;
+ }
+
+ /**
+ * Gets user-supplied metadata for this item.
+ *
+ *
Note: metadata is only supported for objects. This value is always an empty map for buckets.
+ */
+ Map getMetadata() {
+ return metadata;
+ }
+
+ /**
+ * Indicates whether this item is a bucket. Root is not considered to be a bucket.
+ */
+ boolean isBucket() {
+ return resourceId.isBucket();
+ }
+
+ /**
+ * Indicates whether this item refers to the GCS root (gs://).
+ */
+ boolean isRoot() {
+ return resourceId.isRoot();
+ }
+
+ /**
+ * Indicates whether this instance has information about the unique, shared root of the underlying
+ * storage system.
+ */
+ boolean isGlobalRoot() {
+ return isRoot() && exists();
+ }
+
+ /**
+ * Indicates whether {@code itemInfo} is a directory.
+ */
+ boolean isDirectory() {
+ return isGlobalRoot() || isBucket() || resourceId.isDirectory();
+ }
+
+ /**
+ * Indicates whether {@code itemInfo} is an inferred directory.
+ */
+ boolean isInferredDirectory() {
+ return creationTime == 0 && modificationTime == 0 && size == 0 && contentGeneration == 0
+ && metaGeneration == 0;
+ }
+
+ /**
+ * Get the content generation of the object.
+ */
+ long getContentGeneration() {
+ return contentGeneration;
+ }
+
+ /**
+ * Get the meta generation of the object.
+ */
+ long getMetaGeneration() {
+ return metaGeneration;
+ }
+
+ /**
+ * Get object validation attributes.
+ */
+ VerificationAttributes getVerificationAttributes() {
+ return verificationAttributes;
+ }
+
+ /**
+ * Indicates whether this item exists.
+ */
+ boolean exists() {
+ return size >= 0;
+ }
+
+ /**
+ * Helper for checking logical equality of metadata maps, checking equality of keySet() between
+ * this.metadata and otherMetadata, and then using Arrays.equals to compare contents of
+ * corresponding byte arrays.
+ */
+ @VisibleForTesting
+ public boolean metadataEquals(Map otherMetadata) {
+ if (metadata == otherMetadata) {
+ // Fast-path for common cases where the same actual default metadata instance may be
+ // used in
+ // multiple different item infos.
+ return true;
+ }
+ // No need to check if other `metadata` is not null,
+ // because previous `if` checks if both of them are null.
+ if (metadata == null || otherMetadata == null) {
+ return false;
+ }
+ if (!metadata.keySet().equals(otherMetadata.keySet())) {
+ return false;
+ }
+
+ // Compare each byte[] with Arrays.equals.
+ for (Map.Entry metadataEntry : metadata.entrySet()) {
+ if (!Arrays.equals(metadataEntry.getValue(), otherMetadata.get(metadataEntry.getKey()))) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ /**
+ * Gets string representation of this instance.
+ */
+ @Override
+ public String toString() {
+ return exists() ?
+ String.format("%s: created on: %s", resourceId, Instant.ofEpochMilli(creationTime)) :
+ String.format("%s: exists: no", resourceId);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof GoogleCloudStorageItemInfo) {
+ GoogleCloudStorageItemInfo other = (GoogleCloudStorageItemInfo) obj;
+ return resourceId.equals(other.resourceId) && creationTime == other.creationTime
+ && modificationTime == other.modificationTime && size == other.size && Objects.equals(
+ location, other.location) && Objects.equals(storageClass, other.storageClass)
+ && Objects.equals(verificationAttributes, other.verificationAttributes)
+ && metaGeneration == other.metaGeneration && contentGeneration == other.contentGeneration
+ && metadataEquals(other.getMetadata());
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + resourceId.hashCode();
+ result = prime * result + (int) creationTime;
+ result = prime * result + (int) modificationTime;
+ result = prime * result + (int) size;
+ result = prime * result + Objects.hashCode(location);
+ result = prime * result + Objects.hashCode(storageClass);
+ result = prime * result + Objects.hashCode(verificationAttributes);
+ result = prime * result + (int) metaGeneration;
+ result = prime * result + (int) contentGeneration;
+ result = prime * result + metadata.entrySet().stream()
+ .mapToInt(e -> Objects.hash(e.getKey()) + Arrays.hashCode(e.getValue())).sum();
+ return result;
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleHadoopFileSystem.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleHadoopFileSystem.java
new file mode 100644
index 0000000000000..8831568a3560d
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleHadoopFileSystem.java
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import static org.apache.hadoop.fs.gs.Constants.GCS_CONFIG_PREFIX;
+import static org.apache.hadoop.fs.gs.GoogleHadoopFileSystemConfiguration.GCS_WORKING_DIRECTORY;
+
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkState;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Strings.isNullOrEmpty;
+
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.hadoop.thirdparty.com.google.common.base.Ascii;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.DirectoryNotEmptyException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * GoogleHadoopFileSystem is rooted in a single bucket at initialization time; in this case, Hadoop
+ * paths no longer correspond directly to general GCS paths, and all Hadoop operations going through
+ * this FileSystem will never touch any GCS bucket other than the bucket on which this FileSystem is
+ * rooted.
+ *
+ * This implementation sacrifices a small amount of cross-bucket interoperability in favor of
+ * more straightforward FileSystem semantics and compatibility with existing Hadoop applications. In
+ * particular, it is not subject to bucket-naming constraints, and files are allowed to be placed in
+ * root.
+ */
+public class GoogleHadoopFileSystem extends FileSystem {
+
+ public static final Logger LOG = LoggerFactory.getLogger(GoogleHadoopFileSystem.class);
+
+ /**
+ * URI scheme for GoogleHadoopFileSystem.
+ */
+ private static final String SCHEME = Constants.SCHEME;
+
+ /**
+ * Default value of replication factor.
+ */
+ static final short REPLICATION_FACTOR_DEFAULT = 3;
+
+ // TODO: Take this from config
+ private static final int PERMISSIONS_TO_REPORT = 700;
+
+ /**
+ * The URI the File System is passed in initialize.
+ */
+ private URI initUri;
+
+ /**
+ * Default block size. Note that this is the size that is reported to Hadoop FS clients. It does
+ * not modify the actual block size of an underlying GCS object, because GCS JSON API does not
+ * allow modifying or querying the value. Modifying this value allows one to control how many
+ * mappers are used to process a given file.
+ */
+ private long defaultBlockSize = GoogleHadoopFileSystemConfiguration.BLOCK_SIZE.getDefault();
+
+ // The bucket the file system is rooted in used for default values of:
+ // -- working directory
+ // -- user home directories (only for Hadoop purposes).
+ private Path fsRoot;
+
+ /**
+ * Current working directory; overridden in initialize() if {@link
+ * GoogleHadoopFileSystemConfiguration#GCS_WORKING_DIRECTORY} is set.
+ */
+ private Path workingDirectory;
+ private GoogleCloudStorageFileSystem gcsFs;
+ private boolean isClosed;
+ private FsPermission reportedPermissions;
+
+ public GoogleHadoopFileSystemConfiguration getFileSystemConfiguration() {
+ return fileSystemConfiguration;
+ }
+
+ private GoogleHadoopFileSystemConfiguration fileSystemConfiguration;
+
+ @Override
+ public void initialize(final URI path, Configuration config) throws IOException {
+ LOG.trace("initialize(path: {}, config: {})", path, config);
+
+ checkArgument(path != null, "path must not be null");
+ checkArgument(config != null, "config must not be null");
+ checkArgument(path.getScheme() != null, "scheme of path must not be null");
+ checkArgument(path.getScheme().equals(getScheme()), "URI scheme not supported: {}", path);
+
+ config =
+ ProviderUtils.excludeIncompatibleCredentialProviders(config, GoogleHadoopFileSystem.class);
+ super.initialize(path, config);
+
+ initUri = path;
+
+ // Set this configuration as the default config for this instance; configure()
+ // will perform some file-system-specific adjustments, but the original should
+ // be sufficient (and is required) for the delegation token binding initialization.
+ setConf(config);
+
+ this.reportedPermissions = new FsPermission(PERMISSIONS_TO_REPORT);
+
+ initializeFsRoot();
+
+ this.fileSystemConfiguration = new GoogleHadoopFileSystemConfiguration(config);
+ initializeWorkingDirectory(fileSystemConfiguration);
+ initializeGcsFs(fileSystemConfiguration);
+ }
+
+ private void initializeFsRoot() {
+ String rootBucket = initUri.getAuthority();
+ checkArgument(rootBucket != null, "No bucket specified in GCS URI: {}", initUri);
+ // Validate root bucket name
+ URI rootUri = UriPaths.fromStringPathComponents(rootBucket, /* objectName= */
+ null, /* allowEmptyObjectName= */ true);
+ fsRoot = new Path(rootUri);
+ LOG.trace("Configured FS root: '{}'", fsRoot);
+ }
+
+ private void initializeWorkingDirectory(final GoogleHadoopFileSystemConfiguration config) {
+ String configWorkingDirectory = config.getWorkingDirectory();
+ if (isNullOrEmpty(configWorkingDirectory)) {
+ LOG.warn("No working directory configured, using default: '{}'", workingDirectory);
+ }
+ // Use the public method to ensure proper behavior of normalizing and resolving the new
+ // working directory relative to the initial filesystem-root directory.
+ setWorkingDirectory(
+ isNullOrEmpty(configWorkingDirectory) ? fsRoot : new Path(configWorkingDirectory));
+ LOG.trace("Configured working directory: {} = {}", GCS_WORKING_DIRECTORY.getKey(),
+ getWorkingDirectory());
+ }
+
+ private synchronized void initializeGcsFs(final GoogleHadoopFileSystemConfiguration config)
+ throws IOException {
+ this.gcsFs = createGcsFs(config);
+ }
+
+ private GoogleCloudStorageFileSystem createGcsFs(final GoogleHadoopFileSystemConfiguration config)
+ throws IOException {
+ GoogleCredentials credentials = getCredentials(config);
+ return new GoogleCloudStorageFileSystem(config, credentials);
+ }
+
+ private GoogleCredentials getCredentials(GoogleHadoopFileSystemConfiguration config)
+ throws IOException {
+ return getCredentials(config, GCS_CONFIG_PREFIX);
+ }
+
+ public static GoogleCredentials getCredentials(GoogleHadoopFileSystemConfiguration config,
+ String... keyPrefixesVararg) throws IOException {
+ return GoogleCredentials.getApplicationDefault(); // TODO: Add other Auth mechanisms
+ }
+
+ @Override
+ protected void checkPath(final Path path) {
+ LOG.trace("checkPath(path: {})", path);
+ // Validate scheme
+ URI uri = path.toUri();
+
+ String scheme = uri.getScheme();
+ if (scheme != null && !scheme.equalsIgnoreCase(getScheme())) {
+ throw new IllegalArgumentException(
+ String.format("Wrong scheme: {}, in path: {}, expected scheme: {}", scheme, path,
+ getScheme()));
+ }
+
+ String bucket = uri.getAuthority();
+ String rootBucket = fsRoot.toUri().getAuthority();
+
+ // Bucket-less URIs will be qualified later
+ if (bucket == null || bucket.equals(rootBucket)) {
+ return;
+ }
+
+ throw new IllegalArgumentException(
+ String.format("Wrong bucket: {}, in path: {}, expected bucket: {}", bucket, path,
+ rootBucket));
+ }
+
+ /**
+ * Validates that GCS path belongs to this file system. The bucket must match the root bucket
+ * provided at initialization time.
+ */
+ Path getHadoopPath(final URI gcsPath) {
+ LOG.trace("getHadoopPath(gcsPath: {})", gcsPath);
+
+ // Handle root. Delegate to getGcsPath on "gs:/" to resolve the appropriate gs:// URI.
+ if (gcsPath.equals(getGcsPath(fsRoot))) {
+ return fsRoot;
+ }
+
+ StorageResourceId resourceId = StorageResourceId.fromUriPath(gcsPath, true);
+
+ checkArgument(!resourceId.isRoot(), "Missing authority in gcsPath '{}'", gcsPath);
+ String rootBucket = fsRoot.toUri().getAuthority();
+ checkArgument(resourceId.getBucketName().equals(rootBucket),
+ "Authority of URI '{}' doesn't match root bucket '{}'", resourceId.getBucketName(),
+ rootBucket);
+
+ Path hadoopPath = new Path(fsRoot,
+ new Path(/* schema= */ null, /* authority= */ null, resourceId.getObjectName()));
+ LOG.trace("getHadoopPath(gcsPath: {}): {}", gcsPath, hadoopPath);
+ return hadoopPath;
+ }
+
+ /**
+ * Translates a "gs:/" style hadoopPath (or relative path which is not fully-qualified) into the
+ * appropriate GCS path which is compatible with the underlying GcsFs.
+ */
+ URI getGcsPath(final Path hadoopPath) {
+ LOG.trace("getGcsPath(hadoopPath: {})", hadoopPath);
+
+ // Convert to fully qualified absolute path; the Path object will call back to get our current
+ // workingDirectory as part of fully resolving the path.
+ Path resolvedPath = makeQualified(hadoopPath);
+
+ String objectName = resolvedPath.toUri().getPath();
+ if (objectName != null && resolvedPath.isAbsolute()) {
+ // Strip off leading '/' because GoogleCloudStorageFileSystem.getPath appends it explicitly
+ // between bucket and objectName.
+ objectName = objectName.substring(1);
+ }
+
+ // Construct GCS path URI
+ String rootBucket = fsRoot.toUri().getAuthority();
+ URI gcsPath =
+ UriPaths.fromStringPathComponents(rootBucket, objectName, /* allowEmptyObjectName= */ true);
+ LOG.trace("getGcsPath(hadoopPath: {}): {}", hadoopPath, gcsPath);
+ return gcsPath;
+ }
+
+ @Override
+ public String getScheme() {
+ return SCHEME;
+ }
+
+ @Override
+ public FSDataInputStream open(final Path path, final int bufferSize) throws IOException {
+ LOG.trace("open({})", path);
+ throw new UnsupportedOperationException(path.toString());
+ }
+
+ @Override
+ public FSDataOutputStream create(Path hadoopPath, FsPermission permission, boolean overwrite,
+ int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
+ checkArgument(hadoopPath != null, "hadoopPath must not be null");
+ checkArgument(replication > 0, "replication must be a positive integer: %s", replication);
+ checkArgument(blockSize > 0, "blockSize must be a positive integer: %s", blockSize);
+
+ checkOpen();
+
+ LOG.trace("create(hadoopPath: {}, overwrite: {}, bufferSize: {} [ignored])", hadoopPath,
+ overwrite, bufferSize);
+
+ CreateOptions.WriteMode writeMode =
+ overwrite ? CreateOptions.WriteMode.OVERWRITE : CreateOptions.WriteMode.CREATE_NEW;
+ FSDataOutputStream response = new FSDataOutputStream(
+ new GoogleHadoopOutputStream(this, getGcsPath(hadoopPath),
+ CreateOptions.builder().setWriteMode(writeMode).build(), statistics), statistics);
+
+ return response;
+ }
+
+ @Override
+ public FSDataOutputStream createNonRecursive(
+ Path hadoopPath,
+ FsPermission permission,
+ EnumSet flags,
+ int bufferSize,
+ short replication,
+ long blockSize,
+ Progressable progress)
+ throws IOException {
+ URI gcsPath = getGcsPath(checkNotNull(hadoopPath, "hadoopPath must not be null"));
+ URI parentGcsPath = UriPaths.getParentPath(gcsPath);
+ if (!getGcsFs().getFileInfo(parentGcsPath).exists()) {
+ throw new FileNotFoundException(
+ String.format(
+ "Can not create '%s' file, because parent folder does not exist: %s",
+ gcsPath, parentGcsPath));
+ }
+
+ return create(
+ hadoopPath,
+ permission,
+ flags.contains(CreateFlag.OVERWRITE),
+ bufferSize,
+ replication,
+ blockSize,
+ progress);
+ }
+
+ @Override
+ public FSDataOutputStream append(final Path path, final int i, final Progressable progressable)
+ throws IOException {
+ throw new UnsupportedOperationException(path.toString());
+ }
+
+ @Override
+ public boolean rename(final Path path, final Path path1) throws IOException {
+ LOG.trace("rename({}, {})", path, path1);
+ throw new UnsupportedOperationException(path.toString());
+ }
+
+ @Override
+ public boolean delete(final Path hadoopPath, final boolean recursive) throws IOException {
+ LOG.trace("delete({}, {})", hadoopPath, recursive);
+ checkArgument(hadoopPath != null, "hadoopPath must not be null");
+
+ checkOpen();
+
+ URI gcsPath = getGcsPath(hadoopPath);
+ try {
+ getGcsFs().delete(gcsPath, recursive);
+ } catch (DirectoryNotEmptyException e) {
+ throw e;
+ } catch (IOException e) {
+ if (ApiErrorExtractor.INSTANCE.requestFailure(e)) {
+ throw e;
+ }
+ LOG.trace("delete(hadoopPath: {}, recursive: {}): false [failed]", hadoopPath, recursive, e);
+ return false;
+ }
+
+ LOG.trace("delete(hadoopPath: %s, recursive: %b): true", hadoopPath, recursive);
+ return true;
+ }
+
+ @Override
+ public FileStatus[] listStatus(final Path hadoopPath) throws IOException {
+ checkArgument(hadoopPath != null, "hadoopPath must not be null");
+
+ checkOpen();
+
+ LOG.trace("listStatus(hadoopPath: {})", hadoopPath);
+
+ URI gcsPath = getGcsPath(hadoopPath);
+ List status;
+
+ try {
+ List fileInfos = getGcsFs().listFileInfo(gcsPath, ListFileOptions.OBJECTFIELDS);
+ status = new ArrayList<>(fileInfos.size());
+ String userName = getUgiUserName();
+ for (FileInfo fileInfo : fileInfos) {
+ status.add(getFileStatus(fileInfo, userName));
+ }
+ } catch (FileNotFoundException fnfe) {
+ throw (FileNotFoundException)
+ new FileNotFoundException(
+ String.format(
+ "listStatus(hadoopPath: %s): '%s' does not exist.",
+ hadoopPath, gcsPath))
+ .initCause(fnfe);
+ }
+
+ return status.toArray(new FileStatus[0]);
+ }
+
+ /**
+ * Overridden to make root its own parent. This is POSIX compliant, but more importantly guards
+ * against poor directory accounting in the PathData class of Hadoop 2's FsShell.
+ */
+ @Override
+ public Path makeQualified(final Path path) {
+ Path qualifiedPath = super.makeQualified(path);
+
+ URI uri = qualifiedPath.toUri();
+
+ checkState("".equals(uri.getPath()) || qualifiedPath.isAbsolute(),
+ "Path '{}' must be fully qualified.", qualifiedPath);
+
+ Path result;
+ String upath = uri.getPath();
+
+ // Strip initial '..'s to make root is its own parent.
+ int i = 0;
+ while (upath.startsWith("/../", i)) {
+ // Leave a preceding slash, so path is still absolute.
+ i += 3;
+ }
+ if (i == upath.length() || upath.substring(i).equals("/..")) {
+ // Allow a Path of gs://someBucket to map to gs://someBucket/
+ result = new Path(uri.getScheme(), uri.getAuthority(), "/");
+ } else if (i == 0) {
+ result = qualifiedPath;
+ } else {
+ result = new Path(uri.getScheme(), uri.getAuthority(), upath.substring(i));
+ }
+
+ LOG.trace("makeQualified(path: {}): {}", path, result);
+ return result;
+ }
+
+ /**
+ * Returns a URI of the root of this FileSystem.
+ */
+ @Override
+ public URI getUri() {
+ return fsRoot.toUri();
+ }
+
+ /**
+ * The default port is listed as -1 as an indication that ports are not used.
+ */
+ @Override
+ protected int getDefaultPort() {
+ int result = -1;
+ LOG.trace("getDefaultPort(): %d", result);
+ return result;
+ }
+
+ @Override
+ public boolean hasPathCapability(final Path path, final String capability) {
+ checkNotNull(path, "path must not be null");
+ checkArgument(!isNullOrEmpty(capability), "capability must not be null or empty string for {}",
+ path);
+ switch (Ascii.toLowerCase(capability)) {
+ case CommonPathCapabilities.FS_APPEND:
+ case CommonPathCapabilities.FS_CONCAT:
+ return false;
+ default:
+ return false;
+ }
+ }
+
+ /**
+ * Gets the current working directory.
+ *
+ * @return The current working directory.
+ */
+ @Override
+ public Path getWorkingDirectory() {
+ LOG.trace("getWorkingDirectory(): {}", workingDirectory);
+ return workingDirectory;
+ }
+
+ @Override
+ public boolean mkdirs(final Path hadoopPath, final FsPermission permission) throws IOException {
+ checkArgument(hadoopPath != null, "hadoopPath must not be null");
+
+ LOG.trace(
+ "mkdirs(hadoopPath: {}, permission: {}): true", hadoopPath, permission);
+
+ checkOpen();
+
+ URI gcsPath = getGcsPath(hadoopPath);
+ try {
+ getGcsFs().mkdirs(gcsPath);
+ } catch (java.nio.file.FileAlreadyExistsException faee) {
+ // Need to convert to the Hadoop flavor of FileAlreadyExistsException.
+ throw (FileAlreadyExistsException)
+ new FileAlreadyExistsException(
+ String.format(
+ "mkdirs(hadoopPath: %s, permission: %s): failed",
+ hadoopPath, permission))
+ .initCause(faee);
+ }
+
+ return true;
+ }
+
+ @Override
+ public FileStatus getFileStatus(final Path path) throws IOException {
+ checkArgument(path != null, "path must not be null");
+
+ checkOpen();
+
+ URI gcsPath = getGcsPath(path);
+
+ FileInfo fileInfo = getGcsFs().getFileInfo(gcsPath);
+ if (!fileInfo.exists()) {
+ throw new FileNotFoundException(
+ String.format(
+ "%s not found: %s", fileInfo.isDirectory() ? "Directory" : "File", path));
+ }
+ String userName = getUgiUserName();
+ return getFileStatus(fileInfo, userName);
+ }
+
+ /**
+ * Returns home directory of the current user.
+ *
+ * Note: This directory is only used for Hadoop purposes. It is not the same as a user's OS
+ * home directory.
+ */
+ @Override
+ public Path getHomeDirectory() {
+ Path result = new Path(fsRoot, "user/" + System.getProperty("user.name"));
+ LOG.trace("getHomeDirectory(): {}", result);
+ return result;
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ *
Returns the service if delegation tokens are configured, otherwise, null.
+ */
+ @Override
+ public String getCanonicalServiceName() {
+ // TODO: Add delegation token support
+ return null;
+ }
+
+ /**
+ * Gets GCS FS instance.
+ */
+ GoogleCloudStorageFileSystem getGcsFs() {
+ return gcsFs;
+ }
+
+ /**
+ * Assert that the FileSystem has been initialized and not close()d.
+ */
+ private void checkOpen() throws IOException {
+ if (isClosed) {
+ throw new IOException("GoogleHadoopFileSystem has been closed or not initialized.");
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ LOG.trace("close()");
+ if (isClosed) {
+ return;
+ }
+
+ super.close();
+
+ getGcsFs().close();
+
+ this.isClosed = true;
+ }
+
+ @Override
+ public long getUsed() throws IOException {
+ long result = super.getUsed();
+ LOG.trace("getUsed(): {}", result);
+ return result;
+ }
+
+// @Override
+// public long getDefaultBlockSize() {
+// LOG.trace("getDefaultBlockSize(): {}", defaultBlockSize);
+// return defaultBlockSize;
+// }
+
+ @Override
+ public void setWorkingDirectory(final Path hadoopPath) {
+ checkArgument(hadoopPath != null, "hadoopPath must not be null");
+ URI gcsPath = UriPaths.toDirectory(getGcsPath(hadoopPath));
+ workingDirectory = getHadoopPath(gcsPath);
+ LOG.trace("setWorkingDirectory(hadoopPath: {}): {}", hadoopPath, workingDirectory);
+ }
+
+
+ private static String getUgiUserName() throws IOException {
+ UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+ return ugi.getShortUserName();
+ }
+
+ private FileStatus getFileStatus(FileInfo fileInfo, String userName) {
+ checkNotNull(fileInfo, "fileInfo should not be null");
+ // GCS does not provide modification time. It only provides creation time.
+ // It works for objects because they are immutable once created.
+ FileStatus status = new FileStatus(
+ fileInfo.getSize(),
+ fileInfo.isDirectory(),
+ REPLICATION_FACTOR_DEFAULT,
+ defaultBlockSize,
+ fileInfo.getModificationTime(),
+ fileInfo.getModificationTime(),
+ reportedPermissions,
+ userName,
+ userName,
+ getHadoopPath(fileInfo.getPath()));
+ LOG.trace("FileStatus(path: {}, userName: {}): {}", fileInfo.getPath(), userName, status);
+ return status;
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleHadoopFileSystemConfiguration.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleHadoopFileSystemConfiguration.java
new file mode 100644
index 0000000000000..a480a72e60bd2
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleHadoopFileSystemConfiguration.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import static java.lang.Math.toIntExact;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * This class provides a configuration for the {@link GoogleHadoopFileSystem} implementations.
+ */
+class GoogleHadoopFileSystemConfiguration {
+ /**
+ * Configuration key for default block size of a file.
+ *
+ *
Note that this is the size that is reported to Hadoop FS clients. It does not modify the
+ * actual block size of an underlying GCS object, because GCS JSON API does not allow modifying or
+ * querying the value. Modifying this value allows one to control how many mappers are used to
+ * process a given file.
+ */
+ static final HadoopConfigurationProperty BLOCK_SIZE =
+ new HadoopConfigurationProperty<>("fs.gs.block.size", 64 * 1024 * 1024L);
+
+ /**
+ * Configuration key for GCS project ID. Default value: none
+ */
+ static final HadoopConfigurationProperty GCS_PROJECT_ID =
+ new HadoopConfigurationProperty<>("fs.gs.project.id");
+
+ /**
+ * Configuration key for initial working directory of a GHFS instance. Default value: '/'
+ */
+ static final HadoopConfigurationProperty GCS_WORKING_DIRECTORY =
+ new HadoopConfigurationProperty<>("fs.gs.working.dir", "/");
+
+ /**
+ * Configuration key for setting write buffer size.
+ */
+ static final HadoopConfigurationProperty GCS_OUTPUT_STREAM_BUFFER_SIZE =
+ new HadoopConfigurationProperty<>("fs.gs.outputstream.buffer.size", 8L * 1024 * 1024);
+
+ private final String workingDirectory;
+ private final String projectId;
+
+ public int getOutStreamBufferSize() {
+ return outStreamBufferSize;
+ }
+
+ private final int outStreamBufferSize;
+
+ GoogleHadoopFileSystemConfiguration(Configuration config) {
+ this.workingDirectory = GCS_WORKING_DIRECTORY.get(config, config::get);
+ this.outStreamBufferSize =
+ toIntExact(GCS_OUTPUT_STREAM_BUFFER_SIZE.get(config, config::getLongBytes));
+ this.projectId = GCS_PROJECT_ID.get(config, config::get);
+ }
+
+ public String getWorkingDirectory() {
+ return this.workingDirectory;
+ }
+
+ String getProjectId() {
+ return this.projectId;
+ }
+
+ public long getMaxListItemsPerCall() {
+ return 5000L; //TODO: Make this configurable
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleHadoopOutputStream.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleHadoopOutputStream.java
new file mode 100644
index 0000000000000..747d9f001c517
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleHadoopOutputStream.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.nio.channels.Channels;
+import java.nio.channels.ClosedChannelException;
+import java.nio.channels.WritableByteChannel;
+import javax.annotation.Nonnull;
+
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystem;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class GoogleHadoopOutputStream extends OutputStream {
+ public static final Logger LOG = LoggerFactory.getLogger(StorageResourceId.class);
+
+ private final GoogleHadoopFileSystem ghfs;
+
+ // Path of the file to write to.
+ private final URI dstGcsPath;
+
+ private OutputStream outputStream;
+
+ // Statistics tracker provided by the parent GoogleHadoopFileSystem for recording
+ // numbers of bytes written.
+ private final FileSystem.Statistics statistics;
+
+ /**
+ * Constructs an instance of GoogleHadoopOutputStream object.
+ *
+ * @param ghfs Instance of {@link GoogleHadoopFileSystem}.
+ * @param dstGcsPath Path of the file to write to.
+ * @param statistics File system statistics object.
+ * @param createFileOptions options for file creation
+ * @throws IOException if an IO error occurs.
+ */
+ GoogleHadoopOutputStream(GoogleHadoopFileSystem ghfs, URI dstGcsPath,
+ CreateOptions createFileOptions, FileSystem.Statistics statistics) throws IOException {
+ LOG.trace("GoogleHadoopOutputStream(gcsPath: {}, createFileOptions: {})", dstGcsPath,
+ createFileOptions);
+ this.ghfs = ghfs;
+ this.dstGcsPath = dstGcsPath;
+ this.statistics = statistics;
+
+ this.outputStream = createOutputStream(ghfs.getGcsFs(), dstGcsPath, createFileOptions,
+ ghfs.getFileSystemConfiguration());
+ }
+
+ private static OutputStream createOutputStream(GoogleCloudStorageFileSystem gcsfs, URI gcsPath,
+ CreateOptions options, GoogleHadoopFileSystemConfiguration fileSystemConfiguration)
+ throws IOException {
+ WritableByteChannel channel;
+ try {
+ channel = gcsfs.create(gcsPath, options);
+ } catch (java.nio.file.FileAlreadyExistsException e) {
+
+ throw (FileAlreadyExistsException) new FileAlreadyExistsException(
+ String.format("'%s' already exists", gcsPath)).initCause(e);
+ }
+ OutputStream outputStream = Channels.newOutputStream(channel);
+ int bufferSize = fileSystemConfiguration.getOutStreamBufferSize();
+ return bufferSize > 0 ? new BufferedOutputStream(outputStream, bufferSize) : outputStream;
+ }
+
+ @Override
+ public void write(int b) throws IOException {
+ throwIfNotOpen();
+ outputStream.write(b);
+ statistics.incrementBytesWritten(1);
+ statistics.incrementWriteOps(1);
+ }
+
+ @Override
+ public void write(@Nonnull byte[] b, int offset, int len) throws IOException {
+ throwIfNotOpen();
+ outputStream.write(b, offset, len);
+ statistics.incrementBytesWritten(len);
+ statistics.incrementWriteOps(1);
+ }
+
+ @Override
+ public void close() throws IOException {
+ LOG.trace("close(): final destination: {}", dstGcsPath);
+
+ if (outputStream == null) {
+ LOG.trace("close(): Ignoring; stream already closed.");
+ return;
+ }
+
+ try {
+ outputStream.close();
+ } finally {
+ outputStream = null;
+ }
+ }
+
+ private void throwIfNotOpen() throws IOException {
+ if (outputStream == null) {
+ throw new ClosedChannelException();
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/HadoopConfigurationProperty.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/HadoopConfigurationProperty.java
new file mode 100644
index 0000000000000..9360290a09c5b
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/HadoopConfigurationProperty.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.function.BiFunction;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Hadoop configuration property.
+ */
+class HadoopConfigurationProperty {
+ private static final Logger LOG = LoggerFactory.getLogger(HadoopConfigurationProperty.class);
+
+ private final String key;
+ private final List deprecatedKeys;
+ private final T defaultValue;
+
+ private List keyPrefixes = ImmutableList.of("");
+
+ HadoopConfigurationProperty(String key) {
+ this(key, null);
+ }
+
+ HadoopConfigurationProperty(String key, T defaultValue, String... deprecatedKeys) {
+ this.key = key;
+ this.deprecatedKeys =
+ deprecatedKeys == null ? ImmutableList.of() : ImmutableList.copyOf(deprecatedKeys);
+ this.defaultValue = defaultValue;
+ }
+
+ String getKey() {
+ return key;
+ }
+
+ T getDefault() {
+ return defaultValue;
+ }
+
+ T get(Configuration config, BiFunction getterFn) {
+ String lookupKey = getLookupKey(config, key, (c, k) -> c.get(k) != null);
+ return logProperty(lookupKey, getterFn.apply(lookupKey, defaultValue));
+ }
+
+ private String getLookupKey(Configuration config, String lookupKey,
+ BiFunction checkFn) {
+ for (String prefix : keyPrefixes) {
+ String prefixedKey = prefix + lookupKey;
+ if (checkFn.apply(config, prefixedKey)) {
+ return prefixedKey;
+ }
+ for (String deprecatedKey : deprecatedKeys) {
+ String prefixedDeprecatedKey = prefix + deprecatedKey;
+ if (checkFn.apply(config, prefixedDeprecatedKey)) {
+ LOG.warn("Using deprecated key '{}', use '{}' key instead.", prefixedDeprecatedKey,
+ prefixedKey);
+ return prefixedDeprecatedKey;
+ }
+ }
+ }
+ return keyPrefixes.get(0) + lookupKey;
+ }
+
+ private static S logProperty(String key, S value) {
+ LOG.trace("{} = {}", key, value);
+ return value;
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/IoExceptionHelper.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/IoExceptionHelper.java
new file mode 100644
index 0000000000000..c68a6cac1a1c8
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/IoExceptionHelper.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+import javax.net.ssl.SSLException;
+
+/**
+ * Translates exceptions from API calls into higher-level meaning, while allowing injectability for
+ * testing how API errors are handled.
+ */
+public final class IoExceptionHelper {
+
+ private IoExceptionHelper() {}
+
+ /**
+ * Determines if a given {@link Throwable} is caused by an IO error.
+ *
+ * Recursively checks {@code getCause()} if outer exception isn't an instance of the correct
+ * class.
+ *
+ * @param throwable The {@link Throwable} to check.
+ * @return True if the {@link Throwable} is a result of an IO error.
+ */
+ public static boolean isIoError(Throwable throwable) {
+ if (throwable instanceof IOException || throwable instanceof IOError) {
+ return true;
+ }
+ Throwable cause = throwable.getCause();
+ return cause != null && isIoError(cause);
+ }
+
+ /**
+ * Determines if a given {@link Throwable} is caused by a socket error.
+ *
+ *
Recursively checks {@code getCause()} if outer exception isn't an instance of the correct
+ * class.
+ *
+ * @param throwable The {@link Throwable} to check.
+ * @return True if the {@link Throwable} is a result of a socket error.
+ */
+ public static boolean isSocketError(Throwable throwable) {
+ if (throwable instanceof SocketException || throwable instanceof SocketTimeoutException) {
+ return true;
+ }
+ Throwable cause = throwable.getCause();
+ // Subset of SSL exceptions that are caused by IO errors (e.g. SSLHandshakeException due to
+ // unexpected connection closure) is also a socket error.
+ if (throwable instanceof SSLException && cause != null && isIoError(cause)) {
+ return true;
+ }
+ return cause != null && isSocketError(cause);
+ }
+
+ /**
+ * Determines if a given {@link IOException} is caused by a timed out read.
+ *
+ * @param e The {@link IOException} to check.
+ * @return True if the {@link IOException} is a result of a read timeout.
+ */
+ public static boolean isReadTimedOut(IOException e) {
+ return e instanceof SocketTimeoutException && e.getMessage().equalsIgnoreCase("Read timed out");
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/ListFileOptions.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/ListFileOptions.java
new file mode 100644
index 0000000000000..2bc74c6fc2190
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/ListFileOptions.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import javax.annotation.Nonnull;
+
+final class ListFileOptions {
+ static final ListFileOptions OBJECTFIELDS = new ListFileOptions("bucket,name,size,updated");
+ private final String fields;
+
+ private ListFileOptions(@Nonnull String fields) {
+ this.fields = fields;
+ }
+
+ String getFields() {
+ return fields;
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/ListObjectOptions.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/ListObjectOptions.java
new file mode 100644
index 0000000000000..60ec409b5c7d8
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/ListObjectOptions.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import javax.annotation.Nullable;
+
+import static org.apache.hadoop.fs.gs.Constants.PATH_DELIMITER;
+
+/** Options that can be specified when listing objects in the {@link GoogleCloudStorage}. */
+final class ListObjectOptions {
+
+ /** List all objects in the directory. */
+ public static final ListObjectOptions DEFAULT = new Builder().build();
+
+ /** List all objects with the prefix. */
+ public static final ListObjectOptions DEFAULT_FLAT_LIST =
+ DEFAULT.builder().setDelimiter(null).build();
+
+ Builder builder() {
+ Builder result = new Builder();
+ result.fields = fields;
+ result.delimiter = delimiter;
+ result.maxResults = maxResult;
+ result.includePrefix = includePrefix;
+
+ return result;
+ }
+
+ private final String delimiter;
+ private final boolean includePrefix;
+ private final long maxResult;
+ private final String fields;
+
+ private ListObjectOptions(Builder builder) {
+ this.delimiter = builder.delimiter;
+ this.includePrefix = builder.includePrefix;
+ this.maxResult = builder.maxResults;
+ this.fields = builder.fields;
+ }
+
+ /** Delimiter to use (typically {@code /}), otherwise {@code null}. */
+ @Nullable
+ String getDelimiter() {
+ return delimiter;
+ }
+
+ /** Whether to include prefix object in the result. */
+ boolean isIncludePrefix() {
+ return includePrefix;
+ }
+
+ /** Maximum number of results to return, unlimited if negative or zero. */
+ long getMaxResults() {
+ return maxResult;
+ }
+
+ /**
+ * Comma separated list of object fields to include in the list response.
+ *
+ *
See
+ * object resource for reference.
+ */
+ @Nullable
+ String getFields() {
+ return fields;
+ }
+
+ static class Builder {
+ private static final int MAX_RESULTS_UNLIMITED = -1;
+
+ static final String OBJECT_FIELDS =
+ String.join(
+ /* delimiter= */ ",",
+ "bucket",
+ "name",
+ "timeCreated",
+ "updated",
+ "generation",
+ "metageneration",
+ "size",
+ "contentType",
+ "contentEncoding",
+ "md5Hash",
+ "crc32c",
+ "metadata");
+
+ private String delimiter;
+ private boolean includePrefix;
+
+ private long maxResults;
+
+ private String fields;
+
+ Builder() {
+ this.delimiter = PATH_DELIMITER;
+ this.includePrefix = false;
+ this.maxResults = MAX_RESULTS_UNLIMITED;
+ this.fields = OBJECT_FIELDS;
+ }
+ public Builder setDelimiter(String d) {
+ this.delimiter = d;
+ return this;
+ }
+
+ public Builder setIncludePrefix(boolean value) {
+ this.includePrefix = value;
+ return this;
+ }
+
+ public Builder setMaxResults(long mr) {
+ this.maxResults = mr;
+ return this;
+ }
+
+ public Builder setFields(String f) {
+ this.fields = f;
+ return this;
+ }
+
+ public ListObjectOptions build() {
+ return new ListObjectOptions(this);
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/StorageResourceId.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/StorageResourceId.java
new file mode 100644
index 0000000000000..5935564feedfa
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/StorageResourceId.java
@@ -0,0 +1,328 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.thirdparty.com.google.common.base.Strings.isNullOrEmpty;
+import static org.apache.hadoop.fs.gs.Constants.SCHEME;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Data struct representing either a GCS StorageObject, a GCS Bucket or the GCS root (gs://). If
+ * both bucketName and objectName are null, the StorageResourceId refers to GCS root (gs://). If
+ * bucketName is non-null, and objectName is null, then this refers to a GCS Bucket. Otherwise, if
+ * bucketName and objectName are both non-null, this refers to a GCS StorageObject.
+ */
+class StorageResourceId {
+
+ private static final Logger LOG = LoggerFactory.getLogger(StorageResourceId.class);
+
+ // The generationId used to denote "unknown"; if given to a method expecting generationId
+ // constraints, the method may perform extra low-level GETs to determine an existing
+ // generationId
+ // if idempotency constraints require doing so.
+ static final long UNKNOWN_GENERATION_ID = -1L;
+
+ // Pattern that parses out bucket and object names.
+ // Given 'gs://foo-bucket/foo/bar/baz', matcher.group(x) will return:
+ // 0 = gs://foo-bucket/foo/bar/baz
+ // 1 = foo-bucket/foo/bar/baz
+ // 2 = foo-bucket
+ // 3 = /foo/bar/baz
+ // 4 = foo/bar/baz
+ // Groups 2 and 4 can be used to create an instance.
+ private static final Pattern GCS_PATH_PATTERN = Pattern.compile("gs://(([^/]+)(/(.+)?)?)?");
+
+ // The singleton instance identifying the GCS root (gs://). Both getObjectName() and
+ // getBucketName() will return null.
+ static final StorageResourceId ROOT = new StorageResourceId();
+
+ // Bucket name of this storage resource to be used with the Google Cloud Storage API.
+ private final String bucketName;
+
+ // Object name of this storage resource to be used with the Google Cloud Storage API.
+ private final String objectName;
+
+ // Human-readable String to be returned by toString(); kept as 'final' member for efficiency.
+ private final String stringPath;
+
+ // The generationId to be used with precondition checks when using this StorageResourceId
+ // as an identifier for mutation requests.
+ private final long generationId;
+
+ /**
+ * Constructor for a StorageResourceId that refers to the GCS root (gs://). Private because all
+ * external users should just use the singleton StorageResourceId.ROOT.
+ */
+ private StorageResourceId() {
+ this.bucketName = null;
+ this.objectName = null;
+ this.stringPath = StringPaths.fromComponents(bucketName, objectName);
+ this.generationId = UNKNOWN_GENERATION_ID;
+ }
+
+ /**
+ * Constructor for a StorageResourceId representing a Bucket; {@code getObjectName()} will return
+ * null for a StorageResourceId that represents a Bucket.
+ *
+ * @param bucketName The bucket name of the resource. Must be non-empty and non-null.
+ */
+ StorageResourceId(String bucketName) {
+ checkArgument(!isNullOrEmpty(bucketName), "bucketName must not be null or empty");
+
+ this.bucketName = bucketName;
+ this.objectName = null;
+ this.stringPath = StringPaths.fromComponents(bucketName, objectName);
+ this.generationId = UNKNOWN_GENERATION_ID;
+ }
+
+ /**
+ * Constructor for a StorageResourceId representing a full StorageObject, including bucketName and
+ * objectName.
+ *
+ * @param bucketName The bucket name of the resource. Must be non-empty and non-null.
+ * @param objectName The object name of the resource. Must be non-empty and non-null.
+ */
+ StorageResourceId(String bucketName, String objectName) {
+ checkArgument(!isNullOrEmpty(bucketName), "bucketName must not be null or empty");
+ checkArgument(!isNullOrEmpty(objectName), "objectName must not be null or empty");
+
+ this.bucketName = bucketName;
+ this.objectName = objectName;
+ this.stringPath = StringPaths.fromComponents(bucketName, objectName);
+ this.generationId = UNKNOWN_GENERATION_ID;
+ }
+
+ /**
+ * Constructor for a StorageResourceId representing a full StorageObject, including bucketName and
+ * objectName.
+ *
+ * @param bucketName The bucket name of the resource. Must be non-empty and non-null.
+ * @param objectName The object name of the resource. Must be non-empty and non-null.
+ * @param generationId The generationId to be used with precondition checks when using this
+ * StorageResourceId as an identifier for mutation requests.
+ */
+ StorageResourceId(String bucketName, String objectName, long generationId) {
+ checkArgument(!isNullOrEmpty(bucketName), "bucketName must not be null or empty");
+ checkArgument(!isNullOrEmpty(objectName), "objectName must not be null or empty");
+
+ this.bucketName = bucketName;
+ this.objectName = objectName;
+ this.stringPath = StringPaths.fromComponents(bucketName, objectName);
+ this.generationId = generationId;
+ }
+
+ /**
+ * Constructor for a StorageResourceId representing a full StorageObject, including bucketName and
+ * objectName.
+ *
+ * @param bucketName The bucket name of the resource. Must be non-empty and non-null.
+ * @param generationId The generationId to be used with precondition checks when using this
+ * StorageResourceId as an identifier for mutation requests.
+ */
+ StorageResourceId(String bucketName, long generationId) {
+ checkArgument(!isNullOrEmpty(bucketName), "bucketName must not be null or empty");
+ this.bucketName = bucketName;
+ this.objectName = null;
+ this.stringPath = StringPaths.fromComponents(bucketName, objectName);
+ this.generationId = generationId;
+ }
+
+ /**
+ * Returns true if this StorageResourceId represents a GCS StorageObject; if true, both {@code
+ * getBucketName} and {@code getObjectName} will be non-empty and non-null.
+ */
+ boolean isStorageObject() {
+ return bucketName != null && objectName != null;
+ }
+
+ /**
+ * Returns true if this StorageResourceId represents a GCS Bucket; if true, then {@code
+ * getObjectName} will return null.
+ */
+ boolean isBucket() {
+ return bucketName != null && objectName == null;
+ }
+
+ /**
+ * Returns true if this StorageResourceId represents the GCS root (gs://); if true, then both
+ * {@code getBucketName} and {@code getObjectName} will be null.
+ */
+ boolean isRoot() {
+ return bucketName == null && objectName == null;
+ }
+
+ /**
+ * Indicates if this StorageResourceId corresponds to a 'directory'; similar to {@link
+ * FileInfo#isDirectory} except deals entirely with pathnames instead of also checking for
+ * exists() to be true on a corresponding GoogleCloudStorageItemInfo.
+ */
+ boolean isDirectory() {
+ return isRoot() || isBucket() || StringPaths.isDirectoryPath(objectName);
+ }
+
+ /**
+ * Gets the bucket name component of this resource identifier.
+ */
+ String getBucketName() {
+ return bucketName;
+ }
+
+ /**
+ * Gets the object name component of this resource identifier.
+ */
+ String getObjectName() {
+ return objectName;
+ }
+
+ /**
+ * The generationId to be used with precondition checks when using this StorageResourceId as an
+ * identifier for mutation requests. The generationId is *not* used when determining equals() or
+ * hashCode().
+ */
+ long getGenerationId() {
+ return generationId;
+ }
+
+ /**
+ * Returns true if generationId is not UNKNOWN_GENERATION_ID.
+ */
+ boolean hasGenerationId() {
+ return generationId != UNKNOWN_GENERATION_ID;
+ }
+
+ /**
+ * Returns a string of the form {@code gs:///}.
+ */
+ @Override
+ public String toString() {
+ return stringPath;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof StorageResourceId) {
+ StorageResourceId other = (StorageResourceId) obj;
+ return Objects.equals(bucketName, other.bucketName) && Objects.equals(objectName,
+ other.objectName);
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return stringPath.hashCode();
+ }
+
+ /**
+ * Converts StorageResourceId instance to look like a directory path. If the path already looks
+ * like a directory path then this call is a no-op.
+ *
+ * @return A resourceId with a directory path corresponding to the given resourceId.
+ */
+ StorageResourceId toDirectoryId() {
+ if (isStorageObject() && !StringPaths.isDirectoryPath(getObjectName())) {
+ return new StorageResourceId(getBucketName(), StringPaths.toDirectoryPath(getObjectName()));
+ }
+ return this;
+ }
+
+ /**
+ * Parses {@link StorageResourceId} from specified string.
+ */
+ static StorageResourceId fromStringPath(String path) {
+ return fromStringPath(path, UNKNOWN_GENERATION_ID);
+ }
+
+ /**
+ * Parses {@link StorageResourceId} from specified string and generationId.
+ */
+ static StorageResourceId fromStringPath(String path, long generationId) {
+ checkArgument(path != null, "path must not be null");
+
+ Matcher matcher = GCS_PATH_PATTERN.matcher(path);
+ checkArgument(matcher.matches(), "'%s' is not a valid GCS object name.", path);
+
+ String bucketName = matcher.group(2);
+ String relativePath = matcher.group(4);
+ if (bucketName == null) {
+ checkArgument(generationId == UNKNOWN_GENERATION_ID,
+ "Cannot specify generationId '%s' for root object '%s'", generationId, path);
+ return ROOT;
+ } else if (relativePath != null) {
+ return new StorageResourceId(bucketName, relativePath, generationId);
+ }
+ checkArgument(generationId == UNKNOWN_GENERATION_ID,
+ "Cannot specify generationId '%s' for bucket '%s'", generationId, path);
+ return new StorageResourceId(bucketName);
+ }
+
+ /**
+ * Validates the given URI and if valid, returns the associated StorageResourceId.
+ *
+ * @param path The GCS URI to validate.
+ * @param allowEmptyObjectName If true, a missing object name is not considered invalid.
+ * @return a StorageResourceId that may be the GCS root, a Bucket, or a StorageObject.
+ */
+ static StorageResourceId fromUriPath(URI path, boolean allowEmptyObjectName) {
+ return fromUriPath(path, allowEmptyObjectName, UNKNOWN_GENERATION_ID);
+ }
+
+ /**
+ * Validates the given URI and if valid, returns the associated StorageResourceId.
+ *
+ * @param path The GCS URI to validate.
+ * @param allowEmptyObjectName If true, a missing object name is not considered invalid.
+ * @param generationId The generationId to be used with precondition checks when
+ * using this
+ * @return a StorageResourceId that may be the GCS root, a Bucket, or a StorageObject.
+ */
+ static StorageResourceId fromUriPath(URI path, boolean allowEmptyObjectName,
+ long generationId) {
+ LOG.trace("fromUriPath('{}', {})", path, allowEmptyObjectName);
+ checkNotNull(path);
+
+ if (!SCHEME.equals(path.getScheme())) {
+ throw new IllegalArgumentException(
+ String.format("GCS path supports only '%s' scheme, instead got '%s' from '%s'.", SCHEME,
+ path.getScheme(), path));
+ }
+
+ if (path.equals(GoogleCloudStorageFileSystem.GCSROOT)) {
+ return ROOT;
+ }
+
+ String bucketName = StringPaths.validateBucketName(path.getAuthority());
+ // Note that we're using getPath here instead of rawPath, etc. This is because it is assumed
+ // that the path was properly encoded in getPath (or another similar method):
+ String objectName = StringPaths.validateObjectName(path.getPath(), allowEmptyObjectName);
+
+ return isNullOrEmpty(objectName) ?
+ new StorageResourceId(bucketName, generationId) :
+ new StorageResourceId(bucketName, objectName, generationId);
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/StringPaths.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/StringPaths.java
new file mode 100644
index 0000000000000..80682c3ed2a31
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/StringPaths.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import static org.apache.hadoop.thirdparty.com.google.common.base.Strings.isNullOrEmpty;
+import static org.apache.hadoop.fs.gs.Constants.PATH_DELIMITER;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.CharMatcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility methods for String GCS paths.
+ */
+final class StringPaths {
+
+ public static final Logger LOG = LoggerFactory.getLogger(StorageResourceId.class);
+
+ private StringPaths() {
+ }
+
+ // 14x faster (20ns vs 280ns) than "^[a-z0-9_.-]+$" regex
+ private static final CharMatcher BUCKET_NAME_CHAR_MATCHER = CharMatcher.ascii()
+ .and(CharMatcher.inRange('0', '9').or(CharMatcher.inRange('a', 'z'))
+ .or(CharMatcher.anyOf("_.-")))
+ .precomputed();
+
+ /**
+ * Validate the given bucket name to make sure that it can be used as a part of a file system
+ * path.
+ *
+ * Note: this is not designed to duplicate the exact checks that GCS would perform on the
+ * server side. We make some checks that are relevant to using GCS as a file system.
+ *
+ * @param bucketName Bucket name to check.
+ */
+ static String validateBucketName(String bucketName) {
+ // If the name ends with '/', remove it.
+ bucketName = toFilePath(bucketName);
+
+ if (isNullOrEmpty(bucketName)) {
+ throw new IllegalArgumentException("GCS bucket name cannot be empty.");
+ }
+
+ if (!BUCKET_NAME_CHAR_MATCHER.matchesAllOf(bucketName)) {
+ throw new IllegalArgumentException(String.format(
+ "Invalid GCS bucket name '%s': bucket name must contain only 'a-z0-9_.-' characters.",
+ bucketName));
+ }
+
+ return bucketName;
+ }
+
+ /**
+ * Validate the given object name to make sure that it can be used as a part of a file system
+ * path.
+ *
+ *
Note: this is not designed to duplicate the exact checks that GCS would perform on the
+ * server side. We make some checks that are relevant to using GCS as a file system.
+ *
+ * @param objectName Object name to check.
+ * @param allowEmptyObjectName If true, a missing object name is not considered invalid.
+ */
+ static String validateObjectName(String objectName, boolean allowEmptyObjectName) {
+ LOG.trace("validateObjectName('{}', {})", objectName, allowEmptyObjectName);
+
+ if (isNullOrEmpty(objectName) || objectName.equals(PATH_DELIMITER)) {
+ if (allowEmptyObjectName) {
+ objectName = "";
+ } else {
+ throw new IllegalArgumentException(String.format(
+ "GCS path must include non-empty object name [objectName='%s',"
+ + " allowEmptyObjectName=%s]", objectName, allowEmptyObjectName));
+ }
+ }
+
+ // We want objectName to look like a traditional file system path,
+ // therefore, disallow objectName with consecutive '/' chars.
+ for (int i = 0; i < (objectName.length() - 1); i++) {
+ if (objectName.charAt(i) == '/' && objectName.charAt(i + 1) == '/') {
+ throw new IllegalArgumentException(
+ String.format("GCS path must not have consecutive '/' characters: '%s'", objectName));
+ }
+ }
+
+ // Remove leading '/' if it exists.
+ if (objectName.startsWith(PATH_DELIMITER)) {
+ objectName = objectName.substring(1);
+ }
+
+ LOG.trace("validateObjectName -> '{}'", objectName);
+ return objectName;
+ }
+
+ /**
+ * Helper for standardizing the way various human-readable messages in logs/exceptions that refer
+ * to a bucket/object pair.
+ */
+ public static String fromComponents(String bucketName, String objectName) {
+ if (bucketName == null && objectName != null) {
+ throw new IllegalArgumentException(
+ String.format("Invalid bucketName/objectName pair: gs://%s/%s", "", objectName));
+ }
+ // TODO(user): Unify this method with other methods that convert bucketName/objectName
+ // to a URI; maybe use the single slash for compatibility.
+ StringBuilder result = new StringBuilder("gs://");
+ if (bucketName != null) {
+ result.append(bucketName);
+ }
+ if (objectName != null) {
+ result.append('/').append(objectName);
+ }
+ return result.toString();
+ }
+
+ /**
+ * Indicates whether the given object name looks like a directory path.
+ *
+ * @param path Name of the object to inspect.
+ * @return Whether the given object name looks like a directory path.
+ */
+ public static boolean isDirectoryPath(String path) {
+ return !isNullOrEmpty(path) && path.endsWith(PATH_DELIMITER);
+ }
+
+ /**
+ * Converts the given object name to look like a file path. If the object name already looks like
+ * a file path then this call is a no-op.
+ *
+ * If the object name is null or empty, it is returned as-is.
+ *
+ * @param path Name of the object to inspect.
+ * @return File path for the given path.
+ */
+ public static String toFilePath(String path) {
+ return !isNullOrEmpty(path) && isDirectoryPath(path) ?
+ path.substring(0, path.length() - 1) :
+ path;
+ }
+
+ /**
+ * Converts the given object name to look like a directory path. If the object name already looks
+ * like a directory path then this call is a no-op.
+ *
+ *
If the object name is null or empty, it is returned as-is.
+ *
+ * @param path Name of the object to inspect.
+ * @return Directory path for the given path.
+ */
+ static String toDirectoryPath(String path) {
+ return isNullOrEmpty(path) || isDirectoryPath(path) ? path : path + PATH_DELIMITER;
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/UriPaths.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/UriPaths.java
new file mode 100644
index 0000000000000..30e13cb33cbf1
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/UriPaths.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.fs.gs.Constants.PATH_DELIMITER;
+import static org.apache.hadoop.fs.gs.Constants.SCHEME;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Utility methods for URI GCS paths.
+ */
+final class UriPaths {
+
+ private UriPaths() {
+ }
+
+ /**
+ * Converts the given path to look like a directory path. If the path already looks like a
+ * directory path then this call is a no-op.
+ *
+ * @param path Path to convert.
+ * @return Directory path for the given path.
+ */
+ public static URI toDirectory(URI path) {
+ StorageResourceId resourceId = StorageResourceId.fromUriPath(path, true);
+
+ if (resourceId.isStorageObject() && !resourceId.isDirectory()) {
+ resourceId = resourceId.toDirectoryId();
+ path = fromResourceId(resourceId, /* allowEmptyObjectName= */ false);
+ }
+ return path;
+ }
+
+ /**
+ * Gets the parent directory of the given path.
+ *
+ * @param path Path to convert.
+ * @return Path of parent directory of the given item or null for root path.
+ */
+ public static URI getParentPath(URI path) {
+ checkNotNull(path);
+
+ // Root path has no parent.
+ if (path.equals(GoogleCloudStorageFileSystem.GCSROOT)) {
+ return null;
+ }
+
+ StorageResourceId resourceId = StorageResourceId.fromUriPath(path, true);
+
+ if (resourceId.isBucket()) {
+ return GoogleCloudStorageFileSystem.GCSROOT;
+ }
+
+ String objectName = resourceId.getObjectName();
+ int index = StringPaths.isDirectoryPath(objectName) ?
+ objectName.lastIndexOf(PATH_DELIMITER, objectName.length() - 2) :
+ objectName.lastIndexOf(PATH_DELIMITER);
+ return index < 0 ?
+ fromStringPathComponents(resourceId.getBucketName(), /* objectName= */
+ null, /* allowEmptyObjectName= */ true) :
+ fromStringPathComponents(resourceId.getBucketName(), objectName.substring(0, index + 1),
+ /* allowEmptyObjectName= */ false);
+ }
+
+ /**
+ * Constructs and returns full path for the given bucket and object names.
+ */
+ public static URI fromResourceId(StorageResourceId resourceId, boolean allowEmptyObjectName) {
+ return fromStringPathComponents(resourceId.getBucketName(), resourceId.getObjectName(),
+ allowEmptyObjectName);
+ }
+
+ /**
+ * Constructs and returns full path for the given bucket and object names.
+ */
+ public static URI fromStringPathComponents(String bucketName, String objectName,
+ boolean allowEmptyObjectName) {
+ if (allowEmptyObjectName && bucketName == null && objectName == null) {
+ return GoogleCloudStorageFileSystem.GCSROOT;
+ }
+
+ String authority = StringPaths.validateBucketName(bucketName);
+ String path = PATH_DELIMITER + StringPaths.validateObjectName(objectName, allowEmptyObjectName);
+
+ try {
+ return new URI(SCHEME, authority, path,
+ /* query= */ null,
+ /* fragment= */ null);
+ } catch (URISyntaxException e) {
+ throw new IllegalArgumentException(
+ String.format("Invalid bucket name (%s) or object name (%s)", bucketName, objectName), e);
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/VerificationAttributes.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/VerificationAttributes.java
new file mode 100644
index 0000000000000..03de0a52e373e
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/VerificationAttributes.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import java.util.Arrays;
+import javax.annotation.Nullable;
+
+/**
+ * GCS provided validation attributes for a single object.
+ */
+class VerificationAttributes {
+ private final byte[] md5hash;
+ private final byte[] crc32c;
+
+ VerificationAttributes(@Nullable byte[] md5hash, @Nullable byte[] crc32c) {
+ this.md5hash = md5hash;
+ this.crc32c = crc32c;
+ }
+
+ /**
+ * MD5 hash of an object, if available.
+ */
+ @Nullable
+ byte[] getMd5hash() {
+ return md5hash;
+ }
+
+ /**
+ * CRC32c checksum of an object, if available.
+ */
+ @Nullable
+ byte[] getCrc32c() {
+ return crc32c;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof VerificationAttributes)) {
+ return false;
+ }
+ VerificationAttributes that = (VerificationAttributes) o;
+ return Arrays.equals(md5hash, that.md5hash) && Arrays.equals(crc32c, that.crc32c);
+ }
+
+ @Override
+ public int hashCode() {
+ int result = Arrays.hashCode(md5hash);
+ result = 31 * result + Arrays.hashCode(crc32c);
+ return result;
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/package-info.java b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/package-info.java
new file mode 100644
index 0000000000000..eedfb7822acd7
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Google Cloud Storage Filesystem.
+ */
+
+package org.apache.hadoop.fs.gs;
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestConfiguration.java b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestConfiguration.java
new file mode 100644
index 0000000000000..f205276d37294
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestConfiguration.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+/** Access to test configurations values. */
+public abstract class TestConfiguration {
+ public static final String GCS_TEST_PROJECT_ID = "GCS_TEST_PROJECT_ID";
+ public static final String GCS_TEST_JSON_KEYFILE = "GCS_TEST_JSON_KEYFILE";
+
+ public static final String GCS_TEST_DIRECT_PATH_PREFERRED = "GCS_TEST_DIRECT_PATH_PREFERRED";
+
+ /** Environment-based test configuration. */
+ public static class EnvironmentBasedTestConfiguration extends TestConfiguration {
+ @Override
+ public String getProjectId() {
+ return System.getenv(GCS_TEST_PROJECT_ID);
+ }
+
+ @Override
+ public String getServiceAccountJsonKeyFile() {
+ return System.getenv(GCS_TEST_JSON_KEYFILE);
+ }
+
+ @Override
+ public boolean isDirectPathPreferred() {
+ String envVar = System.getenv(GCS_TEST_DIRECT_PATH_PREFERRED);
+ // if env variable is not configured default behaviour is to attempt directPath
+ if (envVar == null) {
+ return true;
+ }
+ return Boolean.parseBoolean(envVar);
+ }
+ }
+
+ public static TestConfiguration getInstance() {
+ return LazyHolder.INSTANCE;
+ }
+
+ private static class LazyHolder {
+ private static final TestConfiguration INSTANCE = new EnvironmentBasedTestConfiguration();
+ }
+
+ public abstract String getProjectId();
+
+ public abstract String getServiceAccountJsonKeyFile();
+
+ public abstract boolean isDirectPathPreferred();
+}
diff --git a/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestStorageResourceId.java b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestStorageResourceId.java
new file mode 100644
index 0000000000000..e027c7b40914a
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestStorageResourceId.java
@@ -0,0 +1,287 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import java.net.URI;
+
+import org.junit.Test;
+
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+public class TestStorageResourceId {
+ @Test
+ public void testConstructorInvalid() {
+ assertThrows(IllegalArgumentException.class, () -> {
+ new StorageResourceId(null);
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ new StorageResourceId("");
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ new StorageResourceId(null, null);
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ new StorageResourceId("foo", null);
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ new StorageResourceId("", null);
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ new StorageResourceId(null, null, 0L);
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ new StorageResourceId("foo", null, 0L);
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ new StorageResourceId("", null, 0L);
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ new StorageResourceId(null, 0L);
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ new StorageResourceId("", 0L);
+ });
+ }
+
+ @Test
+ public void testFromStringPathInvalid() {
+ assertThrows(IllegalArgumentException.class, () -> {
+ StorageResourceId.fromStringPath(null);
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ StorageResourceId.fromStringPath("");
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ StorageResourceId.fromStringPath("foo");
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ StorageResourceId.fromStringPath("/foo/bar");
+ });
+
+ assertThrows(IllegalArgumentException.class, () -> {
+ StorageResourceId.fromStringPath("gs:///foo/bar");
+ });
+ }
+
+ @Test
+ public void testConstructor() {
+ String bucketName = "testbucketname";
+ String objectName = "a/b/c.txt";
+
+ verify(new StorageResourceId(bucketName), bucketName,
+ StorageResourceId.UNKNOWN_GENERATION_ID, null, false,
+ true, true, false, false);
+
+ verify(new StorageResourceId(bucketName, objectName), bucketName,
+ StorageResourceId.UNKNOWN_GENERATION_ID, objectName, false,
+ false, false, true, false);
+
+ long genId = System.currentTimeMillis();
+ verify(new StorageResourceId(bucketName, objectName, genId), bucketName,
+ genId, objectName, true,
+ false, false, true, false);
+
+ verify(new StorageResourceId(bucketName, genId), bucketName,
+ genId, null, true,
+ true, true, false, false);
+ }
+
+ @Test
+ public void testEqualsBucket() {
+ StorageResourceId bucket1 = new StorageResourceId("test-bucket");
+ StorageResourceId bucket2 = new StorageResourceId("test-bucket");
+ assertTrue(bucket1.equals(bucket2));
+ assertEquals(bucket1.hashCode(), bucket2.hashCode());
+ }
+
+ @Test
+ public void testEqualsObject() {
+ StorageResourceId obj1 = new StorageResourceId("test-bucket", "test-object");
+ StorageResourceId obj2 = new StorageResourceId("test-bucket", "test-object");
+ assertTrue(obj1.equals(obj2));
+ assertEquals(obj1.hashCode(), obj2.hashCode());
+ }
+
+ @Test
+ public void testEqualsDifferentBucket() {
+ StorageResourceId bucket1 = new StorageResourceId("test-bucket");
+ StorageResourceId bucket2 = new StorageResourceId("other-bucket");
+ assertFalse(bucket1.equals(bucket2));
+ }
+
+ @Test
+ public void testEqualsDifferentObject() {
+ StorageResourceId obj1 = new StorageResourceId("test-bucket", "test-object");
+ StorageResourceId obj2 = new StorageResourceId("test-bucket", "other-object");
+ assertFalse(obj1.equals(obj2));
+ }
+
+ @Test
+ public void testToDirectoryIdFromFile() {
+ StorageResourceId fileId = new StorageResourceId("my-bucket", "path/to/file.txt");
+ StorageResourceId dirId = fileId.toDirectoryId();
+
+ assertNotSame(fileId, dirId); // Should return a new instance
+ assertTrue(dirId.isDirectory());
+ assertEquals("my-bucket", dirId.getBucketName());
+ assertEquals("path/to/file.txt/", dirId.getObjectName());
+ assertEquals(fileId.getGenerationId(), dirId.getGenerationId());
+ }
+
+ @Test
+ public void testToDirectoryIdFromDirectoryObject() {
+ StorageResourceId dirIdOriginal = new StorageResourceId("my-bucket", "path/to/dir/");
+ StorageResourceId dirIdConverted = dirIdOriginal.toDirectoryId();
+
+ assertSame(dirIdOriginal, dirIdConverted); // Should return the same instance
+ assertTrue(dirIdConverted.isDirectory());
+ assertEquals("path/to/dir/", dirIdConverted.getObjectName());
+ }
+
+ @Test
+ public void testToDirectoryIdFromBucket() {
+ StorageResourceId bucketId = new StorageResourceId("my-bucket");
+ StorageResourceId convertedId = bucketId.toDirectoryId();
+ assertSame(bucketId, convertedId);
+ assertTrue(convertedId.isBucket());
+ }
+
+ @Test
+ public void testFromStringPathRoot() {
+ StorageResourceId id = StorageResourceId.fromStringPath("gs://");
+ assertTrue(id.isRoot());
+ }
+
+ @Test
+ public void testFromStringPathBucket() {
+ StorageResourceId id = StorageResourceId.fromStringPath("gs://my-bucket");
+ assertTrue(id.isBucket());
+ assertEquals("my-bucket", id.getBucketName());
+ assertNull(id.getObjectName());
+ assertEquals(StorageResourceId.UNKNOWN_GENERATION_ID, id.getGenerationId());
+ }
+
+ @ParameterizedTest
+ @ValueSource(strings = {
+ "gs://my-bucket/object",
+ "gs://my-bucket/folder/file.txt",
+ "gs://my-bucket/folder/"
+ })
+ public void testFromStringPathObject(String path) {
+ String expectedBucket = path.split("/")[2];
+ String expectedObject =
+ path.substring(path.indexOf(expectedBucket) + expectedBucket.length() + 1);
+
+ StorageResourceId id = StorageResourceId.fromStringPath(path);
+ assertTrue(id.isStorageObject());
+ assertEquals(expectedBucket, id.getBucketName());
+ assertEquals(expectedObject, id.getObjectName());
+ assertEquals(StorageResourceId.UNKNOWN_GENERATION_ID, id.getGenerationId());
+ }
+
+ @Test
+ public void testFromStringPathObjectWithGenerationId() {
+ long genId = 12345L;
+ StorageResourceId id = StorageResourceId.fromStringPath("gs://my-bucket/object.txt", genId);
+ assertTrue(id.isStorageObject());
+ assertEquals("my-bucket", id.getBucketName());
+ assertEquals("object.txt", id.getObjectName());
+ assertEquals(genId, id.getGenerationId());
+ assertTrue(id.hasGenerationId());
+ }
+
+ @Test
+ public void testFromUriPathBucket() throws Exception {
+ URI uri = new URI("gs://my-bucket");
+ StorageResourceId id = StorageResourceId.fromUriPath(uri, true);
+ assertTrue(id.isBucket());
+ assertEquals("my-bucket", id.getBucketName());
+ assertNull(id.getObjectName());
+ }
+
+ @Test
+ public void testFromUriPathObject() throws Exception {
+ URI uri = new URI("gs://my-bucket/path/to/file.txt");
+ StorageResourceId id = StorageResourceId.fromUriPath(uri, false);
+ assertTrue(id.isStorageObject());
+ assertEquals("my-bucket", id.getBucketName());
+ assertEquals("path/to/file.txt", id.getObjectName());
+ }
+
+ @Test
+ public void testFromUriPathObjectWithGenerationId() throws Exception {
+ URI uri = new URI("gs://my-bucket/object.txt");
+ long genId = 54321L;
+ StorageResourceId id = StorageResourceId.fromUriPath(uri, false, genId);
+ assertTrue(id.isStorageObject());
+ assertEquals("my-bucket", id.getBucketName());
+ assertEquals("object.txt", id.getObjectName());
+ assertEquals(genId, id.getGenerationId());
+ assertTrue(id.hasGenerationId());
+ }
+
+ @Test
+ public void testFromUriPathBucketWithGenerationId() throws Exception {
+ assertThrows(IllegalArgumentException.class, () -> {
+ URI uri = new URI("gs://my-bucket");
+ long genId = 54321L;
+ StorageResourceId.fromUriPath(uri, false, genId);
+ });
+ }
+
+ private static void verify(
+ StorageResourceId id,
+ String bucketName,
+ long generationId,
+ String objectName,
+ boolean hasGenerationId,
+ boolean isBucket,
+ boolean isDirectory,
+ boolean isStorageObject,
+ boolean isRoot) {
+ assertEquals(bucketName, id.getBucketName());
+ assertEquals(generationId, id.getGenerationId());
+ assertEquals(objectName, id.getObjectName());
+ assertEquals(hasGenerationId, id.hasGenerationId());
+ assertEquals(isBucket, id.isBucket());
+ assertEquals(isDirectory, id.isDirectory());
+ assertEquals(isStorageObject, id.isStorageObject());
+ assertEquals(isRoot, id.isRoot());
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestStringPaths.java b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestStringPaths.java
new file mode 100644
index 0000000000000..a6b64ff7cffab
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestStringPaths.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import org.junit.Test;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
+public class TestStringPaths {
+ @Test
+ public void testValidateBucketNameValid() {
+ assertEquals("my-bucket", StringPaths.validateBucketName("my-bucket"));
+ assertEquals("my.bucket", StringPaths.validateBucketName("my.bucket"));
+ assertEquals("my_bucket", StringPaths.validateBucketName("my_bucket"));
+ assertEquals("bucket123", StringPaths.validateBucketName("bucket123"));
+ assertEquals("a", StringPaths.validateBucketName("a"));
+ assertEquals("long-bucket-name-with-numbers-123",
+ StringPaths.validateBucketName("long-bucket-name-with-numbers-123"));
+ }
+
+ @Test
+ public void testValidateBucketNameEndsWithSlash() {
+ assertEquals("my-bucket", StringPaths.validateBucketName("my-bucket/"));
+ assertEquals("another-bucket", StringPaths.validateBucketName("another-bucket/"));
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testValidateBucketNameEmpty() {
+ StringPaths.validateBucketName("");
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testValidateBucketNameNull() {
+ StringPaths.validateBucketName(null);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testValidateBucketNameInvalidChars() {
+ StringPaths.validateBucketName("my bucket"); // Space
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testValidateBucketNameInvalidChars2() {
+ StringPaths.validateBucketName("my@bucket"); // @ symbol
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testValidateBucketNameUpperCase() {
+ StringPaths.validateBucketName("MyBucket"); // Uppercase
+ }
+
+ @Test
+ public void testValidateObjectNameValid() {
+ assertEquals("path/to/object",
+ StringPaths.validateObjectName("path/to/object", false));
+ assertEquals("object", StringPaths.validateObjectName("object", false));
+ assertEquals("dir/",
+ StringPaths.validateObjectName("dir/", false)); // Still valid after validation
+ assertEquals("", StringPaths.validateObjectName("/", true)); // Slash becomes empty if allowed
+ assertEquals("", StringPaths.validateObjectName("", true));
+ }
+
+ @Test
+ public void testValidateObjectNameLeadingSlash() {
+ assertEquals("path/to/object", StringPaths.validateObjectName("/path/to/object", false));
+ assertEquals("object", StringPaths.validateObjectName("/object", false));
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testValidateObjectNameEmptyNotAllowed() {
+ StringPaths.validateObjectName("", false);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testValidateObjectNameNullNotAllowed() {
+ StringPaths.validateObjectName(null, false);
+ }
+
+ @Test
+ public void testValidateObjectNameEmptyAllowed() {
+ assertEquals("", StringPaths.validateObjectName("", true));
+ assertEquals("", StringPaths.validateObjectName(null, true));
+ assertEquals("", StringPaths.validateObjectName("/", true)); // Single slash becomes empty
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testValidateObjectNameConsecutiveSlashes() {
+ StringPaths.validateObjectName("path//to/object", false);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testValidateObjectNameConsecutiveSlashesAtStart() {
+ StringPaths.validateObjectName("//path/to/object", false);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testValidateObjectNameConsecutiveSlashesAtEnd() {
+ StringPaths.validateObjectName("path/to/object//", false);
+ }
+
+ @Test
+ public void testFromComponentsValid() {
+ assertEquals("gs://my-bucket/path/to/object",
+ StringPaths.fromComponents("my-bucket", "path/to/object"));
+ assertEquals("gs://my-bucket/dir/", StringPaths.fromComponents("my-bucket", "dir/"));
+ assertEquals("gs://my-bucket/", StringPaths.fromComponents("my-bucket", ""));
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testFromComponentsNullBucketNonNullObject() {
+ StringPaths.fromComponents(null, "path/to/object");
+ }
+
+ @Test
+ public void testFromComponentsNullBucketAndObject() {
+ assertEquals("gs://", StringPaths.fromComponents(null, null));
+ }
+
+ @Test
+ public void testIsDirectoryPath() {
+ assertTrue(StringPaths.isDirectoryPath("dir/"));
+ assertTrue(StringPaths.isDirectoryPath("path/to/dir/"));
+ assertFalse(StringPaths.isDirectoryPath("file.txt"));
+ assertFalse(StringPaths.isDirectoryPath("path/to/file.txt"));
+ assertFalse(StringPaths.isDirectoryPath(""));
+ assertFalse(StringPaths.isDirectoryPath(null));
+ }
+
+ @Test
+ public void testToFilePath() {
+ assertEquals("path/to/file", StringPaths.toFilePath("path/to/file/"));
+ assertEquals("file.txt", StringPaths.toFilePath("file.txt"));
+ assertEquals("dir", StringPaths.toFilePath("dir/"));
+ assertEquals("", StringPaths.toFilePath(""));
+ assertNull(StringPaths.toFilePath(null));
+ }
+
+ // --- Tests for toDirectoryPath ---
+
+ @Test
+ public void testToDirectoryPath() {
+ assertEquals("path/to/dir/", StringPaths.toDirectoryPath("path/to/dir"));
+ assertEquals("dir/", StringPaths.toDirectoryPath("dir/"));
+ assertEquals("file/", StringPaths.toDirectoryPath("file"));
+ assertEquals("", StringPaths.toDirectoryPath(""));
+ assertNull(StringPaths.toDirectoryPath(null));
+ }
+}
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestUriPaths.java b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestUriPaths.java
new file mode 100644
index 0000000000000..0325df52f9b72
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestUriPaths.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs;
+
+import java.net.URI;
+
+import org.junit.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestUriPaths {
+ @Test
+ public void testToDirectoryFile() throws Exception {
+ URI fileUri = new URI("gs://my-bucket/path/to/file.txt");
+ URI expectedDirUri = new URI("gs://my-bucket/path/to/file.txt/");
+ // Temporarily override the behavior for testing purposes
+ // This is not a clean mocking strategy for static methods, but demonstrates the test intent.
+ // In a real environment, you'd use PowerMock or refactor.
+ URI result = UriPaths.toDirectory(fileUri);
+ assertEquals(expectedDirUri, result);
+ }
+
+ @Test
+ public void testToDirectoryAlreadyDirectory() throws Exception {
+ URI dirUri = new URI("gs://my-bucket/path/to/dir/");
+ URI result = UriPaths.toDirectory(dirUri);
+ assertEquals(dirUri, result);
+ }
+
+ @Test
+ public void testToDirectoryRootBucket() throws Exception {
+ URI bucketUri = new URI("gs://my-bucket");
+ URI result = UriPaths.toDirectory(bucketUri);
+ assertEquals(bucketUri, result); // Buckets are implicitly directories
+ }
+
+ @Test
+ public void testGetParentPathFile() throws Exception {
+ URI uri = new URI("gs://my-bucket/path/to/file.txt");
+ URI expectedParent = new URI("gs://my-bucket/path/to/");
+ assertEquals(expectedParent, UriPaths.getParentPath(uri));
+ }
+
+ @Test
+ public void testGetParentPathDirectory() throws Exception {
+ URI uri = new URI("gs://my-bucket/path/to/dir/");
+ URI expectedParent = new URI("gs://my-bucket/path/to/");
+ assertEquals(expectedParent, UriPaths.getParentPath(uri));
+ }
+
+ @Test
+ public void testGetParentPathObjectAtBucketRoot() throws Exception {
+ URI uri = new URI("gs://my-bucket/file.txt");
+ URI expectedParent = new URI("gs://my-bucket/");
+ assertEquals(expectedParent, UriPaths.getParentPath(uri));
+ }
+
+ @Test
+ public void testGetParentPathDirectoryAtBucketRoot() throws Exception {
+ URI uri = new URI("gs://my-bucket/dir/");
+ URI expectedParent = new URI("gs://my-bucket/");
+ assertEquals(expectedParent, UriPaths.getParentPath(uri));
+ }
+
+ @Test
+ public void testGetParentPathBucket() throws Exception {
+ URI uri = new URI("gs://my-bucket");
+ assertEquals(GoogleCloudStorageFileSystem.GCSROOT, UriPaths.getParentPath(uri));
+ }
+
+ @Test
+ public void testFromResourceIdObject() throws Exception {
+ StorageResourceId resourceId = new StorageResourceId("my-bucket", "path/to/object");
+ URI expectedUri = new URI("gs://my-bucket/path/to/object");
+ assertEquals(expectedUri, UriPaths.fromResourceId(resourceId, false));
+ }
+
+ @Test
+ public void testFromResourceIdDirectory() throws Exception {
+ StorageResourceId resourceId = new StorageResourceId("my-bucket", "path/to/dir/");
+ URI expectedUri = new URI("gs://my-bucket/path/to/dir/");
+ assertEquals(expectedUri, UriPaths.fromResourceId(resourceId, false));
+ }
+
+ @Test
+ public void testFromResourceIdBucket() throws Exception {
+ StorageResourceId resourceId = new StorageResourceId("my-bucket");
+ URI expectedUri = new URI("gs://my-bucket/");
+ assertEquals(expectedUri, UriPaths.fromResourceId(resourceId, true));
+ }
+
+ @Test
+ public void testFromResourceIdEmptyObjectAllowed() throws Exception {
+ StorageResourceId resourceId = new StorageResourceId("my-bucket");
+ URI expectedUri = new URI("gs://my-bucket/");
+ assertEquals(expectedUri, UriPaths.fromResourceId(resourceId, true));
+ }
+
+ @Test
+ public void testFromResourceIdNullObjectAllowed() throws Exception {
+ StorageResourceId resourceId = new StorageResourceId("my-bucket");
+ URI expectedUri = new URI("gs://my-bucket/");
+ assertEquals(expectedUri, UriPaths.fromResourceId(resourceId, true));
+ }
+
+ @Test
+ public void testFromStringPathComponentsValid() throws Exception {
+ assertEquals(new URI("gs://my-bucket/path/to/object"),
+ UriPaths.fromStringPathComponents("my-bucket", "path/to/object", false));
+ assertEquals(new URI("gs://my-bucket/path/to/dir/"),
+ UriPaths.fromStringPathComponents("my-bucket", "path/to/dir/", false));
+ assertEquals(new URI("gs://my-bucket/"),
+ UriPaths.fromStringPathComponents("my-bucket", null, true));
+ assertEquals(new URI("gs://my-bucket/"),
+ UriPaths.fromStringPathComponents("my-bucket", "", true));
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testFromStringPathComponentsNullBucketNameNotAllowed() {
+ UriPaths.fromStringPathComponents(null, "object", false);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testFromStringPathComponentsEmptyObjectNameNotAllowed() {
+ UriPaths.fromStringPathComponents("my-bucket", "", false);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testFromStringPathComponentsConsecutiveSlashes() {
+ UriPaths.fromStringPathComponents("my-bucket", "path//to/object", false);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testFromStringPathComponentsInvalidBucketName() {
+ UriPaths.fromStringPathComponents("MyBucket", "object", false); // Uppercase
+ }
+}
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/GoogleContract.java b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/GoogleContract.java
new file mode 100644
index 0000000000000..aa131981caf2b
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/GoogleContract.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs.contract;
+
+import org.apache.hadoop.fs.gs.TestConfiguration;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
+
+/** Contract of GoogleHadoopFileSystem via scheme "gs". */
+public class GoogleContract extends AbstractBondedFSContract {
+ private static final String CONTRACT_XML = "contract/gs.xml";
+
+ public GoogleContract(Configuration conf) {
+ super(conf);
+ addConfResource(CONTRACT_XML);
+ conf.set("fs.contract.test.fs.gs", "gs://arunchacko-oss-test-bucket"); // TODO:
+
+ TestConfiguration testConf = TestConfiguration.getInstance();
+ if (testConf.getProjectId() != null) {
+ conf.set("fs.gs.project.id", testConf.getProjectId());
+ }
+ }
+
+ @Override
+ public String getScheme() {
+ return "gs";
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/ITestGoogleContractDelete.java b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/ITestGoogleContractDelete.java
new file mode 100644
index 0000000000000..7ed3834025c3c
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/ITestGoogleContractDelete.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+public class ITestGoogleContractDelete extends AbstractContractDeleteTest {
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new GoogleContract(conf);
+ }
+
+ @Override
+ public void testDeleteEmptyDirNonRecursive() {
+ // TODO: Enable this
+ ContractTestUtils.skip("Skipping the test. This will be enabled in a subsequent change");
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/ITestGoogleContractGetFileStatus.java b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/ITestGoogleContractGetFileStatus.java
new file mode 100644
index 0000000000000..aae16c2a410ef
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/ITestGoogleContractGetFileStatus.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class ITestGoogleContractGetFileStatus extends AbstractContractGetFileStatusTest {
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new GoogleContract(conf);
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/ITestGoogleContractMkdir.java b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/ITestGoogleContractMkdir.java
new file mode 100644
index 0000000000000..26181f20385a3
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/ITestGoogleContractMkdir.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.gs.contract;
+
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class ITestGoogleContractMkdir extends AbstractContractMkdirTest {
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new GoogleContract(conf);
+ }
+
+ @Override
+ public void testMkdirsDoesNotRemoveParentDirectories() {
+ // TODO: Enable this
+ ContractTestUtils.skip("Skipping the test. This will be enabled in a subsequent change");
+ }
+
+ @Override
+ public void testCreateDirWithExistingDir() {
+ // TODO: Enable this
+ ContractTestUtils.skip("Skipping the test. This will be enabled in a subsequent change");
+ }
+
+ @Override
+ public void testMkDirRmDir() {
+ // TODO: Enable this
+ ContractTestUtils.skip("Skipping the test. This will be enabled in a subsequent change");
+ }
+
+ @Override
+ public void testNoMkdirOverFile() {
+ // TODO: Enable this
+ ContractTestUtils.skip("Skipping the test. This will be enabled in a subsequent change");
+ }
+
+ @Override
+ public void testMkdirOverParentFile() {
+ // TODO: Enable this
+ ContractTestUtils.skip("Skipping the test. This will be enabled in a subsequent change");
+ }
+}
diff --git a/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/package-info.java b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/package-info.java
new file mode 100644
index 0000000000000..8806dc9f45bf7
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/contract/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Google Cloud Storage Filesystem contract tests.
+ */
+package org.apache.hadoop.fs.gs.contract;
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/package-info.java b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/package-info.java
new file mode 100644
index 0000000000000..fe289cc6d3dc2
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Google Cloud Storage Filesystem tests.
+ */
+package org.apache.hadoop.fs.gs;
\ No newline at end of file
diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml
index 8c1256a177cc4..70df9b02d0fa9 100644
--- a/hadoop-tools/pom.xml
+++ b/hadoop-tools/pom.xml
@@ -53,6 +53,7 @@
hadoop-fs2img
hadoop-benchmark
hadoop-compat-bench
+ hadoop-gcp