Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-33694][gs-fs-hadoop] Support overriding GCS root URL (backport to 1.17) #24124

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.fs.gs;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.FileSystemFactory;
Expand Down Expand Up @@ -92,11 +93,16 @@ public void configure(Configuration flinkConfig) {
this.fileSystemOptions = new GSFileSystemOptions(flinkConfig);
LOGGER.info("Using file system options {}", fileSystemOptions);

// get storage credentials and construct Storage instance
StorageOptions.Builder storageOptionsBuilder = StorageOptions.newBuilder();

// get storage credentials
Optional<GoogleCredentials> credentials =
ConfigUtils.getStorageCredentials(hadoopConfig, configContext);
StorageOptions.Builder storageOptionsBuilder = StorageOptions.newBuilder();
credentials.ifPresent(storageOptionsBuilder::setCredentials);

// override the GCS root URL only if overridden in the Hadoop config
ConfigUtils.getGcsRootUrl(hadoopConfig).ifPresent(storageOptionsBuilder::setHost);

this.storage = storageOptionsBuilder.build().getService();
}

Expand All @@ -123,6 +129,11 @@ public FileSystem create(URI fsUri) throws IOException {
return new GSFileSystem(googleHadoopFileSystem, storage, fileSystemOptions);
}

@VisibleForTesting
Storage getStorage() {
return storage;
}

/** Config context implementation used at runtime. */
private static class RuntimeConfigContext implements ConfigUtils.ConfigContext {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.flink.runtime.util.HadoopConfigLoader;

import com.google.auth.oauth2.GoogleCredentials;
import com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystemConfiguration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -31,6 +32,7 @@
import java.io.Writer;
import java.util.Collections;
import java.util.Optional;
import java.util.function.BiFunction;

/** Utilities class for configuration of Hadoop and Google Storage. */
public class ConfigUtils {
Expand Down Expand Up @@ -156,6 +158,14 @@ public static Optional<GoogleCredentials> getStorageCredentials(
}
}

public static Optional<String> getGcsRootUrl(
org.apache.hadoop.conf.Configuration hadoopConfig) {
// Ignore the default value, only returning a value if actually included in the config
BiFunction<String, String, String> getterFn = (key, defaultValue) -> hadoopConfig.get(key);
String value = GoogleHadoopFileSystemConfiguration.GCS_ROOT_URL.get(hadoopConfig, getterFn);
return Optional.ofNullable(value);
}

/**
* Helper to serialize a Hadoop config to a string, for logging.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* 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.flink.fs.gs;

import org.apache.flink.configuration.Configuration;

import org.junit.jupiter.api.Test;

import static org.junit.Assert.assertEquals;

/** Tests for {@link GSFileSystemFactory}. */
public class GSFileSystemFactoryTest {

@Test
public void testOverrideStorageRootUrl() {
Configuration flinkConfig = new Configuration();
flinkConfig.setString("gs.storage.root.url", "http://240.0.0.0:12345");

GSFileSystemFactory factory = new GSFileSystemFactory();
factory.configure(flinkConfig);

String gsStorageClientHost = factory.getStorage().getOptions().getHost();
assertEquals(gsStorageClientHost, "http://240.0.0.0:12345");
}
}