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 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));
+ }
+
+ public void close() {
+ try {
+ storage.close();
+ } catch (Exception e) {
+ LOG.warn("Error occurred while closing the storage client", e);
+ }
+ }
+
+ public 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 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 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:// 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);
+ }
+}
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..16d940b16f49c
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/GoogleHadoopFileSystemConfiguration.java
@@ -0,0 +1,75 @@
+/*
+ * 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.
+ */
+ public static final HadoopConfigurationProperty 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", " 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..4155482fc7d33
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/main/java/org/apache/hadoop/fs/gs/VerificationAttributes.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2016 Google Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package 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/TestStorageResourceId.java b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestStorageResourceId.java
new file mode 100644
index 0000000000000..e0a39b2d7e403
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestStorageResourceId.java
@@ -0,0 +1,285 @@
+/*
+ * Copyright 2013 Google Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package 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..16234e0ce1d57
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestStringPaths.java
@@ -0,0 +1,164 @@
+/*
+ * Copyright 2013 Google Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package 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..fe93a28dc435c
--- /dev/null
+++ b/hadoop-tools/hadoop-gcp/src/test/java/org/apache/hadoop/fs/gs/TestUriPaths.java
@@ -0,0 +1,150 @@
+/*
+ * Copyright 2013 Google Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package 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/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 @@
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/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://