From e94eda7fbe7624aebc00c9435c40b1cff1300242 Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Wed, 27 Aug 2025 20:08:51 +0000 Subject: [PATCH 01/14] Add the FileAwareFactoryFn and the KerberosConsumerFactoryFn classes to support consumer factories which pull files from GCS. --- sdks/java/io/kafka/build.gradle | 3 + .../client/rest/entities/ErrorMessage.java | 158 +++++++++++ .../beam/sdk/io/kafka/FileAwareFactoryFn.java | 256 ++++++++++++++++++ .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 31 +++ .../io/kafka/KerberosConsumerFactoryFn.java | 106 ++++++++ .../apache/beam/sdk/io/kafka/KafkaIOIT.java | 88 ++++++ 6 files changed, 642 insertions(+) create mode 100644 sdks/java/io/kafka/src/main/java/io/confluent/kafka/schemaregistry/client/rest/entities/ErrorMessage.java create mode 100644 sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java create mode 100644 sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index ba25078b64e3..3cc03ab8e94f 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -102,6 +102,9 @@ dependencies { testImplementation project(path: ":sdks:java:extensions:protobuf", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:io:common") testImplementation project(path: ":sdks:java:testing:test-utils") + testImplementation project(path: ":runners:google-cloud-dataflow-java") + testImplementation project(path: ":sdks:java:extensions:google-cloud-platform-core") + testImplementation project(path: ":sdks:java:io:google-cloud-platform") // For testing Cross-language transforms testImplementation library.java.avro testImplementation library.java.junit diff --git a/sdks/java/io/kafka/src/main/java/io/confluent/kafka/schemaregistry/client/rest/entities/ErrorMessage.java b/sdks/java/io/kafka/src/main/java/io/confluent/kafka/schemaregistry/client/rest/entities/ErrorMessage.java new file mode 100644 index 000000000000..17dcfdfe45f7 --- /dev/null +++ b/sdks/java/io/kafka/src/main/java/io/confluent/kafka/schemaregistry/client/rest/entities/ErrorMessage.java @@ -0,0 +1,158 @@ +/* + * 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 io.confluent.kafka.schemaregistry.client.rest.entities; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.swagger.v3.oas.annotations.media.Schema; +import java.util.Map; +import java.util.Objects; +import javax.annotation.Nullable; + +/** + * Generic JSON error message. + * + *

This is an updated version of the original + * io.confluent.kafka.schemaregistry.client.rest.entities.ErrorMessage class from the + * Confluent Schema Registry Client, which has the same structure in Java and is still fully + * compatible with the client code, but supports parsing the source JSON error message in more + * formats via Jackson annotations (see {@link #unpackNestedError}). + * + *

In additional to the original error format: + * + *

+ *   {
+ *     "error_code": 400,
+ *     "message": "..."
+ *   }
+ * 
+ * + * It also supports this format: + * + *
+ *   {
+ *     "error: {
+ *       "code": 400,
+ *       "message": "..."
+ *     }
+ *   }
+ * 
+ * + * Since the class is in the same package + * io.confluent.kafka.schemaregistry.client.rest.entities, pre-pending this library to the + * classpath will override its implementation from the Schema Registry Client library and allow + * parsing Managed Kafka style error messages. + */ +@Schema(description = "Error message") +@JsonInclude(JsonInclude.Include.NON_EMPTY) +@JsonIgnoreProperties(ignoreUnknown = true) +public class ErrorMessage { + + private int errorCode = -1; + private String message = ""; + + public ErrorMessage() {} + + // Required for compatibility with the original class. + // @JsonProperty("error_code") and @JsonProperty("message") annotations + // have been removed to force Jackson always use the default constructor + // and then set the values via either setErrorCode/setMessage or + // unpackNestedError methods. + public ErrorMessage(int errorCode, String message) { + this.errorCode = errorCode; + this.message = message; + } + + @Schema(description = "Error code") + @JsonProperty("error_code") + public int getErrorCode() { + return errorCode; + } + + @JsonProperty("error_code") + public void setErrorCode(int errorCode) { + this.errorCode = errorCode; + } + + @Schema(description = "Detailed error message") + @JsonProperty + public String getMessage() { + return message; + } + + @JsonProperty + public void setMessage(String message) { + this.message = message; + } + + @JsonProperty("error") + public void unpackNestedError(Map error) { + Object code = error.get("code"); + + if (code != null) { + try { + this.errorCode = ((Integer) code); + } catch (RuntimeException e1) { + try { + this.errorCode = Integer.parseInt(code.toString()); + } catch (RuntimeException e2) { + // ignore if can't parse the error code as an integer + } + } + } + + String status = tryGetNonEmptyString(error.get("status")); + String message = tryGetNonEmptyString(error.get("message")); + + if (message != null && status != null) { + this.message = status + ": " + message; + } else if (message != null) { + this.message = message; + } else if (status != null) { + this.message = status; + } + } + + private static String tryGetNonEmptyString(@Nullable Object object) { + if (object != null) { + String s = object.toString(); + if (s.length() > 0) { + return s; + } + } + return ""; + } + + @Override + public boolean equals(@Nullable Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ErrorMessage that = (ErrorMessage) o; + return errorCode == that.errorCode && Objects.equals(message, that.message); + } + + @Override + public int hashCode() { + return Objects.hash(errorCode, message); + } +} diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java new file mode 100644 index 000000000000..6433ae9517ad --- /dev/null +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java @@ -0,0 +1,256 @@ +/* + * 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.beam.sdk.io.kafka; + +import com.google.common.base.Preconditions; +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.channels.ReadableByteChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class FileAwareFactoryFn + implements SerializableFunction, T> { + + public static final String GCS_PATH_PREFIX = "gs://"; + + public static final String DIRECTORY_PREFIX = "/tmp"; + + private static final Pattern GCS_PATH_PATTERN = Pattern.compile("(gs://[^\"]+)"); + // private static final Map secretCache = new ConcurrentHashMap<>(); + + private final String factoryType; + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(FileAwareFactoryFn.class); + + public FileAwareFactoryFn(String factoryType) { + Preconditions.checkNotNull(factoryType); + this.factoryType = factoryType; + } + + protected abstract T createObject(Map config); + + @Override + public T apply(Map config) { + if (config == null) { + return createObject(config); + } + + Map processedConfig = new HashMap<>(config); + + String key = ""; + Object value = null; + try { + downloadAndProcessExtraFiles(); + + for (Map.Entry e : config.entrySet()) { + try { + key = e.getKey(); + value = e.getValue(); + if (value instanceof String) { + String originalValue = (String) value; + String processedValue = replaceGcsPathWithLocal(originalValue); + processedConfig.put(key, processedValue); + } + } catch (IOException ex) { + throw new RuntimeException( + "Failed trying to process value " + value + " for key " + key + ".", ex); + } + } + } catch (IOException e) { + throw new RuntimeException("Failed trying to process extra files.", e); + } + + LOG.info("ProcessedConfig: {}", processedConfig); + return createObject(processedConfig); + + // String key = e.getKey(); + // String value = e.getValue() == null ? "" : e.getValue().toString(); + // String processedValue = null; + // + // try { + // if (value.startsWith(GCS_PATH_PREFIX)) { + // processedValue = downloadGcsFile(value, generateLocalFileName()); + // processedConfig.put(key, processedValue); + // } else if (JAAS_CONFIG_PROPERTY.equals(key)) { + // Matcher matcher = KEYTAB_PATH_PATTERN.matcher(value); + // if (matcher.find()) { + // String keytabGcsPath = matcher.group(1); + // LOG.info("Found GCS path for keytab: {}", keytabGcsPath); + // + // String localKeytabPath = + // downloadGcsFile(keytabGcsPath, generateLocalFileName() + ".keytab"); + // + // // Replace the GCS path in the JAAS string with the new local path + // String updatedJaasConfig = value.replace(keytabGcsPath, localKeytabPath); + // Path path = Paths.get(localKeytabPath); + // Set perms = new HashSet<>(); + // perms.add(PosixFilePermission.OWNER_READ); // Add 'read' for the owner + // Files.setPosixFilePermissions(path, perms); + // File keytabFile = new File(localKeytabPath); + // if (keytabFile.exists() && keytabFile.canRead()) { + // LOG.info("Keytab file {} exists and is readable.", localKeytabPath); + // } else if (keytabFile.exists() && !keytabFile.canRead()) { + // LOG.info("Keytab file {} exists but is not readable.", localKeytabPath); + // } else { + // LOG.info("Keytab file {} does not exist.", localKeytabPath); + // } + // + // processedConfig.put(JAAS_CONFIG_PROPERTY, updatedJaasConfig); + // LOG.info( + // "Updated '{}' to use local keytab path: {}. the full config is {}", + // JAAS_CONFIG_PROPERTY, + // localKeytabPath, + // updatedJaasConfig); + // } + // } else { + // processedConfig.put(key, value); + // } + // } catch (IOException ex) { + // throw new RuntimeException( + // "Couldn't load Kafka consumer property " + key + " = " + value, ex); + // } + // } + + // If the .conf path does not exist then we want to download the file and set the system + // property. We don't want to + // do this more than once, at the very beginning. + + // String kerberosConfigFilePath = "/tmp/krb5.conf"; + // String kerberosConfigGcsFilePath = "gs://fozzie_testing_bucket/kerberos/krb5.conf"; + // if (!new File(kerberosConfigFilePath).exists()) { + // try { + // String kerberosConfigFile = + // downloadGcsFile(kerberosConfigGcsFilePath, kerberosConfigFilePath); + // LOG.info( + // "Successfully downloaded {} into {}.", + // kerberosConfigGcsFilePath, + // kerberosConfigFilePath); + // System.setProperty("java.security.krb5.conf", kerberosConfigFile); + // Configuration.getConfiguration().refresh(); + // LOG.info( + // "Successfully set system property {} to {}.", + // "java.security.krb5.conf", + // System.getProperty("java.security.krb5.conf")); + // } catch (IOException e) { + // throw new RuntimeException("Could not load krb5.conf.", e); + // } + // } + // + // return createObject(processedConfig); + } + + /** + * A function to download files from their specified gcs path and copy them to the provided local + * filepath. The local filepath is provded by the replaceGcsPathWithLocal. + * + * @param gcsFilePath + * @param outputFileString + * @return + * @throws IOException + */ + protected static synchronized String downloadGcsFile(String gcsFilePath, String outputFileString) + throws IOException { + // create the file only if it doesn't exist + if (!new File(outputFileString).exists()) { + Path outputFilePath = Paths.get(outputFileString); + Path parentDir = outputFilePath.getParent(); + if (parentDir != null) { + Files.createDirectories(parentDir); + } + + LOG.info("Staging GCS file [{}] to [{}]", gcsFilePath, outputFileString); + Set options = new HashSet<>(2); + options.add(StandardOpenOption.CREATE); + options.add(StandardOpenOption.WRITE); + + // Copy the GCS file into a local file and will throw an I/O exception in case file not found. + try (ReadableByteChannel readerChannel = + FileSystems.open(FileSystems.matchSingleFileSpec(gcsFilePath).resourceId())) { + try (FileChannel writeChannel = FileChannel.open(outputFilePath, options)) { + writeChannel.transferFrom(readerChannel, 0, Long.MAX_VALUE); + } + } + } + return outputFileString; + } + + /** + * A helper method to create a new string with the gcs paths replaced with their local path and + * subdirectory based on the factory type in the /tmp directory. For example, the kerberos factory + * type will replace the file paths with /tmp/kerberos/file.path + * + * @param value + * @return a string with all instances of GCS paths converted to the local paths where the files + * sit. + */ + private String replaceGcsPathWithLocal(String value) throws IOException { + Matcher matcher = GCS_PATH_PATTERN.matcher(value); + StringBuffer sb = new StringBuffer(); + + LOG.info("the current value being processed: {}", value); + while (matcher.find()) { + String gcsPath = matcher.group(1); + LOG.info("THE GCS PATH IS: {}", gcsPath); + if (gcsPath != null) { + try { + String tmpPath = + DIRECTORY_PREFIX + + "/" + + factoryType + + "/" + + gcsPath.substring(GCS_PATH_PREFIX.length()); + String localPath = downloadGcsFile(gcsPath, tmpPath); + matcher.appendReplacement(sb, Matcher.quoteReplacement(localPath)); + LOG.info("Downloaded {} to {}", gcsPath, localPath); + } catch (IOException e) { + throw new IOException("Failed to download file : " + gcsPath, e); + } + } + } + matcher.appendTail(sb); + return sb.toString(); + } + + /** + * @throws IOException A hook for subclasses to download and process specific files before the + * main configuration is handled. For example, the kerberos factory can use this to download a + * krb5.conf and set a system property. + */ + protected void downloadAndProcessExtraFiles() throws IOException { + // Default implementation should do nothing. + } + + protected String getBaseDirectory() { + return DIRECTORY_PREFIX; + } +} diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 568fe49217b3..a61720d96e3c 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -930,6 +930,27 @@ static void setupExternalBuilder( builder.setOffsetDeduplication(false); builder.setRedistributeByRecordKey(false); } + + if (config.consumerFactoryFn != null) { + switch (config.consumerFactoryFn) { + case "kerberos": + if (config.kerberosConfigFile != null && !config.kerberosConfigFile.isEmpty()) { + builder.setConsumerFactoryFn( + new KerberosConsumerFactoryFn(config.kerberosConfigFile)); + } else { + throw new IllegalArgumentException( + "The kerberosConfigFile cannot be empty or null when using the" + + " kerberos consumerFactoryFn type."); + } + + break; + default: + throw new IllegalArgumentException( + config.consumerFactoryFn + + " is not a supported consumer factory type. Please use one of: " + + "(\"kerberos\")"); + } + } } private static Coder resolveCoder(Class> deserializer) { @@ -1000,6 +1021,8 @@ public static class Configuration { private Boolean offsetDeduplication; private Boolean redistributeByRecordKey; private Long dynamicReadPollIntervalSeconds; + private String consumerFactoryFn; + private String kerberosConfigFile; public void setConsumerConfig(Map consumerConfig) { this.consumerConfig = consumerConfig; @@ -1068,6 +1091,14 @@ public void setRedistributeByRecordKey(Boolean redistributeByRecordKey) { public void setDynamicReadPollIntervalSeconds(Long dynamicReadPollIntervalSeconds) { this.dynamicReadPollIntervalSeconds = dynamicReadPollIntervalSeconds; } + + public void setConsumerFactoryFn(String consumerFactoryFn) { + this.consumerFactoryFn = consumerFactoryFn; + } + + public void setKerberosConfigFile(String kerberosConfigFile) { + this.kerberosConfigFile = kerberosConfigFile; + } } } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java new file mode 100644 index 000000000000..71d74896c9c0 --- /dev/null +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java @@ -0,0 +1,106 @@ +/* + * 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.beam.sdk.io.kafka; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.attribute.PosixFilePermission; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import javax.security.auth.login.Configuration; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class KerberosConsumerFactoryFn extends FileAwareFactoryFn> { + private static final String LOCAL_FACTORY_TYPE = "kerberos"; + + private String krb5ConfigGcsPath = ""; + private static volatile String localKrb5ConfPath = ""; + + private static final Object lock = new Object(); + // Standard Kafka property for SASL JAAS configuration + // private static final String JAAS_CONFIG_PROPERTY = "sasl.jaas.config"; + + // Regex to find the keyTab path within the JAAS config string + // private static final Pattern KEYTAB_PATH_PATTERN = Pattern.compile("keyTab=\"(gs://[^\"]+)\""); + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(KerberosConsumerFactoryFn.class); + + public KerberosConsumerFactoryFn(String krb5ConfigGcsPath) { + super("kerberos"); + this.krb5ConfigGcsPath = krb5ConfigGcsPath; + } + + @Override + protected Consumer createObject(Map config) { + // This will be called after the config map processing has occurred. Therefore, we know that the + // property will have + // had it's value replaced with a local directory. We don't need to worry about the GCS prefix + // in this case. + LOG.info("config when creating the objects: {}", config); + try { + String jaasConfig = (String) config.get("sasl.jaas.config"); + String localKeytabPath = ""; + if (jaasConfig != null && !jaasConfig.isEmpty()) { + localKeytabPath = + jaasConfig.substring( + jaasConfig.indexOf("keyTab=\"") + 8, jaasConfig.lastIndexOf("\" principal")); + } + + // Set the permissions on the file to be as strict as possible for security reasons. The + // keytab contains + // sensitive information and should be as locked down as possible. + Path path = Paths.get(localKeytabPath); + Set perms = new HashSet<>(); + perms.add(PosixFilePermission.OWNER_READ); + Files.setPosixFilePermissions(path, perms); + } catch (IOException e) { + throw new RuntimeException( + "Could not access keytab file. Make sure that the sasl.jaas.config config property " + + "is set correctly.", + e); + } + return new KafkaConsumer<>(config); + } + + @Override + protected void downloadAndProcessExtraFiles() throws IOException { + synchronized (lock) { + // we only want a new krb5 file if there is not already one present. + LOG.info("gcs krb5.conf path: {}", this.krb5ConfigGcsPath); + if (localKrb5ConfPath.isEmpty()) { + if (this.krb5ConfigGcsPath != null && !this.krb5ConfigGcsPath.isEmpty()) { + String localPath = + super.getBaseDirectory() + "/" + LOCAL_FACTORY_TYPE + "/" + "krb5.conf"; + localKrb5ConfPath = downloadGcsFile(this.krb5ConfigGcsPath, localPath); + + System.setProperty("java.security.krb5.conf", localKrb5ConfPath); + Configuration.getConfiguration().refresh(); + LOG.info( + "Successfully set and refreshed java.security.krb5.conf to {}", localKrb5ConfPath); + } + } + } + } +} diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index 0e8cbd2183ca..51ca6059ed8c 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -26,10 +26,12 @@ import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.time.Instant; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Random; @@ -38,6 +40,8 @@ import java.util.function.BiFunction; import java.util.stream.Collectors; import java.util.stream.LongStream; +import org.apache.beam.runners.dataflow.DataflowRunner; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -221,6 +225,7 @@ public static void afterClass() { } } + @Ignore @Test public void testKafkaIOFailsFastWithInvalidPartitions() throws IOException { thrown.expect(Pipeline.PipelineExecutionException.class); @@ -245,6 +250,7 @@ public void testKafkaIOFailsFastWithInvalidPartitions() throws IOException { cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIOFailsFastWithInvalidPartitionsAndFlagExplicitlySet() throws IOException { thrown.expect(Pipeline.PipelineExecutionException.class); @@ -271,6 +277,7 @@ public void testKafkaIOFailsFastWithInvalidPartitionsAndFlagExplicitlySet() thro cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIODoesNotFailFastWithInvalidPartitionsAndFlagExplicitlyNotSet() throws IOException { @@ -298,6 +305,7 @@ public void testKafkaIODoesNotFailFastWithInvalidPartitionsAndFlagExplicitlyNotS cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIOFailsFastWithInvalidTopics() throws IOException { // This test will fail on versions before 2.3.0 due to the non-existence of the @@ -332,6 +340,7 @@ public void testKafkaIOFailsFastWithInvalidTopics() throws IOException { cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIOFailsFastWithInvalidTopicsAndDynamicRead() throws IOException { // This test will fail on versions before 2.3.0 due to the non-existence of the @@ -367,6 +376,7 @@ public void testKafkaIOFailsFastWithInvalidTopicsAndDynamicRead() throws IOExcep cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIOLogsTopicVerificationWithDynamicRead() throws IOException { // This test will fail on versions before 2.3.0 due to the non-existence of the @@ -403,6 +413,7 @@ public void testKafkaIOLogsTopicVerificationWithDynamicRead() throws IOException cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIODoesNotErrorAtValidationWithBadBootstrapServer() throws IOException { // expect an error during execution that the bootstrap server is bad, not during validation @@ -428,6 +439,7 @@ public void testKafkaIODoesNotErrorAtValidationWithBadBootstrapServer() throws I cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIOReadsAndWritesCorrectlyInStreaming() throws IOException { // Use batch pipeline to write records. @@ -470,6 +482,7 @@ public void testKafkaIOReadsAndWritesCorrectlyInStreaming() throws IOException { assertNotEquals(PipelineResult.State.FAILED, readState); } + @Ignore @Test public void testKafkaIOReadsAndWritesCorrectlyInBatch() throws IOException { writePipeline @@ -511,6 +524,7 @@ public void testKafkaIOReadsAndWritesCorrectlyInBatch() throws IOException { // DoFns. CrashOnExtra will throw an exception if we see any extra records beyond those we // expect, and LogFn acts as a sink we can inspect using ExpectedLogs to verify that we got all // those we expect. + @Ignore @Test public void testKafkaIOSDFResumesCorrectly() throws IOException { roundtripElements("first-pass", 4, writePipeline, sdfReadPipeline); @@ -599,6 +613,7 @@ public void processElement(@Element String element, OutputReceiver outpu } // This test verifies that bad data from Kafka is properly sent to the error handler + @Ignore @Test public void testKafkaIOSDFReadWithErrorHandler() throws IOException { // TODO(https://github.com/apache/beam/issues/32704) re-enable when fixed, or remove the support @@ -640,6 +655,7 @@ public void testKafkaIOSDFReadWithErrorHandler() throws IOException { assertNotEquals(PipelineResult.State.FAILED, readState); } + @Ignore @Test public void testKafkaIOWriteWithErrorHandler() throws IOException { @@ -666,6 +682,7 @@ public void testKafkaIOWriteWithErrorHandler() throws IOException { // This test roundtrips a single KV to verify that externalWithMetadata // can handle null keys and values correctly. + @Ignore @Test public void testKafkaIOExternalRoundtripWithMetadataAndNullKeysAndValues() throws IOException { @@ -705,6 +722,7 @@ public void testKafkaIOExternalRoundtripWithMetadataAndNullKeysAndValues() throw cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaWithDynamicPartitions() throws IOException { AdminClient client = @@ -792,6 +810,7 @@ public void testKafkaWithDynamicPartitions() throws IOException { } } + @Ignore @Test public void testKafkaWithStopReadingFunction() { AlwaysStopCheckStopReadingFn checkStopReadingFn = new AlwaysStopCheckStopReadingFn(); @@ -806,6 +825,7 @@ public Boolean apply(TopicPartition input) { } } + @Ignore @Test public void testKafkaWithDelayedStopReadingFunction() { DelayedCheckStopReadingFn checkStopReadingFn = new DelayedCheckStopReadingFn(); @@ -853,12 +873,14 @@ public void testKafkaWithDelayedStopReadingFunction() { private static final int FIVE_MINUTES_IN_MS = 5 * 60 * 1000; + @Ignore @Test(timeout = FIVE_MINUTES_IN_MS) public void testKafkaViaManagedSchemaTransformJson() { runReadWriteKafkaViaManagedSchemaTransforms( "JSON", SCHEMA_IN_JSON, JsonUtils.beamSchemaFromJsonSchema(SCHEMA_IN_JSON)); } + @Ignore @Test(timeout = FIVE_MINUTES_IN_MS) public void testKafkaViaManagedSchemaTransformAvro() { runReadWriteKafkaViaManagedSchemaTransforms( @@ -996,6 +1018,7 @@ private void runWithStopReadingFn( readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); } + @Ignore @Test public void testWatermarkUpdateWithSparseMessages() throws IOException, InterruptedException { AdminClient client = @@ -1060,6 +1083,70 @@ public void testWatermarkUpdateWithSparseMessages() throws IOException, Interrup } } + @Test + public void testReadFromKafkaWithKerberosAuthentication() throws IOException { + DataflowPipelineOptions pReadOptions = + PipelineOptionsFactory.create().as(DataflowPipelineOptions.class); + pReadOptions.setAppName("KafkaIOIT-testReadFromKafkaWithKerberosAuthentication"); + List experiments = new ArrayList<>(); + experiments.add("use_sdf_read"); + experiments.add("beam_fn_api"); + experiments.add("use_staged_dataflow_worker_jar"); + // experiments.add("java_worker_options=-Djava.security.krb5.conf=/tmp/krb5.conf"); + pReadOptions.setExperiments(experiments); + pReadOptions.setRunner(DataflowRunner.class); + pReadOptions.setProject("dataflow-testing-311516"); + pReadOptions.setRegion("us-central1"); + pReadOptions.setJobName("testReadFromKafkaWithKerberosAuthentication" + UUID.randomUUID()); + pReadOptions.setNetwork("fozzie-test-vpc"); + pReadOptions.setSubnetwork("regions/us-central1/subnetworks/fozzie-test-vpc-subnet"); + + Pipeline pRead = Pipeline.create(pReadOptions); + String topicName = "fozzie_test_kerberos_topic"; + String bootstrapServer = + "fozzie-test-kafka-broker.us-central1-c.c.dataflow-testing-311516.internal:9092"; + String saslJaasConfig = + "com.sun.security.auth.module.Krb5LoginModule required " + + "useTicketCache=false " + + "useKeyTab=true " + + "storeKey=true " + + "keyTab=\"gs://fozzie_testing_bucket/kerberos/kafka-client-keytab.keytab\" " + + "principal=\"kafka-client@US-CENTRAL1-B.C.DATAFLOW-TESTING-311516.INTERNAL\";"; + + Map consumerConfig = new HashMap<>(); + + consumerConfig.put("security.protocol", "SASL_PLAINTEXT"); + consumerConfig.put("sasl.mechanism", "GSSAPI"); + consumerConfig.put("sasl.kerberos.service.name", "kafka"); + consumerConfig.put("auto.offset.reset", "earliest"); + consumerConfig.put("sasl.jaas.config", saslJaasConfig); + + PCollection count = + pRead + .apply( + "Read from Kafka", + KafkaIO.read() + .withBootstrapServers(bootstrapServer) + .withConsumerConfigUpdates(consumerConfig) + .withTopic(topicName) + .withKeyDeserializer(StringDeserializer.class) + .withValueDeserializer(StringDeserializer.class) + .withConsumerFactoryFn( + new KerberosConsumerFactoryFn( + "gs://fozzie_testing_bucket/kerberos/krb5.conf")) + .withMaxNumRecords((long) 11)) + .apply("Counting element", Count.globally()); + + PipelineResult readResult = pRead.run(); + + // Only waiting 5 seconds here because we don't expect any processing at this point + PipelineResult.State readState = readResult.waitUntilFinish(Duration.standardMinutes(15)); + cancelIfTimeouted(readResult, readState); + assertNotEquals(PipelineResult.State.FAILED, readState); + assertNotEquals(PipelineResult.State.CANCELLED, readState); + PAssert.thatSingleton(count).isEqualTo((long) 11); + } + @Ignore( "Test is ignored until GMK is utilized as part of this test suite (https://github.com/apache/beam/issues/32721).") @Test @@ -1239,6 +1326,7 @@ private static void setupKafkaContainer() { options.setKafkaBootstrapServerAddresses(kafkaContainer.getBootstrapServers()); } + @Ignore @Test public void testCustomRowDeserializerWithViaSDF() throws IOException { // This test verifies that the SDF implementation of KafkaIO correctly handles From 90e9f914e79c772b514514f5c9572be4f4ae4fec Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Wed, 27 Aug 2025 23:23:34 +0000 Subject: [PATCH 02/14] Revert "Add the FileAwareFactoryFn and the KerberosConsumerFactoryFn classes to support consumer factories which pull files from GCS." This reverts commit f8f69d9bc1ecd6cba6104c82af65474b0354b664. --- sdks/java/io/kafka/build.gradle | 3 - .../client/rest/entities/ErrorMessage.java | 158 ----------- .../beam/sdk/io/kafka/FileAwareFactoryFn.java | 256 ------------------ .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 31 --- .../io/kafka/KerberosConsumerFactoryFn.java | 106 -------- .../apache/beam/sdk/io/kafka/KafkaIOIT.java | 88 ------ 6 files changed, 642 deletions(-) delete mode 100644 sdks/java/io/kafka/src/main/java/io/confluent/kafka/schemaregistry/client/rest/entities/ErrorMessage.java delete mode 100644 sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java delete mode 100644 sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index 3cc03ab8e94f..ba25078b64e3 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -102,9 +102,6 @@ dependencies { testImplementation project(path: ":sdks:java:extensions:protobuf", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:io:common") testImplementation project(path: ":sdks:java:testing:test-utils") - testImplementation project(path: ":runners:google-cloud-dataflow-java") - testImplementation project(path: ":sdks:java:extensions:google-cloud-platform-core") - testImplementation project(path: ":sdks:java:io:google-cloud-platform") // For testing Cross-language transforms testImplementation library.java.avro testImplementation library.java.junit diff --git a/sdks/java/io/kafka/src/main/java/io/confluent/kafka/schemaregistry/client/rest/entities/ErrorMessage.java b/sdks/java/io/kafka/src/main/java/io/confluent/kafka/schemaregistry/client/rest/entities/ErrorMessage.java deleted file mode 100644 index 17dcfdfe45f7..000000000000 --- a/sdks/java/io/kafka/src/main/java/io/confluent/kafka/schemaregistry/client/rest/entities/ErrorMessage.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * 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 io.confluent.kafka.schemaregistry.client.rest.entities; - -import com.fasterxml.jackson.annotation.JsonIgnoreProperties; -import com.fasterxml.jackson.annotation.JsonInclude; -import com.fasterxml.jackson.annotation.JsonProperty; -import io.swagger.v3.oas.annotations.media.Schema; -import java.util.Map; -import java.util.Objects; -import javax.annotation.Nullable; - -/** - * Generic JSON error message. - * - *

This is an updated version of the original - * io.confluent.kafka.schemaregistry.client.rest.entities.ErrorMessage class from the - * Confluent Schema Registry Client, which has the same structure in Java and is still fully - * compatible with the client code, but supports parsing the source JSON error message in more - * formats via Jackson annotations (see {@link #unpackNestedError}). - * - *

In additional to the original error format: - * - *

- *   {
- *     "error_code": 400,
- *     "message": "..."
- *   }
- * 
- * - * It also supports this format: - * - *
- *   {
- *     "error: {
- *       "code": 400,
- *       "message": "..."
- *     }
- *   }
- * 
- * - * Since the class is in the same package - * io.confluent.kafka.schemaregistry.client.rest.entities, pre-pending this library to the - * classpath will override its implementation from the Schema Registry Client library and allow - * parsing Managed Kafka style error messages. - */ -@Schema(description = "Error message") -@JsonInclude(JsonInclude.Include.NON_EMPTY) -@JsonIgnoreProperties(ignoreUnknown = true) -public class ErrorMessage { - - private int errorCode = -1; - private String message = ""; - - public ErrorMessage() {} - - // Required for compatibility with the original class. - // @JsonProperty("error_code") and @JsonProperty("message") annotations - // have been removed to force Jackson always use the default constructor - // and then set the values via either setErrorCode/setMessage or - // unpackNestedError methods. - public ErrorMessage(int errorCode, String message) { - this.errorCode = errorCode; - this.message = message; - } - - @Schema(description = "Error code") - @JsonProperty("error_code") - public int getErrorCode() { - return errorCode; - } - - @JsonProperty("error_code") - public void setErrorCode(int errorCode) { - this.errorCode = errorCode; - } - - @Schema(description = "Detailed error message") - @JsonProperty - public String getMessage() { - return message; - } - - @JsonProperty - public void setMessage(String message) { - this.message = message; - } - - @JsonProperty("error") - public void unpackNestedError(Map error) { - Object code = error.get("code"); - - if (code != null) { - try { - this.errorCode = ((Integer) code); - } catch (RuntimeException e1) { - try { - this.errorCode = Integer.parseInt(code.toString()); - } catch (RuntimeException e2) { - // ignore if can't parse the error code as an integer - } - } - } - - String status = tryGetNonEmptyString(error.get("status")); - String message = tryGetNonEmptyString(error.get("message")); - - if (message != null && status != null) { - this.message = status + ": " + message; - } else if (message != null) { - this.message = message; - } else if (status != null) { - this.message = status; - } - } - - private static String tryGetNonEmptyString(@Nullable Object object) { - if (object != null) { - String s = object.toString(); - if (s.length() > 0) { - return s; - } - } - return ""; - } - - @Override - public boolean equals(@Nullable Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ErrorMessage that = (ErrorMessage) o; - return errorCode == that.errorCode && Objects.equals(message, that.message); - } - - @Override - public int hashCode() { - return Objects.hash(errorCode, message); - } -} diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java deleted file mode 100644 index 6433ae9517ad..000000000000 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java +++ /dev/null @@ -1,256 +0,0 @@ -/* - * 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.beam.sdk.io.kafka; - -import com.google.common.base.Preconditions; -import java.io.File; -import java.io.IOException; -import java.nio.channels.FileChannel; -import java.nio.channels.ReadableByteChannel; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.nio.file.StandardOpenOption; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.beam.sdk.io.FileSystems; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public abstract class FileAwareFactoryFn - implements SerializableFunction, T> { - - public static final String GCS_PATH_PREFIX = "gs://"; - - public static final String DIRECTORY_PREFIX = "/tmp"; - - private static final Pattern GCS_PATH_PATTERN = Pattern.compile("(gs://[^\"]+)"); - // private static final Map secretCache = new ConcurrentHashMap<>(); - - private final String factoryType; - - /* Logger for class.*/ - private static final Logger LOG = LoggerFactory.getLogger(FileAwareFactoryFn.class); - - public FileAwareFactoryFn(String factoryType) { - Preconditions.checkNotNull(factoryType); - this.factoryType = factoryType; - } - - protected abstract T createObject(Map config); - - @Override - public T apply(Map config) { - if (config == null) { - return createObject(config); - } - - Map processedConfig = new HashMap<>(config); - - String key = ""; - Object value = null; - try { - downloadAndProcessExtraFiles(); - - for (Map.Entry e : config.entrySet()) { - try { - key = e.getKey(); - value = e.getValue(); - if (value instanceof String) { - String originalValue = (String) value; - String processedValue = replaceGcsPathWithLocal(originalValue); - processedConfig.put(key, processedValue); - } - } catch (IOException ex) { - throw new RuntimeException( - "Failed trying to process value " + value + " for key " + key + ".", ex); - } - } - } catch (IOException e) { - throw new RuntimeException("Failed trying to process extra files.", e); - } - - LOG.info("ProcessedConfig: {}", processedConfig); - return createObject(processedConfig); - - // String key = e.getKey(); - // String value = e.getValue() == null ? "" : e.getValue().toString(); - // String processedValue = null; - // - // try { - // if (value.startsWith(GCS_PATH_PREFIX)) { - // processedValue = downloadGcsFile(value, generateLocalFileName()); - // processedConfig.put(key, processedValue); - // } else if (JAAS_CONFIG_PROPERTY.equals(key)) { - // Matcher matcher = KEYTAB_PATH_PATTERN.matcher(value); - // if (matcher.find()) { - // String keytabGcsPath = matcher.group(1); - // LOG.info("Found GCS path for keytab: {}", keytabGcsPath); - // - // String localKeytabPath = - // downloadGcsFile(keytabGcsPath, generateLocalFileName() + ".keytab"); - // - // // Replace the GCS path in the JAAS string with the new local path - // String updatedJaasConfig = value.replace(keytabGcsPath, localKeytabPath); - // Path path = Paths.get(localKeytabPath); - // Set perms = new HashSet<>(); - // perms.add(PosixFilePermission.OWNER_READ); // Add 'read' for the owner - // Files.setPosixFilePermissions(path, perms); - // File keytabFile = new File(localKeytabPath); - // if (keytabFile.exists() && keytabFile.canRead()) { - // LOG.info("Keytab file {} exists and is readable.", localKeytabPath); - // } else if (keytabFile.exists() && !keytabFile.canRead()) { - // LOG.info("Keytab file {} exists but is not readable.", localKeytabPath); - // } else { - // LOG.info("Keytab file {} does not exist.", localKeytabPath); - // } - // - // processedConfig.put(JAAS_CONFIG_PROPERTY, updatedJaasConfig); - // LOG.info( - // "Updated '{}' to use local keytab path: {}. the full config is {}", - // JAAS_CONFIG_PROPERTY, - // localKeytabPath, - // updatedJaasConfig); - // } - // } else { - // processedConfig.put(key, value); - // } - // } catch (IOException ex) { - // throw new RuntimeException( - // "Couldn't load Kafka consumer property " + key + " = " + value, ex); - // } - // } - - // If the .conf path does not exist then we want to download the file and set the system - // property. We don't want to - // do this more than once, at the very beginning. - - // String kerberosConfigFilePath = "/tmp/krb5.conf"; - // String kerberosConfigGcsFilePath = "gs://fozzie_testing_bucket/kerberos/krb5.conf"; - // if (!new File(kerberosConfigFilePath).exists()) { - // try { - // String kerberosConfigFile = - // downloadGcsFile(kerberosConfigGcsFilePath, kerberosConfigFilePath); - // LOG.info( - // "Successfully downloaded {} into {}.", - // kerberosConfigGcsFilePath, - // kerberosConfigFilePath); - // System.setProperty("java.security.krb5.conf", kerberosConfigFile); - // Configuration.getConfiguration().refresh(); - // LOG.info( - // "Successfully set system property {} to {}.", - // "java.security.krb5.conf", - // System.getProperty("java.security.krb5.conf")); - // } catch (IOException e) { - // throw new RuntimeException("Could not load krb5.conf.", e); - // } - // } - // - // return createObject(processedConfig); - } - - /** - * A function to download files from their specified gcs path and copy them to the provided local - * filepath. The local filepath is provded by the replaceGcsPathWithLocal. - * - * @param gcsFilePath - * @param outputFileString - * @return - * @throws IOException - */ - protected static synchronized String downloadGcsFile(String gcsFilePath, String outputFileString) - throws IOException { - // create the file only if it doesn't exist - if (!new File(outputFileString).exists()) { - Path outputFilePath = Paths.get(outputFileString); - Path parentDir = outputFilePath.getParent(); - if (parentDir != null) { - Files.createDirectories(parentDir); - } - - LOG.info("Staging GCS file [{}] to [{}]", gcsFilePath, outputFileString); - Set options = new HashSet<>(2); - options.add(StandardOpenOption.CREATE); - options.add(StandardOpenOption.WRITE); - - // Copy the GCS file into a local file and will throw an I/O exception in case file not found. - try (ReadableByteChannel readerChannel = - FileSystems.open(FileSystems.matchSingleFileSpec(gcsFilePath).resourceId())) { - try (FileChannel writeChannel = FileChannel.open(outputFilePath, options)) { - writeChannel.transferFrom(readerChannel, 0, Long.MAX_VALUE); - } - } - } - return outputFileString; - } - - /** - * A helper method to create a new string with the gcs paths replaced with their local path and - * subdirectory based on the factory type in the /tmp directory. For example, the kerberos factory - * type will replace the file paths with /tmp/kerberos/file.path - * - * @param value - * @return a string with all instances of GCS paths converted to the local paths where the files - * sit. - */ - private String replaceGcsPathWithLocal(String value) throws IOException { - Matcher matcher = GCS_PATH_PATTERN.matcher(value); - StringBuffer sb = new StringBuffer(); - - LOG.info("the current value being processed: {}", value); - while (matcher.find()) { - String gcsPath = matcher.group(1); - LOG.info("THE GCS PATH IS: {}", gcsPath); - if (gcsPath != null) { - try { - String tmpPath = - DIRECTORY_PREFIX - + "/" - + factoryType - + "/" - + gcsPath.substring(GCS_PATH_PREFIX.length()); - String localPath = downloadGcsFile(gcsPath, tmpPath); - matcher.appendReplacement(sb, Matcher.quoteReplacement(localPath)); - LOG.info("Downloaded {} to {}", gcsPath, localPath); - } catch (IOException e) { - throw new IOException("Failed to download file : " + gcsPath, e); - } - } - } - matcher.appendTail(sb); - return sb.toString(); - } - - /** - * @throws IOException A hook for subclasses to download and process specific files before the - * main configuration is handled. For example, the kerberos factory can use this to download a - * krb5.conf and set a system property. - */ - protected void downloadAndProcessExtraFiles() throws IOException { - // Default implementation should do nothing. - } - - protected String getBaseDirectory() { - return DIRECTORY_PREFIX; - } -} diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index a61720d96e3c..568fe49217b3 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -930,27 +930,6 @@ static void setupExternalBuilder( builder.setOffsetDeduplication(false); builder.setRedistributeByRecordKey(false); } - - if (config.consumerFactoryFn != null) { - switch (config.consumerFactoryFn) { - case "kerberos": - if (config.kerberosConfigFile != null && !config.kerberosConfigFile.isEmpty()) { - builder.setConsumerFactoryFn( - new KerberosConsumerFactoryFn(config.kerberosConfigFile)); - } else { - throw new IllegalArgumentException( - "The kerberosConfigFile cannot be empty or null when using the" - + " kerberos consumerFactoryFn type."); - } - - break; - default: - throw new IllegalArgumentException( - config.consumerFactoryFn - + " is not a supported consumer factory type. Please use one of: " - + "(\"kerberos\")"); - } - } } private static Coder resolveCoder(Class> deserializer) { @@ -1021,8 +1000,6 @@ public static class Configuration { private Boolean offsetDeduplication; private Boolean redistributeByRecordKey; private Long dynamicReadPollIntervalSeconds; - private String consumerFactoryFn; - private String kerberosConfigFile; public void setConsumerConfig(Map consumerConfig) { this.consumerConfig = consumerConfig; @@ -1091,14 +1068,6 @@ public void setRedistributeByRecordKey(Boolean redistributeByRecordKey) { public void setDynamicReadPollIntervalSeconds(Long dynamicReadPollIntervalSeconds) { this.dynamicReadPollIntervalSeconds = dynamicReadPollIntervalSeconds; } - - public void setConsumerFactoryFn(String consumerFactoryFn) { - this.consumerFactoryFn = consumerFactoryFn; - } - - public void setKerberosConfigFile(String kerberosConfigFile) { - this.kerberosConfigFile = kerberosConfigFile; - } } } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java deleted file mode 100644 index 71d74896c9c0..000000000000 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * 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.beam.sdk.io.kafka; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.nio.file.attribute.PosixFilePermission; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import javax.security.auth.login.Configuration; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class KerberosConsumerFactoryFn extends FileAwareFactoryFn> { - private static final String LOCAL_FACTORY_TYPE = "kerberos"; - - private String krb5ConfigGcsPath = ""; - private static volatile String localKrb5ConfPath = ""; - - private static final Object lock = new Object(); - // Standard Kafka property for SASL JAAS configuration - // private static final String JAAS_CONFIG_PROPERTY = "sasl.jaas.config"; - - // Regex to find the keyTab path within the JAAS config string - // private static final Pattern KEYTAB_PATH_PATTERN = Pattern.compile("keyTab=\"(gs://[^\"]+)\""); - - /* Logger for class.*/ - private static final Logger LOG = LoggerFactory.getLogger(KerberosConsumerFactoryFn.class); - - public KerberosConsumerFactoryFn(String krb5ConfigGcsPath) { - super("kerberos"); - this.krb5ConfigGcsPath = krb5ConfigGcsPath; - } - - @Override - protected Consumer createObject(Map config) { - // This will be called after the config map processing has occurred. Therefore, we know that the - // property will have - // had it's value replaced with a local directory. We don't need to worry about the GCS prefix - // in this case. - LOG.info("config when creating the objects: {}", config); - try { - String jaasConfig = (String) config.get("sasl.jaas.config"); - String localKeytabPath = ""; - if (jaasConfig != null && !jaasConfig.isEmpty()) { - localKeytabPath = - jaasConfig.substring( - jaasConfig.indexOf("keyTab=\"") + 8, jaasConfig.lastIndexOf("\" principal")); - } - - // Set the permissions on the file to be as strict as possible for security reasons. The - // keytab contains - // sensitive information and should be as locked down as possible. - Path path = Paths.get(localKeytabPath); - Set perms = new HashSet<>(); - perms.add(PosixFilePermission.OWNER_READ); - Files.setPosixFilePermissions(path, perms); - } catch (IOException e) { - throw new RuntimeException( - "Could not access keytab file. Make sure that the sasl.jaas.config config property " - + "is set correctly.", - e); - } - return new KafkaConsumer<>(config); - } - - @Override - protected void downloadAndProcessExtraFiles() throws IOException { - synchronized (lock) { - // we only want a new krb5 file if there is not already one present. - LOG.info("gcs krb5.conf path: {}", this.krb5ConfigGcsPath); - if (localKrb5ConfPath.isEmpty()) { - if (this.krb5ConfigGcsPath != null && !this.krb5ConfigGcsPath.isEmpty()) { - String localPath = - super.getBaseDirectory() + "/" + LOCAL_FACTORY_TYPE + "/" + "krb5.conf"; - localKrb5ConfPath = downloadGcsFile(this.krb5ConfigGcsPath, localPath); - - System.setProperty("java.security.krb5.conf", localKrb5ConfPath); - Configuration.getConfiguration().refresh(); - LOG.info( - "Successfully set and refreshed java.security.krb5.conf to {}", localKrb5ConfPath); - } - } - } - } -} diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index 51ca6059ed8c..0e8cbd2183ca 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -26,12 +26,10 @@ import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.time.Instant; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Random; @@ -40,8 +38,6 @@ import java.util.function.BiFunction; import java.util.stream.Collectors; import java.util.stream.LongStream; -import org.apache.beam.runners.dataflow.DataflowRunner; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -225,7 +221,6 @@ public static void afterClass() { } } - @Ignore @Test public void testKafkaIOFailsFastWithInvalidPartitions() throws IOException { thrown.expect(Pipeline.PipelineExecutionException.class); @@ -250,7 +245,6 @@ public void testKafkaIOFailsFastWithInvalidPartitions() throws IOException { cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIOFailsFastWithInvalidPartitionsAndFlagExplicitlySet() throws IOException { thrown.expect(Pipeline.PipelineExecutionException.class); @@ -277,7 +271,6 @@ public void testKafkaIOFailsFastWithInvalidPartitionsAndFlagExplicitlySet() thro cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIODoesNotFailFastWithInvalidPartitionsAndFlagExplicitlyNotSet() throws IOException { @@ -305,7 +298,6 @@ public void testKafkaIODoesNotFailFastWithInvalidPartitionsAndFlagExplicitlyNotS cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIOFailsFastWithInvalidTopics() throws IOException { // This test will fail on versions before 2.3.0 due to the non-existence of the @@ -340,7 +332,6 @@ public void testKafkaIOFailsFastWithInvalidTopics() throws IOException { cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIOFailsFastWithInvalidTopicsAndDynamicRead() throws IOException { // This test will fail on versions before 2.3.0 due to the non-existence of the @@ -376,7 +367,6 @@ public void testKafkaIOFailsFastWithInvalidTopicsAndDynamicRead() throws IOExcep cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIOLogsTopicVerificationWithDynamicRead() throws IOException { // This test will fail on versions before 2.3.0 due to the non-existence of the @@ -413,7 +403,6 @@ public void testKafkaIOLogsTopicVerificationWithDynamicRead() throws IOException cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIODoesNotErrorAtValidationWithBadBootstrapServer() throws IOException { // expect an error during execution that the bootstrap server is bad, not during validation @@ -439,7 +428,6 @@ public void testKafkaIODoesNotErrorAtValidationWithBadBootstrapServer() throws I cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIOReadsAndWritesCorrectlyInStreaming() throws IOException { // Use batch pipeline to write records. @@ -482,7 +470,6 @@ public void testKafkaIOReadsAndWritesCorrectlyInStreaming() throws IOException { assertNotEquals(PipelineResult.State.FAILED, readState); } - @Ignore @Test public void testKafkaIOReadsAndWritesCorrectlyInBatch() throws IOException { writePipeline @@ -524,7 +511,6 @@ public void testKafkaIOReadsAndWritesCorrectlyInBatch() throws IOException { // DoFns. CrashOnExtra will throw an exception if we see any extra records beyond those we // expect, and LogFn acts as a sink we can inspect using ExpectedLogs to verify that we got all // those we expect. - @Ignore @Test public void testKafkaIOSDFResumesCorrectly() throws IOException { roundtripElements("first-pass", 4, writePipeline, sdfReadPipeline); @@ -613,7 +599,6 @@ public void processElement(@Element String element, OutputReceiver outpu } // This test verifies that bad data from Kafka is properly sent to the error handler - @Ignore @Test public void testKafkaIOSDFReadWithErrorHandler() throws IOException { // TODO(https://github.com/apache/beam/issues/32704) re-enable when fixed, or remove the support @@ -655,7 +640,6 @@ public void testKafkaIOSDFReadWithErrorHandler() throws IOException { assertNotEquals(PipelineResult.State.FAILED, readState); } - @Ignore @Test public void testKafkaIOWriteWithErrorHandler() throws IOException { @@ -682,7 +666,6 @@ public void testKafkaIOWriteWithErrorHandler() throws IOException { // This test roundtrips a single KV to verify that externalWithMetadata // can handle null keys and values correctly. - @Ignore @Test public void testKafkaIOExternalRoundtripWithMetadataAndNullKeysAndValues() throws IOException { @@ -722,7 +705,6 @@ public void testKafkaIOExternalRoundtripWithMetadataAndNullKeysAndValues() throw cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaWithDynamicPartitions() throws IOException { AdminClient client = @@ -810,7 +792,6 @@ public void testKafkaWithDynamicPartitions() throws IOException { } } - @Ignore @Test public void testKafkaWithStopReadingFunction() { AlwaysStopCheckStopReadingFn checkStopReadingFn = new AlwaysStopCheckStopReadingFn(); @@ -825,7 +806,6 @@ public Boolean apply(TopicPartition input) { } } - @Ignore @Test public void testKafkaWithDelayedStopReadingFunction() { DelayedCheckStopReadingFn checkStopReadingFn = new DelayedCheckStopReadingFn(); @@ -873,14 +853,12 @@ public void testKafkaWithDelayedStopReadingFunction() { private static final int FIVE_MINUTES_IN_MS = 5 * 60 * 1000; - @Ignore @Test(timeout = FIVE_MINUTES_IN_MS) public void testKafkaViaManagedSchemaTransformJson() { runReadWriteKafkaViaManagedSchemaTransforms( "JSON", SCHEMA_IN_JSON, JsonUtils.beamSchemaFromJsonSchema(SCHEMA_IN_JSON)); } - @Ignore @Test(timeout = FIVE_MINUTES_IN_MS) public void testKafkaViaManagedSchemaTransformAvro() { runReadWriteKafkaViaManagedSchemaTransforms( @@ -1018,7 +996,6 @@ private void runWithStopReadingFn( readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); } - @Ignore @Test public void testWatermarkUpdateWithSparseMessages() throws IOException, InterruptedException { AdminClient client = @@ -1083,70 +1060,6 @@ public void testWatermarkUpdateWithSparseMessages() throws IOException, Interrup } } - @Test - public void testReadFromKafkaWithKerberosAuthentication() throws IOException { - DataflowPipelineOptions pReadOptions = - PipelineOptionsFactory.create().as(DataflowPipelineOptions.class); - pReadOptions.setAppName("KafkaIOIT-testReadFromKafkaWithKerberosAuthentication"); - List experiments = new ArrayList<>(); - experiments.add("use_sdf_read"); - experiments.add("beam_fn_api"); - experiments.add("use_staged_dataflow_worker_jar"); - // experiments.add("java_worker_options=-Djava.security.krb5.conf=/tmp/krb5.conf"); - pReadOptions.setExperiments(experiments); - pReadOptions.setRunner(DataflowRunner.class); - pReadOptions.setProject("dataflow-testing-311516"); - pReadOptions.setRegion("us-central1"); - pReadOptions.setJobName("testReadFromKafkaWithKerberosAuthentication" + UUID.randomUUID()); - pReadOptions.setNetwork("fozzie-test-vpc"); - pReadOptions.setSubnetwork("regions/us-central1/subnetworks/fozzie-test-vpc-subnet"); - - Pipeline pRead = Pipeline.create(pReadOptions); - String topicName = "fozzie_test_kerberos_topic"; - String bootstrapServer = - "fozzie-test-kafka-broker.us-central1-c.c.dataflow-testing-311516.internal:9092"; - String saslJaasConfig = - "com.sun.security.auth.module.Krb5LoginModule required " - + "useTicketCache=false " - + "useKeyTab=true " - + "storeKey=true " - + "keyTab=\"gs://fozzie_testing_bucket/kerberos/kafka-client-keytab.keytab\" " - + "principal=\"kafka-client@US-CENTRAL1-B.C.DATAFLOW-TESTING-311516.INTERNAL\";"; - - Map consumerConfig = new HashMap<>(); - - consumerConfig.put("security.protocol", "SASL_PLAINTEXT"); - consumerConfig.put("sasl.mechanism", "GSSAPI"); - consumerConfig.put("sasl.kerberos.service.name", "kafka"); - consumerConfig.put("auto.offset.reset", "earliest"); - consumerConfig.put("sasl.jaas.config", saslJaasConfig); - - PCollection count = - pRead - .apply( - "Read from Kafka", - KafkaIO.read() - .withBootstrapServers(bootstrapServer) - .withConsumerConfigUpdates(consumerConfig) - .withTopic(topicName) - .withKeyDeserializer(StringDeserializer.class) - .withValueDeserializer(StringDeserializer.class) - .withConsumerFactoryFn( - new KerberosConsumerFactoryFn( - "gs://fozzie_testing_bucket/kerberos/krb5.conf")) - .withMaxNumRecords((long) 11)) - .apply("Counting element", Count.globally()); - - PipelineResult readResult = pRead.run(); - - // Only waiting 5 seconds here because we don't expect any processing at this point - PipelineResult.State readState = readResult.waitUntilFinish(Duration.standardMinutes(15)); - cancelIfTimeouted(readResult, readState); - assertNotEquals(PipelineResult.State.FAILED, readState); - assertNotEquals(PipelineResult.State.CANCELLED, readState); - PAssert.thatSingleton(count).isEqualTo((long) 11); - } - @Ignore( "Test is ignored until GMK is utilized as part of this test suite (https://github.com/apache/beam/issues/32721).") @Test @@ -1326,7 +1239,6 @@ private static void setupKafkaContainer() { options.setKafkaBootstrapServerAddresses(kafkaContainer.getBootstrapServers()); } - @Ignore @Test public void testCustomRowDeserializerWithViaSDF() throws IOException { // This test verifies that the SDF implementation of KafkaIO correctly handles From 68230afd3893a872a50d162e83c80fab0c30585b Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Wed, 27 Aug 2025 23:37:04 +0000 Subject: [PATCH 03/14] Add tests for file aware factory fn --- .../beam/sdk/io/kafka/FileAwareFactoryFn.java | 239 +++++++++++++++ .../io/kafka/KerberosConsumerFactoryFn.java | 158 ++++++++++ .../sdk/io/kafka/FileAwareFactoryFnTest.java | 275 ++++++++++++++++++ .../kafka/KerberosConsumerFactoryFnTest.java | 153 ++++++++++ 4 files changed, 825 insertions(+) create mode 100644 sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java create mode 100644 sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java create mode 100644 sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFnTest.java create mode 100644 sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFnTest.java diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java new file mode 100644 index 000000000000..3ce30a8c7463 --- /dev/null +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java @@ -0,0 +1,239 @@ +/* + * 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.beam.sdk.io.kafka; + +/* + * 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. + */ + +import com.google.cloud.secretmanager.v1.AccessSecretVersionResponse; +import com.google.cloud.secretmanager.v1.SecretManagerServiceClient; +import com.google.cloud.secretmanager.v1.SecretVersionName; +import com.google.common.base.Preconditions; +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.channels.ReadableByteChannel; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class FileAwareFactoryFn + implements SerializableFunction, T> { + + public static final String GCS_PATH_PREFIX = "gs://"; + public static final String SECRET_VALUE_PREFIX = "secretValue:"; + public static final String DIRECTORY_PREFIX = "/tmp"; + private static final Pattern PATH_PATTERN = + Pattern.compile("(gs://[^\"]+)|(secretValue:[^\"]+)|(secretFile:[^\"]+)"); + + private static final Map secretCache = new ConcurrentHashMap<>(); + + private final String factoryType; + private static final Logger LOG = LoggerFactory.getLogger(FileAwareFactoryFn.class); + + public FileAwareFactoryFn(String factoryType) { + Preconditions.checkNotNull(factoryType); + this.factoryType = factoryType; + } + + protected abstract T createObject(Map config); + + @Override + public T apply(Map config) { + if (config == null) { + return createObject(config); + } + + Map processedConfig = new HashMap<>(config); + + String key = ""; + Object value = null; + try { + downloadAndProcessExtraFiles(); + + for (Map.Entry e : config.entrySet()) { + try { + key = e.getKey(); + value = e.getValue(); + if (value instanceof String) { + String originalValue = (String) value; + Matcher matcher = PATH_PATTERN.matcher(originalValue); + StringBuffer sb = new StringBuffer(); + + while (matcher.find()) { + String gcsPath = matcher.group(1); + String secretValue = matcher.group(2); + String secretFile = matcher.group(3); + + if (gcsPath != null) { + try { + String tmpPath = replacePathWithLocal(gcsPath); + String localPath = downloadGcsFile(gcsPath, tmpPath); + matcher.appendReplacement(sb, Matcher.quoteReplacement(localPath)); + LOG.info("Downloaded {} to {}", gcsPath, localPath); + } catch (IOException io) { + throw new IOException("Failed to download file : " + gcsPath, io); + } + } else if (secretValue != null) { + try { + String secretId = secretValue.substring(SECRET_VALUE_PREFIX.length()); + String processedSecret = + processSecret(originalValue, secretId, getSecretWithCache(secretId)); + + matcher.appendReplacement(sb, Matcher.quoteReplacement(processedSecret)); + } catch (IllegalArgumentException ia) { + throw new IllegalArgumentException("Failed to get secret.", ia); + } + } else if (secretFile != null) { + throw new UnsupportedOperationException("Not yet implemented."); + } + } + matcher.appendTail(sb); + String processedValue = sb.toString(); + processedConfig.put(key, processedValue); + } + } catch (IOException ex) { + throw new RuntimeException( + "Failed trying to process value " + value + " for key " + key + ".", ex); + } + } + } catch (IOException e) { + throw new RuntimeException("Failed trying to process extra files.", e); + } + + LOG.info("ProcessedConfig: {}", processedConfig); + return createObject(processedConfig); + } + + /** + * A function to download files from their specified gcs path and copy them to the provided local + * filepath. The local filepath is provided by the replacePathWithLocal. + * + * @param gcsFilePath + * @param outputFileString + * @return + * @throws IOException + */ + protected static synchronized String downloadGcsFile(String gcsFilePath, String outputFileString) + throws IOException { + // create the file only if it doesn't exist + if (!new File(outputFileString).exists()) { + Path outputFilePath = Paths.get(outputFileString); + Path parentDir = outputFilePath.getParent(); + if (parentDir != null) { + Files.createDirectories(parentDir); + } + + LOG.info("Staging GCS file [{}] to [{}]", gcsFilePath, outputFileString); + Set options = new HashSet<>(2); + options.add(StandardOpenOption.CREATE); + options.add(StandardOpenOption.WRITE); + + // Copy the GCS file into a local file and will throw an I/O exception in case file not found. + try (ReadableByteChannel readerChannel = + FileSystems.open(FileSystems.matchSingleFileSpec(gcsFilePath).resourceId())) { + try (FileChannel writeChannel = FileChannel.open(outputFilePath, options)) { + writeChannel.transferFrom(readerChannel, 0, Long.MAX_VALUE); + } + } + } + return outputFileString; + } + + protected byte[] getSecretWithCache(String secretId) { + return secretCache.computeIfAbsent(secretId, this::getSecret); + } + + /** + * A helper method to create a new string with the gcs paths replaced with their local path and + * subdirectory based on the factory type in the /tmp directory. For example, the kerberos factory + * type will replace the file paths with /tmp/kerberos/file.path + * + * @param gcsPath + * @return a string with all instances of GCS paths converted to the local paths where the files + * sit. + */ + private String replacePathWithLocal(String gcsPath) throws IOException { + return DIRECTORY_PREFIX + "/" + factoryType + "/" + gcsPath.substring(GCS_PATH_PREFIX.length()); + } + + /** + * @throws IOException A hook for subclasses to download and process specific files before the + * main configuration is handled. For example, the kerberos factory can use this to download a + * krb5.conf and set a system property. + */ + protected void downloadAndProcessExtraFiles() throws IOException { + // Default implementation should do nothing. + } + + protected String getBaseDirectory() { + return DIRECTORY_PREFIX; + } + + protected byte[] getSecret(String secretVersion) { + SecretVersionName secretVersionName; + if (SecretVersionName.isParsableFrom(secretVersion)) { + secretVersionName = SecretVersionName.parse(secretVersion); + } else { + throw new IllegalArgumentException( + "Provided Secret must be in the form" + + " projects/{project}/secrets/{secret}/versions/{secret_version}"); + } + try (SecretManagerServiceClient client = SecretManagerServiceClient.create()) { + AccessSecretVersionResponse response = client.accessSecretVersion(secretVersionName); + return response.getPayload().getData().toByteArray(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + protected String processSecret(String originalValue, String secretId, byte[] secretValue) + throws RuntimeException { + // By Default, this will return the secret value directly. This function can be overridden by + // derived classes. + return new String(secretValue, StandardCharsets.UTF_8); + } +} diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java new file mode 100644 index 000000000000..7a95eb66826a --- /dev/null +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java @@ -0,0 +1,158 @@ +/* + * 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.beam.sdk.io.kafka; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.attribute.PosixFilePermission; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import javax.security.auth.login.Configuration; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class KerberosConsumerFactoryFn extends FileAwareFactoryFn> { + private static final String LOCAL_FACTORY_TYPE = "kerberos"; + private String krb5ConfigGcsPath = ""; + private static volatile String localKrb5ConfPath = ""; + + private static final Object lock = new Object(); + + // Standard Kafka property for SASL JAAS configuration + private static final String JAAS_CONFIG_PROPERTY = "sasl.jaas.config"; + private static final String KEYTAB_SECRET_PREFIX = "keyTab=\"secretValue:"; + private static final Pattern KEYTAB_SECRET_PATTERN = + Pattern.compile("(keyTab=\"secretValue:[^\"]+)"); + + private static final Logger LOG = LoggerFactory.getLogger(KerberosConsumerFactoryFn.class); + + public KerberosConsumerFactoryFn(String krb5ConfigGcsPath) { + super("kerberos"); + this.krb5ConfigGcsPath = krb5ConfigGcsPath; + } + + @Override + protected Consumer createObject(Map config) { + // This will be called after the config map processing has occurred. Therefore, we know that the + // property will have + // had it's value replaced with a local directory. We don't need to worry about the GCS prefix + // in this case. + LOG.info("config when creating the objects: {}", config); + try { + String jaasConfig = (String) config.get(JAAS_CONFIG_PROPERTY); + String localKeytabPath = ""; + if (jaasConfig != null && !jaasConfig.isEmpty()) { + localKeytabPath = + jaasConfig.substring( + jaasConfig.indexOf("keyTab=\"") + 8, jaasConfig.lastIndexOf("\" principal")); + } + + // Set the permissions on the file to be as strict as possible for security reasons. The + // keytab contains + // sensitive information and should be as locked down as possible. + Path path = Paths.get(localKeytabPath); + Set perms = new HashSet<>(); + perms.add(PosixFilePermission.OWNER_READ); + Files.setPosixFilePermissions(path, perms); + } catch (IOException e) { + throw new RuntimeException( + "Could not access keytab file. Make sure that the sasl.jaas.config config property " + + "is set correctly.", + e); + } + return new KafkaConsumer<>(config); + } + + @Override + protected void downloadAndProcessExtraFiles() throws IOException { + synchronized (lock) { + // we only want a new krb5 file if there is not already one present. + if (localKrb5ConfPath.isEmpty()) { + if (this.krb5ConfigGcsPath != null && !this.krb5ConfigGcsPath.isEmpty()) { + String localPath = + super.getBaseDirectory() + "/" + LOCAL_FACTORY_TYPE + "/" + "krb5.conf"; + localKrb5ConfPath = downloadGcsFile(this.krb5ConfigGcsPath, localPath); + + System.setProperty("java.security.krb5.conf", localKrb5ConfPath); + Configuration.getConfiguration().refresh(); + LOG.info( + "Successfully set and refreshed java.security.krb5.conf to {}", localKrb5ConfPath); + } + } + } + } + + @Override + protected String processSecret(String originalValue, String secretId, byte[] secretValue) + throws RuntimeException { + Matcher matcher = KEYTAB_SECRET_PATTERN.matcher(originalValue); + String localFileString = ""; + while (matcher.find()) { + String currentSecretId = matcher.group(1); + if (currentSecretId == null || currentSecretId.isEmpty()) { + throw new RuntimeException( + "Error matching values. Secret was discovered but its value is null"); + } + currentSecretId = currentSecretId.substring(KEYTAB_SECRET_PREFIX.length()); + LOG.info("currentSecretId: {} and secretId: {}", currentSecretId, secretId); + if (!currentSecretId.equals(secretId)) { + // A sasl.jaas.config can contain multiple keytabs in one string. Therefore, we must assume + // that there can + // also be multiple keytab secrets in the same string. If the currently matched secret does + // not equal + // the secret that we are processing (passed in via secretId) then we do not want to create + // a keytab file and + // overwrite it. + continue; + } + String filename = "kafka-client-" + UUID.randomUUID().toString() + ".keytab"; + + localFileString = super.getBaseDirectory() + "/" + LOCAL_FACTORY_TYPE + "/" + filename; + Path localFilePath = Paths.get(localFileString); + Path parentDir = localFilePath.getParent(); + try { + if (parentDir != null) { + Files.createDirectories(parentDir); + } + Files.write(localFilePath, secretValue); + if (!new File(localFileString).canRead()) { + LOG.info("The file is not readable"); + } + LOG.info("Successfully wrote file to path: {}", localFilePath); + } catch (IOException e) { + throw new RuntimeException("Unable to create the keytab file for the provided secret."); + } + } + // if no localFile was created, then we can assume that the secret is meant to be kept as a + // value. + LOG.info("LocalFilestring: {}", localFileString); + return localFileString.isEmpty() + ? new String(secretValue, StandardCharsets.UTF_8) + : localFileString; + } +} diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFnTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFnTest.java new file mode 100644 index 000000000000..3c2727e0e7cb --- /dev/null +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFnTest.java @@ -0,0 +1,275 @@ +/* + * 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.beam.sdk.io.kafka; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.IOException; +import java.nio.channels.Channels; +import java.nio.channels.FileChannel; +import java.nio.channels.ReadableByteChannel; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.MockedStatic; + +@RunWith(JUnit4.class) +public class FileAwareFactoryFnTest { + + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private TestFactoryFn factory; + private String baseDir; + private static final String TEST_FACTORY_TYPE = "test-factory"; + + // A concrete implementation for testing the abstract FileAwareFactoryFn + static class TestFactoryFn extends FileAwareFactoryFn { + public TestFactoryFn() { + super(TEST_FACTORY_TYPE); + } + + @Override + protected Object createObject(Map config) { + // Return the processed config for easy assertion + return config; + } + } + + @Before + public void setup() throws IOException { + baseDir = "/tmp/" + TEST_FACTORY_TYPE; + factory = spy(new TestFactoryFn()); + doReturn(baseDir).when(factory).getBaseDirectory(); + } + + @Test + public void testHappyPathReplacesGcsPath() { + // Arrange + String gcsPath = "gs://test-bucket/config-file.json"; + String expectedLocalPath = + FileAwareFactoryFn.DIRECTORY_PREFIX + + "/" + + TEST_FACTORY_TYPE + + "/test-bucket/config-file.json"; + Map config = new HashMap<>(); + config.put("config.file.path", gcsPath); + + // Act & Assert + // Use try-with-resources to manage the scope of the static mock on FileSystems + try (MockedStatic mockedFileSystems = mockStatic(FileSystems.class)) { + // 1. Mock the underlying static FileSystems calls to avoid real network I/O + MatchResult.Metadata metadata = mock(MatchResult.Metadata.class); + ResourceId resourceId = mock(ResourceId.class); + when(metadata.resourceId()).thenReturn(resourceId); + mockedFileSystems.when(() -> FileSystems.matchSingleFileSpec(gcsPath)).thenReturn(metadata); + + // 2. Mock 'open' to return a channel with no data, simulating a successful download + ReadableByteChannel channel = Channels.newChannel(new ByteArrayInputStream(new byte[0])); + mockedFileSystems.when(() -> FileSystems.open(resourceId)).thenReturn(channel); + + // Act + Map processedConfig = (Map) factory.apply(config); + + // Assert + assertEquals(expectedLocalPath, processedConfig.get("config.file.path")); + assertTrue("Local file should have been created", new File(expectedLocalPath).exists()); + } + } + + @Test + public void testApplyFailurePathThrowsRuntimeExceptionOnDownloadFailure() { + // Arrange + String gcsPath = "gs://test-bucket/failing-file.txt"; + Map config = new HashMap<>(); + config.put("critical.file", gcsPath); + + // Mock the static FileSystems.matchSingleFileSpec to throw an exception + try (MockedStatic mockedFileSystems = mockStatic(FileSystems.class)) { + mockedFileSystems + .when(() -> FileSystems.matchSingleFileSpec(gcsPath)) + .thenThrow(new IOException("GCS file not found")); + + // Act & Assert + RuntimeException exception = + assertThrows(RuntimeException.class, () -> factory.apply(config)); + assertTrue(exception.getMessage().contains("Failed trying to process value")); + assertTrue(exception.getCause() instanceof IOException); + assertTrue(exception.getCause().getMessage().contains("Failed to download file")); + } + } + + @Test + public void testApplyHappyPathIgnoresNonGcsValues() { + // Arrange + Map config = new HashMap<>(); + config.put("some.string", "/local/path/file.txt"); + config.put("some.number", 42); + config.put("some.boolean", false); + + // Act + Map processedConfig = (Map) factory.apply(config); + + // Assert + assertEquals(config, processedConfig); + } + + @Test + public void testApplyEdgeCaseMultipleGcsPathsInSingleValue() { + // Arrange + String gcsPath1 = "gs://bucket/keytab.keytab"; + String gcsPath2 = "gs://bucket/trust.jks"; + String originalValue = + "jaas_config keyTab=\"" + gcsPath1 + "\" trustStore=\"" + gcsPath2 + "\""; + + String expectedLocalPath1 = + FileAwareFactoryFn.DIRECTORY_PREFIX + "/" + TEST_FACTORY_TYPE + "/bucket/keytab.keytab"; + String expectedLocalPath2 = + FileAwareFactoryFn.DIRECTORY_PREFIX + "/" + TEST_FACTORY_TYPE + "/bucket/trust.jks"; + String expectedProcessedValue = + "jaas_config keyTab=\"" + + expectedLocalPath1 + + "\" trustStore=\"" + + expectedLocalPath2 + + "\""; + + Map config = new HashMap<>(); + config.put("jaas.config", originalValue); + + try (MockedStatic mockedFileSystems = mockStatic(FileSystems.class)) { + // Mock GCS calls for both paths + mockSuccessfulDownload(mockedFileSystems, gcsPath1); + mockSuccessfulDownload(mockedFileSystems, gcsPath2); + + // Act + Map processedConfig = (Map) factory.apply(config); + + // Assert + assertEquals(expectedProcessedValue, processedConfig.get("jaas.config")); + } + } + + @Test + public void testApplyEdgeCaseLocalFileWriteFails() throws IOException { + // Arrange + String gcsPath = "gs://test-bucket/some-file.txt"; + Map config = new HashMap<>(); + config.put("a.file", gcsPath); + + // Mock GCS part to succeed + try (MockedStatic mockedFileSystems = mockStatic(FileSystems.class); + MockedStatic mockedFileChannel = mockStatic(FileChannel.class)) { + mockSuccessfulDownload(mockedFileSystems, gcsPath); + + // Mock the local file writing part to fail + mockedFileChannel + .when(() -> FileChannel.open(any(Path.class), any(Set.class))) + .thenThrow(new IOException("Permission denied")); + + // Act & Assert + RuntimeException exception = + assertThrows(RuntimeException.class, () -> factory.apply(config)); + assertTrue(exception.getMessage().contains("Failed trying to process value")); + assertTrue(exception.getCause() instanceof IOException); + // Check that the root cause is our "Permission denied" mock + assertTrue(exception.getCause().getCause().getMessage().contains("Permission denied")); + } + } + + @Test + public void testApplyHappyPathResolvesSecretValue() { + // Arrange + String secretVersion = "secretValue:projects/p/secrets/s/versions/v"; + String secretVersionParsed = "projects/p/secrets/s/versions/v"; + String secretValue = "my-secret-password"; + String originalValue = "password=" + secretVersion; + String expectedProcessedValue = "password=" + secretValue; + + Map config = new HashMap<>(); + config.put("db.password", originalValue); + + // FIX: Create an anonymous inner class that extends our TestFactoryFn + // and overrides the resolveSecret method to return a hardcoded value. + // This completely avoids the call to the real getSecret method and its + // final class dependencies. + TestFactoryFn factoryWithMockedSecret = + new TestFactoryFn() { + @Override + public byte[] getSecret(String secretIdentifier) { + // Assert that the correct identifier is passed + assertEquals(secretVersionParsed, secretIdentifier); + // Return a predictable, hardcoded value for the test + return secretValue.getBytes(StandardCharsets.UTF_8); + } + }; + + // Act + @SuppressWarnings("unchecked") + Map processedConfig = + (Map) factoryWithMockedSecret.apply(config); + + // Assert + assertEquals(expectedProcessedValue, processedConfig.get("db.password")); + } + + @Test + public void testApplyFailurePathThrowsExceptionForInvalidSecretFormat() { + // Arrange + String invalidSecret = "secretValue:not-a-valid-secret-path"; + Map config = new HashMap<>(); + config.put("db.password", "password=" + invalidSecret); + + // Act & Assert + // The spy will call the real method here, which will throw an exception + // because the secret path is not parsable. + RuntimeException ex = assertThrows(RuntimeException.class, () -> factory.apply(config)); + assertEquals(IllegalArgumentException.class, ex.getCause().getClass()); + } + + // Helper method to reduce boilerplate in mocking successful GCS downloads + private void mockSuccessfulDownload(MockedStatic mockedFileSystems, String gcsPath) { + MatchResult.Metadata metadata = mock(MatchResult.Metadata.class); + ResourceId resourceId = mock(ResourceId.class); + when(metadata.resourceId()).thenReturn(resourceId); + mockedFileSystems.when(() -> FileSystems.matchSingleFileSpec(eq(gcsPath))).thenReturn(metadata); + + ReadableByteChannel channel = Channels.newChannel(new ByteArrayInputStream(new byte[0])); + mockedFileSystems.when(() -> FileSystems.open(eq(resourceId))).thenReturn(channel); + } +} diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFnTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFnTest.java new file mode 100644 index 000000000000..14f6f5f93756 --- /dev/null +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFnTest.java @@ -0,0 +1,153 @@ +/* + * 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.beam.sdk.io.kafka; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockConstruction; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.stream.Stream; +import javax.security.auth.login.Configuration; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentCaptor; +import org.mockito.MockedConstruction; +import org.mockito.MockedStatic; + +@RunWith(JUnit4.class) +public class KerberosConsumerFactoryFnTest { + + private KerberosConsumerFactoryFn factory; + private String originalKrb5Conf; + private static final String KRB5_GCS_PATH = "gs://sec-bucket/kerberos/krb5.conf"; + private static final String LOCAL_FACTORY_TYPE = "kerberos"; + + @Before + public void setup() { + // Use reflection to reset the static variable before each test for isolation + try { + java.lang.reflect.Field field = + KerberosConsumerFactoryFn.class.getDeclaredField("localKrb5ConfPath"); + field.setAccessible(true); + field.set(null, ""); + } catch (Exception e) { + throw new RuntimeException(e); + } + + factory = spy(new KerberosConsumerFactoryFn(KRB5_GCS_PATH)); + + // Store original system property to restore it after tests + originalKrb5Conf = System.getProperty("java.security.krb5.conf"); + } + + @After + public void tearDown() throws IOException { + // Clean up system property to avoid affecting other tests + if (originalKrb5Conf != null) { + System.setProperty("java.security.krb5.conf", originalKrb5Conf); + } else { + System.clearProperty("java.security.krb5.conf"); + } + + // Clean up the directory created outside of the JUnit TemporaryFolder rule. + Path pathToDelete = Paths.get(FileAwareFactoryFn.DIRECTORY_PREFIX, LOCAL_FACTORY_TYPE); + if (Files.exists(pathToDelete)) { + try (Stream walk = Files.walk(pathToDelete)) { + walk.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); + } + } + } + + @Test + @SuppressWarnings("rawtypes") + public void testHappyPath() { + // Arrange + String keytabGcsPath = "gs://sec-bucket/keytabs/my.keytab"; + String expectedKrb5LocalPath = "/tmp/kerberos/krb5.conf"; + String expectedKeytabLocalPath = "/tmp/kerberos/sec-bucket/keytabs/my.keytab"; + + Map config = new HashMap<>(); + config.put( + "sasl.jaas.config", + "com.sun.security.auth.module.Krb5LoginModule required keyTab=\"" + + keytabGcsPath + + "\" principal=\"user@REALM\";"); + + // Use try-with-resources for all necessary static mocks + + try (MockedStatic mockedStaticFactory = + mockStatic(FileAwareFactoryFn.class); + MockedStatic mockedConfiguration = mockStatic(Configuration.class); + MockedStatic mockedFiles = mockStatic(Files.class); + MockedConstruction mockedConsumer = mockConstruction(KafkaConsumer.class)) { + + assertNotNull(mockedConsumer); + // Mock the static downloadGcsFile method to prevent any GCS interaction + mockedStaticFactory + .when(() -> FileAwareFactoryFn.downloadGcsFile(KRB5_GCS_PATH, expectedKrb5LocalPath)) + .thenReturn(expectedKrb5LocalPath); + mockedStaticFactory + .when(() -> FileAwareFactoryFn.downloadGcsFile(keytabGcsPath, expectedKeytabLocalPath)) + .thenReturn(expectedKeytabLocalPath); + + // Mock other dependencies to prevent side effects + Configuration mockConf = mock(Configuration.class); + mockedConfiguration.when(Configuration::getConfiguration).thenReturn(mockConf); + mockedFiles + .when(() -> Files.setPosixFilePermissions(any(Path.class), any(Set.class))) + .thenReturn(null); + mockedFiles.when(() -> Files.createDirectories(any(Path.class))).thenReturn(null); + + // Act + factory.apply(config); + + // Assert + // 1. Verify that the krb5.conf system property was set correctly. + assertEquals(expectedKrb5LocalPath, System.getProperty("java.security.krb5.conf")); + + // 2. Capture the config passed to createObject and verify the keytab path was replaced. + ArgumentCaptor> configCaptor = ArgumentCaptor.forClass(Map.class); + verify(factory).createObject(configCaptor.capture()); + Map capturedConfig = configCaptor.getValue(); + String processedJaasConfig = (String) capturedConfig.get("sasl.jaas.config"); + assertTrue(processedJaasConfig.contains("keyTab=\"" + expectedKeytabLocalPath + "\"")); + + // 3. Verify that the JAAS configuration was refreshed. + verify(mockConf).refresh(); + } + } +} From 4084a4aab18ac609f1823ed9b9ca7bdfcb132cd9 Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Fri, 29 Aug 2025 01:08:44 +0000 Subject: [PATCH 04/14] Add changes to the build and integration files for manual testing. Be sure to remove these later as they cannot stay. --- sdks/java/io/kafka/build.gradle | 4 + .../apache/beam/sdk/io/kafka/KafkaIOIT.java | 88 +++++++++++++++++++ 2 files changed, 92 insertions(+) diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index ba25078b64e3..5d4c86c30b84 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -111,6 +111,10 @@ dependencies { testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") kafkaVersions.each {"kafkaVersion$it.key" "org.apache.kafka:kafka-clients:$it.value"} + implementation 'com.google.cloud:google-cloud-secretmanager:2.72.0' + testImplementation project(path: ":runners:google-cloud-dataflow-java") + testImplementation project(path: ":sdks:java:extensions:google-cloud-platform-core") + testImplementation project(path: ":sdks:java:io:google-cloud-platform") } kafkaVersions.each { kv -> diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index 0e8cbd2183ca..0762d4f18304 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -26,10 +26,12 @@ import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.time.Instant; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Random; @@ -38,6 +40,8 @@ import java.util.function.BiFunction; import java.util.stream.Collectors; import java.util.stream.LongStream; +import org.apache.beam.runners.dataflow.DataflowRunner; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -221,6 +225,7 @@ public static void afterClass() { } } + @Ignore @Test public void testKafkaIOFailsFastWithInvalidPartitions() throws IOException { thrown.expect(Pipeline.PipelineExecutionException.class); @@ -245,6 +250,7 @@ public void testKafkaIOFailsFastWithInvalidPartitions() throws IOException { cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIOFailsFastWithInvalidPartitionsAndFlagExplicitlySet() throws IOException { thrown.expect(Pipeline.PipelineExecutionException.class); @@ -271,6 +277,7 @@ public void testKafkaIOFailsFastWithInvalidPartitionsAndFlagExplicitlySet() thro cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIODoesNotFailFastWithInvalidPartitionsAndFlagExplicitlyNotSet() throws IOException { @@ -298,6 +305,7 @@ public void testKafkaIODoesNotFailFastWithInvalidPartitionsAndFlagExplicitlyNotS cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIOFailsFastWithInvalidTopics() throws IOException { // This test will fail on versions before 2.3.0 due to the non-existence of the @@ -332,6 +340,7 @@ public void testKafkaIOFailsFastWithInvalidTopics() throws IOException { cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIOFailsFastWithInvalidTopicsAndDynamicRead() throws IOException { // This test will fail on versions before 2.3.0 due to the non-existence of the @@ -367,6 +376,7 @@ public void testKafkaIOFailsFastWithInvalidTopicsAndDynamicRead() throws IOExcep cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIOLogsTopicVerificationWithDynamicRead() throws IOException { // This test will fail on versions before 2.3.0 due to the non-existence of the @@ -403,6 +413,7 @@ public void testKafkaIOLogsTopicVerificationWithDynamicRead() throws IOException cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIODoesNotErrorAtValidationWithBadBootstrapServer() throws IOException { // expect an error during execution that the bootstrap server is bad, not during validation @@ -428,6 +439,7 @@ public void testKafkaIODoesNotErrorAtValidationWithBadBootstrapServer() throws I cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaIOReadsAndWritesCorrectlyInStreaming() throws IOException { // Use batch pipeline to write records. @@ -470,6 +482,7 @@ public void testKafkaIOReadsAndWritesCorrectlyInStreaming() throws IOException { assertNotEquals(PipelineResult.State.FAILED, readState); } + @Ignore @Test public void testKafkaIOReadsAndWritesCorrectlyInBatch() throws IOException { writePipeline @@ -511,6 +524,7 @@ public void testKafkaIOReadsAndWritesCorrectlyInBatch() throws IOException { // DoFns. CrashOnExtra will throw an exception if we see any extra records beyond those we // expect, and LogFn acts as a sink we can inspect using ExpectedLogs to verify that we got all // those we expect. + @Ignore @Test public void testKafkaIOSDFResumesCorrectly() throws IOException { roundtripElements("first-pass", 4, writePipeline, sdfReadPipeline); @@ -599,6 +613,7 @@ public void processElement(@Element String element, OutputReceiver outpu } // This test verifies that bad data from Kafka is properly sent to the error handler + @Ignore @Test public void testKafkaIOSDFReadWithErrorHandler() throws IOException { // TODO(https://github.com/apache/beam/issues/32704) re-enable when fixed, or remove the support @@ -640,6 +655,7 @@ public void testKafkaIOSDFReadWithErrorHandler() throws IOException { assertNotEquals(PipelineResult.State.FAILED, readState); } + @Ignore @Test public void testKafkaIOWriteWithErrorHandler() throws IOException { @@ -666,6 +682,7 @@ public void testKafkaIOWriteWithErrorHandler() throws IOException { // This test roundtrips a single KV to verify that externalWithMetadata // can handle null keys and values correctly. + @Ignore @Test public void testKafkaIOExternalRoundtripWithMetadataAndNullKeysAndValues() throws IOException { @@ -705,6 +722,7 @@ public void testKafkaIOExternalRoundtripWithMetadataAndNullKeysAndValues() throw cancelIfTimeouted(readResult, readState); } + @Ignore @Test public void testKafkaWithDynamicPartitions() throws IOException { AdminClient client = @@ -792,6 +810,7 @@ public void testKafkaWithDynamicPartitions() throws IOException { } } + @Ignore @Test public void testKafkaWithStopReadingFunction() { AlwaysStopCheckStopReadingFn checkStopReadingFn = new AlwaysStopCheckStopReadingFn(); @@ -806,6 +825,7 @@ public Boolean apply(TopicPartition input) { } } + @Ignore @Test public void testKafkaWithDelayedStopReadingFunction() { DelayedCheckStopReadingFn checkStopReadingFn = new DelayedCheckStopReadingFn(); @@ -853,12 +873,14 @@ public void testKafkaWithDelayedStopReadingFunction() { private static final int FIVE_MINUTES_IN_MS = 5 * 60 * 1000; + @Ignore @Test(timeout = FIVE_MINUTES_IN_MS) public void testKafkaViaManagedSchemaTransformJson() { runReadWriteKafkaViaManagedSchemaTransforms( "JSON", SCHEMA_IN_JSON, JsonUtils.beamSchemaFromJsonSchema(SCHEMA_IN_JSON)); } + @Ignore @Test(timeout = FIVE_MINUTES_IN_MS) public void testKafkaViaManagedSchemaTransformAvro() { runReadWriteKafkaViaManagedSchemaTransforms( @@ -996,6 +1018,7 @@ private void runWithStopReadingFn( readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); } + @Ignore @Test public void testWatermarkUpdateWithSparseMessages() throws IOException, InterruptedException { AdminClient client = @@ -1116,6 +1139,70 @@ public void testReadAndWriteFromKafkaIOWithGCPApplicationDefaultCredentials() th } } + @Test + public void testReadFromKafkaWithKerberosAuthentication() throws IOException { + DataflowPipelineOptions pReadOptions = + PipelineOptionsFactory.create().as(DataflowPipelineOptions.class); + pReadOptions.setAppName("KafkaIOIT-testReadFromKafkaWithKerberosAuthentication"); + List experiments = new ArrayList<>(); + experiments.add("use_sdf_read"); + experiments.add("beam_fn_api"); + experiments.add("use_staged_dataflow_worker_jar"); + // experiments.add("java_worker_options=-Djava.security.krb5.conf=/tmp/krb5.conf"); + pReadOptions.setExperiments(experiments); + pReadOptions.setRunner(DataflowRunner.class); + pReadOptions.setProject("dataflow-testing-311516"); + pReadOptions.setRegion("us-central1"); + pReadOptions.setJobName("testReadFromKafkaWithKerberosAuthentication" + UUID.randomUUID()); + pReadOptions.setNetwork("fozzie-test-vpc"); + pReadOptions.setSubnetwork("regions/us-central1/subnetworks/fozzie-test-vpc-subnet"); + + Pipeline pRead = Pipeline.create(pReadOptions); + String topicName = "fozzie_test_kerberos_topic"; + String bootstrapServer = + "fozzie-test-kafka-broker.us-central1-c.c.dataflow-testing-311516.internal:9092"; + String saslJaasConfig = + "com.sun.security.auth.module.Krb5LoginModule required " + + "useTicketCache=false " + + "useKeyTab=true " + + "storeKey=true " + + "keyTab=\"secretValue:projects/dataflow-testing-311516/secrets/kafka-client-keytab/versions/latest\" " + + "principal=\"kafka-client@US-CENTRAL1-B.C.DATAFLOW-TESTING-311516.INTERNAL\";"; + + Map consumerConfig = new HashMap<>(); + + consumerConfig.put("security.protocol", "SASL_PLAINTEXT"); + consumerConfig.put("sasl.mechanism", "GSSAPI"); + consumerConfig.put("sasl.kerberos.service.name", "kafka"); + consumerConfig.put("auto.offset.reset", "earliest"); + consumerConfig.put("sasl.jaas.config", saslJaasConfig); + + PCollection count = + pRead + .apply( + "Read from Kafka", + KafkaIO.read() + .withBootstrapServers(bootstrapServer) + .withConsumerConfigUpdates(consumerConfig) + .withTopic(topicName) + .withKeyDeserializer(StringDeserializer.class) + .withValueDeserializer(StringDeserializer.class) + .withConsumerFactoryFn( + new KerberosConsumerFactoryFn( + "gs://fozzie_testing_bucket/kerberos/krb5.conf")) + .withMaxNumRecords((long) 11)) + .apply("Counting element", Count.globally()); + + PipelineResult readResult = pRead.run(); + + // Only waiting 5 seconds here because we don't expect any processing at this point + PipelineResult.State readState = readResult.waitUntilFinish(Duration.standardMinutes(15)); + cancelIfTimeouted(readResult, readState); + assertNotEquals(PipelineResult.State.FAILED, readState); + assertNotEquals(PipelineResult.State.CANCELLED, readState); + PAssert.thatSingleton(count).isEqualTo((long) 11); + } + private static class KeyByPartition extends DoFn, KV>> { @@ -1239,6 +1326,7 @@ private static void setupKafkaContainer() { options.setKafkaBootstrapServerAddresses(kafkaContainer.getBootstrapServers()); } + @Ignore @Test public void testCustomRowDeserializerWithViaSDF() throws IOException { // This test verifies that the SDF implementation of KafkaIO correctly handles From d866b94f8bd7e0c47f9ae3dab29930040f00dba4 Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Tue, 9 Sep 2025 15:19:45 +0000 Subject: [PATCH 05/14] Migrate to a new module such that kafka remains GCP Agnostic. --- .../kafka/file-aware-factories/build.gradle | 42 +++++++++ .../aware/factories}/FileAwareFactoryFn.java | 8 +- .../factories}/KerberosConsumerFactoryFn.java | 2 +- .../factories}/FileAwareFactoryFnTest.java | 88 +++++++++---------- .../KerberosConsumerFactoryFnTest.java | 45 +++++----- .../apache/beam/sdk/io/kafka/KafkaIOIT.java | 88 ------------------- settings.gradle.kts | 1 + 7 files changed, 115 insertions(+), 159 deletions(-) create mode 100644 sdks/java/io/kafka/file-aware-factories/build.gradle rename sdks/java/io/kafka/{src/main/java/org/apache/beam/sdk/io/kafka => file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories}/FileAwareFactoryFn.java (97%) rename sdks/java/io/kafka/{src/main/java/org/apache/beam/sdk/io/kafka => file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories}/KerberosConsumerFactoryFn.java (99%) rename sdks/java/io/kafka/{src/test/java/org/apache/beam/sdk/io/kafka => file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories}/FileAwareFactoryFnTest.java (73%) rename sdks/java/io/kafka/{src/test/java/org/apache/beam/sdk/io/kafka => file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories}/KerberosConsumerFactoryFnTest.java (79%) diff --git a/sdks/java/io/kafka/file-aware-factories/build.gradle b/sdks/java/io/kafka/file-aware-factories/build.gradle new file mode 100644 index 000000000000..8fbf7e117114 --- /dev/null +++ b/sdks/java/io/kafka/file-aware-factories/build.gradle @@ -0,0 +1,42 @@ +/* + * 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. + */ + +plugins { id 'org.apache.beam.module' } +applyJavaNature( + automaticModuleName: 'org.apache.beam.sdk.io.kafka.file.aware.factories', + publish: 'False' +) + +description = "Apache Beam :: SDKs :: Java :: IO :: Kafka :: File :: Aware :: Factories" +ext.summary = "Library to instantiate kafka clients with files from GCS or SecretManager." + +dependencies { + // ------------------------- CORE DEPENDENCIES ------------------------- + implementation project(path: ":sdks:java:core", configuration: "shadow") + provided library.java.kafka_clients + implementation 'com.google.cloud:google-cloud-secretmanager:2.72.0' + implementation library.java.slf4j_api + implementation library.java.vendored_guava_32_1_2_jre + + // ------------------------- TEST DEPENDENCIES ------------------------- + testImplementation 'org.apache.kafka:kafka-clients:3.9.0' + testImplementation library.java.junit + testImplementation library.java.mockito_core + testRuntimeOnly library.java.mockito_inline + testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") +} diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFn.java similarity index 97% rename from sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java rename to sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFn.java index 3ce30a8c7463..2c70b3328088 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFn.java +++ b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFn.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.kafka; +package org.apache.beam.sdk.io.kafka.file.aware.factories; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -196,9 +196,9 @@ protected byte[] getSecretWithCache(String secretId) { * @return a string with all instances of GCS paths converted to the local paths where the files * sit. */ - private String replacePathWithLocal(String gcsPath) throws IOException { - return DIRECTORY_PREFIX + "/" + factoryType + "/" + gcsPath.substring(GCS_PATH_PREFIX.length()); - } + private String replacePathWithLocal(String gcsPath) throws IOException { + return DIRECTORY_PREFIX + "/" + factoryType + "/" + gcsPath.substring(GCS_PATH_PREFIX.length()); + } /** * @throws IOException A hook for subclasses to download and process specific files before the diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFn.java similarity index 99% rename from sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java rename to sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFn.java index 7a95eb66826a..2bb884049195 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFn.java +++ b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFn.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.kafka; +package org.apache.beam.sdk.io.kafka.file.aware.factories; import java.io.File; import java.io.IOException; diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFnTest.java b/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java similarity index 73% rename from sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFnTest.java rename to sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java index 3c2727e0e7cb..f0813f85f947 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/FileAwareFactoryFnTest.java +++ b/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java @@ -15,18 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.kafka; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.mockStatic; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.when; +package org.apache.beam.sdk.io.kafka.file.aware.factories; import java.io.ByteArrayInputStream; import java.io.File; @@ -42,13 +31,16 @@ import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.io.fs.ResourceId; +import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.mockito.ArgumentMatchers; import org.mockito.MockedStatic; +import org.mockito.Mockito; @RunWith(JUnit4.class) public class FileAwareFactoryFnTest { @@ -75,8 +67,8 @@ protected Object createObject(Map config) { @Before public void setup() throws IOException { baseDir = "/tmp/" + TEST_FACTORY_TYPE; - factory = spy(new TestFactoryFn()); - doReturn(baseDir).when(factory).getBaseDirectory(); + factory = Mockito.spy(new TestFactoryFn()); + Mockito.doReturn(baseDir).when(factory).getBaseDirectory(); } @Test @@ -93,11 +85,11 @@ public void testHappyPathReplacesGcsPath() { // Act & Assert // Use try-with-resources to manage the scope of the static mock on FileSystems - try (MockedStatic mockedFileSystems = mockStatic(FileSystems.class)) { + try (MockedStatic mockedFileSystems = Mockito.mockStatic(FileSystems.class)) { // 1. Mock the underlying static FileSystems calls to avoid real network I/O - MatchResult.Metadata metadata = mock(MatchResult.Metadata.class); - ResourceId resourceId = mock(ResourceId.class); - when(metadata.resourceId()).thenReturn(resourceId); + MatchResult.Metadata metadata = Mockito.mock(MatchResult.Metadata.class); + ResourceId resourceId = Mockito.mock(ResourceId.class); + Mockito.when(metadata.resourceId()).thenReturn(resourceId); mockedFileSystems.when(() -> FileSystems.matchSingleFileSpec(gcsPath)).thenReturn(metadata); // 2. Mock 'open' to return a channel with no data, simulating a successful download @@ -108,8 +100,9 @@ public void testHappyPathReplacesGcsPath() { Map processedConfig = (Map) factory.apply(config); // Assert - assertEquals(expectedLocalPath, processedConfig.get("config.file.path")); - assertTrue("Local file should have been created", new File(expectedLocalPath).exists()); + Assert.assertEquals(expectedLocalPath, processedConfig.get("config.file.path")); + Assert.assertTrue( + "Local file should have been created", new File(expectedLocalPath).exists()); } } @@ -121,17 +114,17 @@ public void testApplyFailurePathThrowsRuntimeExceptionOnDownloadFailure() { config.put("critical.file", gcsPath); // Mock the static FileSystems.matchSingleFileSpec to throw an exception - try (MockedStatic mockedFileSystems = mockStatic(FileSystems.class)) { + try (MockedStatic mockedFileSystems = Mockito.mockStatic(FileSystems.class)) { mockedFileSystems .when(() -> FileSystems.matchSingleFileSpec(gcsPath)) .thenThrow(new IOException("GCS file not found")); // Act & Assert RuntimeException exception = - assertThrows(RuntimeException.class, () -> factory.apply(config)); - assertTrue(exception.getMessage().contains("Failed trying to process value")); - assertTrue(exception.getCause() instanceof IOException); - assertTrue(exception.getCause().getMessage().contains("Failed to download file")); + Assert.assertThrows(RuntimeException.class, () -> factory.apply(config)); + Assert.assertTrue(exception.getMessage().contains("Failed trying to process value")); + Assert.assertTrue(exception.getCause() instanceof IOException); + Assert.assertTrue(exception.getCause().getMessage().contains("Failed to download file")); } } @@ -147,7 +140,7 @@ public void testApplyHappyPathIgnoresNonGcsValues() { Map processedConfig = (Map) factory.apply(config); // Assert - assertEquals(config, processedConfig); + Assert.assertEquals(config, processedConfig); } @Test @@ -172,7 +165,7 @@ public void testApplyEdgeCaseMultipleGcsPathsInSingleValue() { Map config = new HashMap<>(); config.put("jaas.config", originalValue); - try (MockedStatic mockedFileSystems = mockStatic(FileSystems.class)) { + try (MockedStatic mockedFileSystems = Mockito.mockStatic(FileSystems.class)) { // Mock GCS calls for both paths mockSuccessfulDownload(mockedFileSystems, gcsPath1); mockSuccessfulDownload(mockedFileSystems, gcsPath2); @@ -181,7 +174,7 @@ public void testApplyEdgeCaseMultipleGcsPathsInSingleValue() { Map processedConfig = (Map) factory.apply(config); // Assert - assertEquals(expectedProcessedValue, processedConfig.get("jaas.config")); + Assert.assertEquals(expectedProcessedValue, processedConfig.get("jaas.config")); } } @@ -193,22 +186,25 @@ public void testApplyEdgeCaseLocalFileWriteFails() throws IOException { config.put("a.file", gcsPath); // Mock GCS part to succeed - try (MockedStatic mockedFileSystems = mockStatic(FileSystems.class); - MockedStatic mockedFileChannel = mockStatic(FileChannel.class)) { + try (MockedStatic mockedFileSystems = Mockito.mockStatic(FileSystems.class); + MockedStatic mockedFileChannel = Mockito.mockStatic(FileChannel.class)) { mockSuccessfulDownload(mockedFileSystems, gcsPath); // Mock the local file writing part to fail mockedFileChannel - .when(() -> FileChannel.open(any(Path.class), any(Set.class))) + .when( + () -> + FileChannel.open( + ArgumentMatchers.any(Path.class), ArgumentMatchers.any(Set.class))) .thenThrow(new IOException("Permission denied")); // Act & Assert RuntimeException exception = - assertThrows(RuntimeException.class, () -> factory.apply(config)); - assertTrue(exception.getMessage().contains("Failed trying to process value")); - assertTrue(exception.getCause() instanceof IOException); + Assert.assertThrows(RuntimeException.class, () -> factory.apply(config)); + Assert.assertTrue(exception.getMessage().contains("Failed trying to process value")); + Assert.assertTrue(exception.getCause() instanceof IOException); // Check that the root cause is our "Permission denied" mock - assertTrue(exception.getCause().getCause().getMessage().contains("Permission denied")); + Assert.assertTrue(exception.getCause().getCause().getMessage().contains("Permission denied")); } } @@ -233,7 +229,7 @@ public void testApplyHappyPathResolvesSecretValue() { @Override public byte[] getSecret(String secretIdentifier) { // Assert that the correct identifier is passed - assertEquals(secretVersionParsed, secretIdentifier); + Assert.assertEquals(secretVersionParsed, secretIdentifier); // Return a predictable, hardcoded value for the test return secretValue.getBytes(StandardCharsets.UTF_8); } @@ -245,7 +241,7 @@ public byte[] getSecret(String secretIdentifier) { (Map) factoryWithMockedSecret.apply(config); // Assert - assertEquals(expectedProcessedValue, processedConfig.get("db.password")); + Assert.assertEquals(expectedProcessedValue, processedConfig.get("db.password")); } @Test @@ -258,18 +254,22 @@ public void testApplyFailurePathThrowsExceptionForInvalidSecretFormat() { // Act & Assert // The spy will call the real method here, which will throw an exception // because the secret path is not parsable. - RuntimeException ex = assertThrows(RuntimeException.class, () -> factory.apply(config)); - assertEquals(IllegalArgumentException.class, ex.getCause().getClass()); + RuntimeException ex = Assert.assertThrows(RuntimeException.class, () -> factory.apply(config)); + Assert.assertEquals(IllegalArgumentException.class, ex.getCause().getClass()); } // Helper method to reduce boilerplate in mocking successful GCS downloads private void mockSuccessfulDownload(MockedStatic mockedFileSystems, String gcsPath) { - MatchResult.Metadata metadata = mock(MatchResult.Metadata.class); - ResourceId resourceId = mock(ResourceId.class); - when(metadata.resourceId()).thenReturn(resourceId); - mockedFileSystems.when(() -> FileSystems.matchSingleFileSpec(eq(gcsPath))).thenReturn(metadata); + MatchResult.Metadata metadata = Mockito.mock(MatchResult.Metadata.class); + ResourceId resourceId = Mockito.mock(ResourceId.class); + Mockito.when(metadata.resourceId()).thenReturn(resourceId); + mockedFileSystems + .when(() -> FileSystems.matchSingleFileSpec(ArgumentMatchers.eq(gcsPath))) + .thenReturn(metadata); ReadableByteChannel channel = Channels.newChannel(new ByteArrayInputStream(new byte[0])); - mockedFileSystems.when(() -> FileSystems.open(eq(resourceId))).thenReturn(channel); + mockedFileSystems + .when(() -> FileSystems.open(ArgumentMatchers.eq(resourceId))) + .thenReturn(channel); } } diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFnTest.java b/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java similarity index 79% rename from sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFnTest.java rename to sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java index 14f6f5f93756..04587a353417 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KerberosConsumerFactoryFnTest.java +++ b/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java @@ -15,17 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.kafka; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.mockConstruction; -import static org.mockito.Mockito.mockStatic; +package org.apache.beam.sdk.io.kafka.file.aware.factories; + import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.verify; import java.io.File; import java.io.IOException; @@ -40,13 +32,16 @@ import javax.security.auth.login.Configuration; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.mockito.ArgumentCaptor; +import org.mockito.ArgumentMatchers; import org.mockito.MockedConstruction; import org.mockito.MockedStatic; +import org.mockito.Mockito; @RunWith(JUnit4.class) public class KerberosConsumerFactoryFnTest { @@ -110,12 +105,13 @@ public void testHappyPath() { // Use try-with-resources for all necessary static mocks try (MockedStatic mockedStaticFactory = - mockStatic(FileAwareFactoryFn.class); - MockedStatic mockedConfiguration = mockStatic(Configuration.class); - MockedStatic mockedFiles = mockStatic(Files.class); - MockedConstruction mockedConsumer = mockConstruction(KafkaConsumer.class)) { + Mockito.mockStatic(FileAwareFactoryFn.class); + MockedStatic mockedConfiguration = Mockito.mockStatic(Configuration.class); + MockedStatic mockedFiles = Mockito.mockStatic(Files.class); + MockedConstruction mockedConsumer = + Mockito.mockConstruction(KafkaConsumer.class)) { - assertNotNull(mockedConsumer); + Assert.assertNotNull(mockedConsumer); // Mock the static downloadGcsFile method to prevent any GCS interaction mockedStaticFactory .when(() -> FileAwareFactoryFn.downloadGcsFile(KRB5_GCS_PATH, expectedKrb5LocalPath)) @@ -125,29 +121,34 @@ public void testHappyPath() { .thenReturn(expectedKeytabLocalPath); // Mock other dependencies to prevent side effects - Configuration mockConf = mock(Configuration.class); + Configuration mockConf = Mockito.mock(Configuration.class); mockedConfiguration.when(Configuration::getConfiguration).thenReturn(mockConf); mockedFiles - .when(() -> Files.setPosixFilePermissions(any(Path.class), any(Set.class))) + .when( + () -> + Files.setPosixFilePermissions( + ArgumentMatchers.any(Path.class), ArgumentMatchers.any(Set.class))) + .thenReturn(null); + mockedFiles + .when(() -> Files.createDirectories(ArgumentMatchers.any(Path.class))) .thenReturn(null); - mockedFiles.when(() -> Files.createDirectories(any(Path.class))).thenReturn(null); // Act factory.apply(config); // Assert // 1. Verify that the krb5.conf system property was set correctly. - assertEquals(expectedKrb5LocalPath, System.getProperty("java.security.krb5.conf")); + Assert.assertEquals(expectedKrb5LocalPath, System.getProperty("java.security.krb5.conf")); // 2. Capture the config passed to createObject and verify the keytab path was replaced. ArgumentCaptor> configCaptor = ArgumentCaptor.forClass(Map.class); - verify(factory).createObject(configCaptor.capture()); + Mockito.verify(factory).createObject(configCaptor.capture()); Map capturedConfig = configCaptor.getValue(); String processedJaasConfig = (String) capturedConfig.get("sasl.jaas.config"); - assertTrue(processedJaasConfig.contains("keyTab=\"" + expectedKeytabLocalPath + "\"")); + Assert.assertTrue(processedJaasConfig.contains("keyTab=\"" + expectedKeytabLocalPath + "\"")); // 3. Verify that the JAAS configuration was refreshed. - verify(mockConf).refresh(); + Mockito.verify(mockConf).refresh(); } } } diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index 0762d4f18304..0e8cbd2183ca 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -26,12 +26,10 @@ import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.time.Instant; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Random; @@ -40,8 +38,6 @@ import java.util.function.BiFunction; import java.util.stream.Collectors; import java.util.stream.LongStream; -import org.apache.beam.runners.dataflow.DataflowRunner; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -225,7 +221,6 @@ public static void afterClass() { } } - @Ignore @Test public void testKafkaIOFailsFastWithInvalidPartitions() throws IOException { thrown.expect(Pipeline.PipelineExecutionException.class); @@ -250,7 +245,6 @@ public void testKafkaIOFailsFastWithInvalidPartitions() throws IOException { cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIOFailsFastWithInvalidPartitionsAndFlagExplicitlySet() throws IOException { thrown.expect(Pipeline.PipelineExecutionException.class); @@ -277,7 +271,6 @@ public void testKafkaIOFailsFastWithInvalidPartitionsAndFlagExplicitlySet() thro cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIODoesNotFailFastWithInvalidPartitionsAndFlagExplicitlyNotSet() throws IOException { @@ -305,7 +298,6 @@ public void testKafkaIODoesNotFailFastWithInvalidPartitionsAndFlagExplicitlyNotS cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIOFailsFastWithInvalidTopics() throws IOException { // This test will fail on versions before 2.3.0 due to the non-existence of the @@ -340,7 +332,6 @@ public void testKafkaIOFailsFastWithInvalidTopics() throws IOException { cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIOFailsFastWithInvalidTopicsAndDynamicRead() throws IOException { // This test will fail on versions before 2.3.0 due to the non-existence of the @@ -376,7 +367,6 @@ public void testKafkaIOFailsFastWithInvalidTopicsAndDynamicRead() throws IOExcep cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIOLogsTopicVerificationWithDynamicRead() throws IOException { // This test will fail on versions before 2.3.0 due to the non-existence of the @@ -413,7 +403,6 @@ public void testKafkaIOLogsTopicVerificationWithDynamicRead() throws IOException cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIODoesNotErrorAtValidationWithBadBootstrapServer() throws IOException { // expect an error during execution that the bootstrap server is bad, not during validation @@ -439,7 +428,6 @@ public void testKafkaIODoesNotErrorAtValidationWithBadBootstrapServer() throws I cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaIOReadsAndWritesCorrectlyInStreaming() throws IOException { // Use batch pipeline to write records. @@ -482,7 +470,6 @@ public void testKafkaIOReadsAndWritesCorrectlyInStreaming() throws IOException { assertNotEquals(PipelineResult.State.FAILED, readState); } - @Ignore @Test public void testKafkaIOReadsAndWritesCorrectlyInBatch() throws IOException { writePipeline @@ -524,7 +511,6 @@ public void testKafkaIOReadsAndWritesCorrectlyInBatch() throws IOException { // DoFns. CrashOnExtra will throw an exception if we see any extra records beyond those we // expect, and LogFn acts as a sink we can inspect using ExpectedLogs to verify that we got all // those we expect. - @Ignore @Test public void testKafkaIOSDFResumesCorrectly() throws IOException { roundtripElements("first-pass", 4, writePipeline, sdfReadPipeline); @@ -613,7 +599,6 @@ public void processElement(@Element String element, OutputReceiver outpu } // This test verifies that bad data from Kafka is properly sent to the error handler - @Ignore @Test public void testKafkaIOSDFReadWithErrorHandler() throws IOException { // TODO(https://github.com/apache/beam/issues/32704) re-enable when fixed, or remove the support @@ -655,7 +640,6 @@ public void testKafkaIOSDFReadWithErrorHandler() throws IOException { assertNotEquals(PipelineResult.State.FAILED, readState); } - @Ignore @Test public void testKafkaIOWriteWithErrorHandler() throws IOException { @@ -682,7 +666,6 @@ public void testKafkaIOWriteWithErrorHandler() throws IOException { // This test roundtrips a single KV to verify that externalWithMetadata // can handle null keys and values correctly. - @Ignore @Test public void testKafkaIOExternalRoundtripWithMetadataAndNullKeysAndValues() throws IOException { @@ -722,7 +705,6 @@ public void testKafkaIOExternalRoundtripWithMetadataAndNullKeysAndValues() throw cancelIfTimeouted(readResult, readState); } - @Ignore @Test public void testKafkaWithDynamicPartitions() throws IOException { AdminClient client = @@ -810,7 +792,6 @@ public void testKafkaWithDynamicPartitions() throws IOException { } } - @Ignore @Test public void testKafkaWithStopReadingFunction() { AlwaysStopCheckStopReadingFn checkStopReadingFn = new AlwaysStopCheckStopReadingFn(); @@ -825,7 +806,6 @@ public Boolean apply(TopicPartition input) { } } - @Ignore @Test public void testKafkaWithDelayedStopReadingFunction() { DelayedCheckStopReadingFn checkStopReadingFn = new DelayedCheckStopReadingFn(); @@ -873,14 +853,12 @@ public void testKafkaWithDelayedStopReadingFunction() { private static final int FIVE_MINUTES_IN_MS = 5 * 60 * 1000; - @Ignore @Test(timeout = FIVE_MINUTES_IN_MS) public void testKafkaViaManagedSchemaTransformJson() { runReadWriteKafkaViaManagedSchemaTransforms( "JSON", SCHEMA_IN_JSON, JsonUtils.beamSchemaFromJsonSchema(SCHEMA_IN_JSON)); } - @Ignore @Test(timeout = FIVE_MINUTES_IN_MS) public void testKafkaViaManagedSchemaTransformAvro() { runReadWriteKafkaViaManagedSchemaTransforms( @@ -1018,7 +996,6 @@ private void runWithStopReadingFn( readResult.waitUntilFinish(Duration.standardSeconds(options.getReadTimeout())); } - @Ignore @Test public void testWatermarkUpdateWithSparseMessages() throws IOException, InterruptedException { AdminClient client = @@ -1139,70 +1116,6 @@ public void testReadAndWriteFromKafkaIOWithGCPApplicationDefaultCredentials() th } } - @Test - public void testReadFromKafkaWithKerberosAuthentication() throws IOException { - DataflowPipelineOptions pReadOptions = - PipelineOptionsFactory.create().as(DataflowPipelineOptions.class); - pReadOptions.setAppName("KafkaIOIT-testReadFromKafkaWithKerberosAuthentication"); - List experiments = new ArrayList<>(); - experiments.add("use_sdf_read"); - experiments.add("beam_fn_api"); - experiments.add("use_staged_dataflow_worker_jar"); - // experiments.add("java_worker_options=-Djava.security.krb5.conf=/tmp/krb5.conf"); - pReadOptions.setExperiments(experiments); - pReadOptions.setRunner(DataflowRunner.class); - pReadOptions.setProject("dataflow-testing-311516"); - pReadOptions.setRegion("us-central1"); - pReadOptions.setJobName("testReadFromKafkaWithKerberosAuthentication" + UUID.randomUUID()); - pReadOptions.setNetwork("fozzie-test-vpc"); - pReadOptions.setSubnetwork("regions/us-central1/subnetworks/fozzie-test-vpc-subnet"); - - Pipeline pRead = Pipeline.create(pReadOptions); - String topicName = "fozzie_test_kerberos_topic"; - String bootstrapServer = - "fozzie-test-kafka-broker.us-central1-c.c.dataflow-testing-311516.internal:9092"; - String saslJaasConfig = - "com.sun.security.auth.module.Krb5LoginModule required " - + "useTicketCache=false " - + "useKeyTab=true " - + "storeKey=true " - + "keyTab=\"secretValue:projects/dataflow-testing-311516/secrets/kafka-client-keytab/versions/latest\" " - + "principal=\"kafka-client@US-CENTRAL1-B.C.DATAFLOW-TESTING-311516.INTERNAL\";"; - - Map consumerConfig = new HashMap<>(); - - consumerConfig.put("security.protocol", "SASL_PLAINTEXT"); - consumerConfig.put("sasl.mechanism", "GSSAPI"); - consumerConfig.put("sasl.kerberos.service.name", "kafka"); - consumerConfig.put("auto.offset.reset", "earliest"); - consumerConfig.put("sasl.jaas.config", saslJaasConfig); - - PCollection count = - pRead - .apply( - "Read from Kafka", - KafkaIO.read() - .withBootstrapServers(bootstrapServer) - .withConsumerConfigUpdates(consumerConfig) - .withTopic(topicName) - .withKeyDeserializer(StringDeserializer.class) - .withValueDeserializer(StringDeserializer.class) - .withConsumerFactoryFn( - new KerberosConsumerFactoryFn( - "gs://fozzie_testing_bucket/kerberos/krb5.conf")) - .withMaxNumRecords((long) 11)) - .apply("Counting element", Count.globally()); - - PipelineResult readResult = pRead.run(); - - // Only waiting 5 seconds here because we don't expect any processing at this point - PipelineResult.State readState = readResult.waitUntilFinish(Duration.standardMinutes(15)); - cancelIfTimeouted(readResult, readState); - assertNotEquals(PipelineResult.State.FAILED, readState); - assertNotEquals(PipelineResult.State.CANCELLED, readState); - PAssert.thatSingleton(count).isEqualTo((long) 11); - } - private static class KeyByPartition extends DoFn, KV>> { @@ -1326,7 +1239,6 @@ private static void setupKafkaContainer() { options.setKafkaBootstrapServerAddresses(kafkaContainer.getBootstrapServers()); } - @Ignore @Test public void testCustomRowDeserializerWithViaSDF() throws IOException { // This test verifies that the SDF implementation of KafkaIO correctly handles diff --git a/settings.gradle.kts b/settings.gradle.kts index 97facd1e3918..7e599cd38ac9 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -246,6 +246,7 @@ include(":sdks:java:io:json") include(":sdks:java:io:kafka") include(":sdks:java:io:kafka:jmh") include(":sdks:java:io:kafka:upgrade") +include(":sdks:java:io:kafka:file-aware-factories") include(":sdks:java:io:kudu") include(":sdks:java:io:mongodb") include(":sdks:java:io:mqtt") From 5f49b64c0ec4f9e98a0db089f3accd1850282ce0 Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Tue, 9 Sep 2025 16:50:31 +0000 Subject: [PATCH 06/14] Clean up classes for PR review --- sdks/java/io/kafka/build.gradle | 4 ---- .../aware/factories/FileAwareFactoryFn.java | 23 ++----------------- .../factories/KerberosConsumerFactoryFn.java | 18 ++++----------- .../file/aware/factories/package-info.java | 20 ++++++++++++++++ .../factories/FileAwareFactoryFnTest.java | 6 ----- .../KerberosConsumerFactoryFnTest.java | 6 ----- 6 files changed, 27 insertions(+), 50 deletions(-) create mode 100644 sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/package-info.java diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index 5d4c86c30b84..ba25078b64e3 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -111,10 +111,6 @@ dependencies { testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") kafkaVersions.each {"kafkaVersion$it.key" "org.apache.kafka:kafka-clients:$it.value"} - implementation 'com.google.cloud:google-cloud-secretmanager:2.72.0' - testImplementation project(path: ":runners:google-cloud-dataflow-java") - testImplementation project(path: ":sdks:java:extensions:google-cloud-platform-core") - testImplementation project(path: ":sdks:java:io:google-cloud-platform") } kafkaVersions.each { kv -> diff --git a/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFn.java b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFn.java index 2c70b3328088..c5b05d92b843 100644 --- a/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFn.java +++ b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFn.java @@ -17,28 +17,9 @@ */ package org.apache.beam.sdk.io.kafka.file.aware.factories; -/* - * 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. - */ - import com.google.cloud.secretmanager.v1.AccessSecretVersionResponse; import com.google.cloud.secretmanager.v1.SecretManagerServiceClient; import com.google.cloud.secretmanager.v1.SecretVersionName; -import com.google.common.base.Preconditions; import java.io.File; import java.io.IOException; import java.nio.channels.FileChannel; @@ -57,6 +38,7 @@ import java.util.regex.Pattern; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -230,8 +212,7 @@ protected byte[] getSecret(String secretVersion) { } } - protected String processSecret(String originalValue, String secretId, byte[] secretValue) - throws RuntimeException { + protected String processSecret(String originalValue, String secretId, byte[] secretValue) { // By Default, this will return the secret value directly. This function can be overridden by // derived classes. return new String(secretValue, StandardCharsets.UTF_8); diff --git a/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFn.java b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFn.java index 2bb884049195..a1dac34197db 100644 --- a/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFn.java +++ b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFn.java @@ -59,10 +59,8 @@ public KerberosConsumerFactoryFn(String krb5ConfigGcsPath) { @Override protected Consumer createObject(Map config) { // This will be called after the config map processing has occurred. Therefore, we know that the - // property will have - // had it's value replaced with a local directory. We don't need to worry about the GCS prefix - // in this case. - LOG.info("config when creating the objects: {}", config); + // property will have had it's value replaced with a local directory. + // We don't need to worry about the GCS prefix in this case. try { String jaasConfig = (String) config.get(JAAS_CONFIG_PROPERTY); String localKeytabPath = ""; @@ -73,8 +71,7 @@ protected Consumer createObject(Map config) { } // Set the permissions on the file to be as strict as possible for security reasons. The - // keytab contains - // sensitive information and should be as locked down as possible. + // keytab contains sensitive information and should be as locked down as possible. Path path = Paths.get(localKeytabPath); Set perms = new HashSet<>(); perms.add(PosixFilePermission.OWNER_READ); @@ -100,8 +97,6 @@ protected void downloadAndProcessExtraFiles() throws IOException { System.setProperty("java.security.krb5.conf", localKrb5ConfPath); Configuration.getConfiguration().refresh(); - LOG.info( - "Successfully set and refreshed java.security.krb5.conf to {}", localKrb5ConfPath); } } } @@ -119,15 +114,13 @@ protected String processSecret(String originalValue, String secretId, byte[] sec "Error matching values. Secret was discovered but its value is null"); } currentSecretId = currentSecretId.substring(KEYTAB_SECRET_PREFIX.length()); - LOG.info("currentSecretId: {} and secretId: {}", currentSecretId, secretId); if (!currentSecretId.equals(secretId)) { // A sasl.jaas.config can contain multiple keytabs in one string. Therefore, we must assume // that there can // also be multiple keytab secrets in the same string. If the currently matched secret does // not equal // the secret that we are processing (passed in via secretId) then we do not want to create - // a keytab file and - // overwrite it. + // a keytab file and overwrite it. continue; } String filename = "kafka-client-" + UUID.randomUUID().toString() + ".keytab"; @@ -141,7 +134,7 @@ protected String processSecret(String originalValue, String secretId, byte[] sec } Files.write(localFilePath, secretValue); if (!new File(localFileString).canRead()) { - LOG.info("The file is not readable"); + LOG.warn("The file is not readable"); } LOG.info("Successfully wrote file to path: {}", localFilePath); } catch (IOException e) { @@ -150,7 +143,6 @@ protected String processSecret(String originalValue, String secretId, byte[] sec } // if no localFile was created, then we can assume that the secret is meant to be kept as a // value. - LOG.info("LocalFilestring: {}", localFileString); return localFileString.isEmpty() ? new String(secretValue, StandardCharsets.UTF_8) : localFileString; diff --git a/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/package-info.java b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/package-info.java new file mode 100644 index 000000000000..885134dc578e --- /dev/null +++ b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** ConsumerFactoryFns for file paths that exist in GCS or Google SecretManager. */ +package org.apache.beam.sdk.io.kafka.file.aware.factories; diff --git a/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java b/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java index f0813f85f947..4f3ce7989502 100644 --- a/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java +++ b/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java @@ -220,10 +220,6 @@ public void testApplyHappyPathResolvesSecretValue() { Map config = new HashMap<>(); config.put("db.password", originalValue); - // FIX: Create an anonymous inner class that extends our TestFactoryFn - // and overrides the resolveSecret method to return a hardcoded value. - // This completely avoids the call to the real getSecret method and its - // final class dependencies. TestFactoryFn factoryWithMockedSecret = new TestFactoryFn() { @Override @@ -252,8 +248,6 @@ public void testApplyFailurePathThrowsExceptionForInvalidSecretFormat() { config.put("db.password", "password=" + invalidSecret); // Act & Assert - // The spy will call the real method here, which will throw an exception - // because the secret path is not parsable. RuntimeException ex = Assert.assertThrows(RuntimeException.class, () -> factory.apply(config)); Assert.assertEquals(IllegalArgumentException.class, ex.getCause().getClass()); } diff --git a/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java b/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java index 04587a353417..b52c08137920 100644 --- a/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java +++ b/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java @@ -53,7 +53,6 @@ public class KerberosConsumerFactoryFnTest { @Before public void setup() { - // Use reflection to reset the static variable before each test for isolation try { java.lang.reflect.Field field = KerberosConsumerFactoryFn.class.getDeclaredField("localKrb5ConfPath"); @@ -64,8 +63,6 @@ public void setup() { } factory = spy(new KerberosConsumerFactoryFn(KRB5_GCS_PATH)); - - // Store original system property to restore it after tests originalKrb5Conf = System.getProperty("java.security.krb5.conf"); } @@ -102,8 +99,6 @@ public void testHappyPath() { + keytabGcsPath + "\" principal=\"user@REALM\";"); - // Use try-with-resources for all necessary static mocks - try (MockedStatic mockedStaticFactory = Mockito.mockStatic(FileAwareFactoryFn.class); MockedStatic mockedConfiguration = Mockito.mockStatic(Configuration.class); @@ -120,7 +115,6 @@ public void testHappyPath() { .when(() -> FileAwareFactoryFn.downloadGcsFile(keytabGcsPath, expectedKeytabLocalPath)) .thenReturn(expectedKeytabLocalPath); - // Mock other dependencies to prevent side effects Configuration mockConf = Mockito.mock(Configuration.class); mockedConfiguration.when(Configuration::getConfiguration).thenReturn(mockConf); mockedFiles From 220ba3d64a54d0775c15bf24bae140341e8d1028 Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Wed, 17 Sep 2025 17:54:19 +0000 Subject: [PATCH 07/14] Move the existing module files to the extensions repo. This module will contain the factory functions to be utilized by users and the cross lang expansion service. --- .../factories/FileAwareFactoryFnTest.java | 0 .../KerberosConsumerFactoryFnTest.java | 0 .../kafka/file-aware-factories/build.gradle | 42 ---- .../aware/factories/FileAwareFactoryFn.java | 220 ------------------ .../factories/KerberosConsumerFactoryFn.java | 150 ------------ .../file/aware/factories/package-info.java | 20 -- settings.gradle.kts | 50 ++-- 7 files changed, 25 insertions(+), 457 deletions(-) rename sdks/java/{io/kafka/file-aware-factories => extensions/kafka-factories}/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java (100%) rename sdks/java/{io/kafka/file-aware-factories => extensions/kafka-factories}/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java (100%) delete mode 100644 sdks/java/io/kafka/file-aware-factories/build.gradle delete mode 100644 sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFn.java delete mode 100644 sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFn.java delete mode 100644 sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/package-info.java diff --git a/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java b/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java similarity index 100% rename from sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java rename to sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java diff --git a/sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java b/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java similarity index 100% rename from sdks/java/io/kafka/file-aware-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java rename to sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java diff --git a/sdks/java/io/kafka/file-aware-factories/build.gradle b/sdks/java/io/kafka/file-aware-factories/build.gradle deleted file mode 100644 index 8fbf7e117114..000000000000 --- a/sdks/java/io/kafka/file-aware-factories/build.gradle +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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. - */ - -plugins { id 'org.apache.beam.module' } -applyJavaNature( - automaticModuleName: 'org.apache.beam.sdk.io.kafka.file.aware.factories', - publish: 'False' -) - -description = "Apache Beam :: SDKs :: Java :: IO :: Kafka :: File :: Aware :: Factories" -ext.summary = "Library to instantiate kafka clients with files from GCS or SecretManager." - -dependencies { - // ------------------------- CORE DEPENDENCIES ------------------------- - implementation project(path: ":sdks:java:core", configuration: "shadow") - provided library.java.kafka_clients - implementation 'com.google.cloud:google-cloud-secretmanager:2.72.0' - implementation library.java.slf4j_api - implementation library.java.vendored_guava_32_1_2_jre - - // ------------------------- TEST DEPENDENCIES ------------------------- - testImplementation 'org.apache.kafka:kafka-clients:3.9.0' - testImplementation library.java.junit - testImplementation library.java.mockito_core - testRuntimeOnly library.java.mockito_inline - testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") -} diff --git a/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFn.java b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFn.java deleted file mode 100644 index c5b05d92b843..000000000000 --- a/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFn.java +++ /dev/null @@ -1,220 +0,0 @@ -/* - * 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.beam.sdk.io.kafka.file.aware.factories; - -import com.google.cloud.secretmanager.v1.AccessSecretVersionResponse; -import com.google.cloud.secretmanager.v1.SecretManagerServiceClient; -import com.google.cloud.secretmanager.v1.SecretVersionName; -import java.io.File; -import java.io.IOException; -import java.nio.channels.FileChannel; -import java.nio.channels.ReadableByteChannel; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.nio.file.StandardOpenOption; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.beam.sdk.io.FileSystems; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public abstract class FileAwareFactoryFn - implements SerializableFunction, T> { - - public static final String GCS_PATH_PREFIX = "gs://"; - public static final String SECRET_VALUE_PREFIX = "secretValue:"; - public static final String DIRECTORY_PREFIX = "/tmp"; - private static final Pattern PATH_PATTERN = - Pattern.compile("(gs://[^\"]+)|(secretValue:[^\"]+)|(secretFile:[^\"]+)"); - - private static final Map secretCache = new ConcurrentHashMap<>(); - - private final String factoryType; - private static final Logger LOG = LoggerFactory.getLogger(FileAwareFactoryFn.class); - - public FileAwareFactoryFn(String factoryType) { - Preconditions.checkNotNull(factoryType); - this.factoryType = factoryType; - } - - protected abstract T createObject(Map config); - - @Override - public T apply(Map config) { - if (config == null) { - return createObject(config); - } - - Map processedConfig = new HashMap<>(config); - - String key = ""; - Object value = null; - try { - downloadAndProcessExtraFiles(); - - for (Map.Entry e : config.entrySet()) { - try { - key = e.getKey(); - value = e.getValue(); - if (value instanceof String) { - String originalValue = (String) value; - Matcher matcher = PATH_PATTERN.matcher(originalValue); - StringBuffer sb = new StringBuffer(); - - while (matcher.find()) { - String gcsPath = matcher.group(1); - String secretValue = matcher.group(2); - String secretFile = matcher.group(3); - - if (gcsPath != null) { - try { - String tmpPath = replacePathWithLocal(gcsPath); - String localPath = downloadGcsFile(gcsPath, tmpPath); - matcher.appendReplacement(sb, Matcher.quoteReplacement(localPath)); - LOG.info("Downloaded {} to {}", gcsPath, localPath); - } catch (IOException io) { - throw new IOException("Failed to download file : " + gcsPath, io); - } - } else if (secretValue != null) { - try { - String secretId = secretValue.substring(SECRET_VALUE_PREFIX.length()); - String processedSecret = - processSecret(originalValue, secretId, getSecretWithCache(secretId)); - - matcher.appendReplacement(sb, Matcher.quoteReplacement(processedSecret)); - } catch (IllegalArgumentException ia) { - throw new IllegalArgumentException("Failed to get secret.", ia); - } - } else if (secretFile != null) { - throw new UnsupportedOperationException("Not yet implemented."); - } - } - matcher.appendTail(sb); - String processedValue = sb.toString(); - processedConfig.put(key, processedValue); - } - } catch (IOException ex) { - throw new RuntimeException( - "Failed trying to process value " + value + " for key " + key + ".", ex); - } - } - } catch (IOException e) { - throw new RuntimeException("Failed trying to process extra files.", e); - } - - LOG.info("ProcessedConfig: {}", processedConfig); - return createObject(processedConfig); - } - - /** - * A function to download files from their specified gcs path and copy them to the provided local - * filepath. The local filepath is provided by the replacePathWithLocal. - * - * @param gcsFilePath - * @param outputFileString - * @return - * @throws IOException - */ - protected static synchronized String downloadGcsFile(String gcsFilePath, String outputFileString) - throws IOException { - // create the file only if it doesn't exist - if (!new File(outputFileString).exists()) { - Path outputFilePath = Paths.get(outputFileString); - Path parentDir = outputFilePath.getParent(); - if (parentDir != null) { - Files.createDirectories(parentDir); - } - - LOG.info("Staging GCS file [{}] to [{}]", gcsFilePath, outputFileString); - Set options = new HashSet<>(2); - options.add(StandardOpenOption.CREATE); - options.add(StandardOpenOption.WRITE); - - // Copy the GCS file into a local file and will throw an I/O exception in case file not found. - try (ReadableByteChannel readerChannel = - FileSystems.open(FileSystems.matchSingleFileSpec(gcsFilePath).resourceId())) { - try (FileChannel writeChannel = FileChannel.open(outputFilePath, options)) { - writeChannel.transferFrom(readerChannel, 0, Long.MAX_VALUE); - } - } - } - return outputFileString; - } - - protected byte[] getSecretWithCache(String secretId) { - return secretCache.computeIfAbsent(secretId, this::getSecret); - } - - /** - * A helper method to create a new string with the gcs paths replaced with their local path and - * subdirectory based on the factory type in the /tmp directory. For example, the kerberos factory - * type will replace the file paths with /tmp/kerberos/file.path - * - * @param gcsPath - * @return a string with all instances of GCS paths converted to the local paths where the files - * sit. - */ - private String replacePathWithLocal(String gcsPath) throws IOException { - return DIRECTORY_PREFIX + "/" + factoryType + "/" + gcsPath.substring(GCS_PATH_PREFIX.length()); - } - - /** - * @throws IOException A hook for subclasses to download and process specific files before the - * main configuration is handled. For example, the kerberos factory can use this to download a - * krb5.conf and set a system property. - */ - protected void downloadAndProcessExtraFiles() throws IOException { - // Default implementation should do nothing. - } - - protected String getBaseDirectory() { - return DIRECTORY_PREFIX; - } - - protected byte[] getSecret(String secretVersion) { - SecretVersionName secretVersionName; - if (SecretVersionName.isParsableFrom(secretVersion)) { - secretVersionName = SecretVersionName.parse(secretVersion); - } else { - throw new IllegalArgumentException( - "Provided Secret must be in the form" - + " projects/{project}/secrets/{secret}/versions/{secret_version}"); - } - try (SecretManagerServiceClient client = SecretManagerServiceClient.create()) { - AccessSecretVersionResponse response = client.accessSecretVersion(secretVersionName); - return response.getPayload().getData().toByteArray(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - protected String processSecret(String originalValue, String secretId, byte[] secretValue) { - // By Default, this will return the secret value directly. This function can be overridden by - // derived classes. - return new String(secretValue, StandardCharsets.UTF_8); - } -} diff --git a/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFn.java b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFn.java deleted file mode 100644 index a1dac34197db..000000000000 --- a/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFn.java +++ /dev/null @@ -1,150 +0,0 @@ -/* - * 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.beam.sdk.io.kafka.file.aware.factories; - -import java.io.File; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.nio.file.attribute.PosixFilePermission; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import javax.security.auth.login.Configuration; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class KerberosConsumerFactoryFn extends FileAwareFactoryFn> { - private static final String LOCAL_FACTORY_TYPE = "kerberos"; - private String krb5ConfigGcsPath = ""; - private static volatile String localKrb5ConfPath = ""; - - private static final Object lock = new Object(); - - // Standard Kafka property for SASL JAAS configuration - private static final String JAAS_CONFIG_PROPERTY = "sasl.jaas.config"; - private static final String KEYTAB_SECRET_PREFIX = "keyTab=\"secretValue:"; - private static final Pattern KEYTAB_SECRET_PATTERN = - Pattern.compile("(keyTab=\"secretValue:[^\"]+)"); - - private static final Logger LOG = LoggerFactory.getLogger(KerberosConsumerFactoryFn.class); - - public KerberosConsumerFactoryFn(String krb5ConfigGcsPath) { - super("kerberos"); - this.krb5ConfigGcsPath = krb5ConfigGcsPath; - } - - @Override - protected Consumer createObject(Map config) { - // This will be called after the config map processing has occurred. Therefore, we know that the - // property will have had it's value replaced with a local directory. - // We don't need to worry about the GCS prefix in this case. - try { - String jaasConfig = (String) config.get(JAAS_CONFIG_PROPERTY); - String localKeytabPath = ""; - if (jaasConfig != null && !jaasConfig.isEmpty()) { - localKeytabPath = - jaasConfig.substring( - jaasConfig.indexOf("keyTab=\"") + 8, jaasConfig.lastIndexOf("\" principal")); - } - - // Set the permissions on the file to be as strict as possible for security reasons. The - // keytab contains sensitive information and should be as locked down as possible. - Path path = Paths.get(localKeytabPath); - Set perms = new HashSet<>(); - perms.add(PosixFilePermission.OWNER_READ); - Files.setPosixFilePermissions(path, perms); - } catch (IOException e) { - throw new RuntimeException( - "Could not access keytab file. Make sure that the sasl.jaas.config config property " - + "is set correctly.", - e); - } - return new KafkaConsumer<>(config); - } - - @Override - protected void downloadAndProcessExtraFiles() throws IOException { - synchronized (lock) { - // we only want a new krb5 file if there is not already one present. - if (localKrb5ConfPath.isEmpty()) { - if (this.krb5ConfigGcsPath != null && !this.krb5ConfigGcsPath.isEmpty()) { - String localPath = - super.getBaseDirectory() + "/" + LOCAL_FACTORY_TYPE + "/" + "krb5.conf"; - localKrb5ConfPath = downloadGcsFile(this.krb5ConfigGcsPath, localPath); - - System.setProperty("java.security.krb5.conf", localKrb5ConfPath); - Configuration.getConfiguration().refresh(); - } - } - } - } - - @Override - protected String processSecret(String originalValue, String secretId, byte[] secretValue) - throws RuntimeException { - Matcher matcher = KEYTAB_SECRET_PATTERN.matcher(originalValue); - String localFileString = ""; - while (matcher.find()) { - String currentSecretId = matcher.group(1); - if (currentSecretId == null || currentSecretId.isEmpty()) { - throw new RuntimeException( - "Error matching values. Secret was discovered but its value is null"); - } - currentSecretId = currentSecretId.substring(KEYTAB_SECRET_PREFIX.length()); - if (!currentSecretId.equals(secretId)) { - // A sasl.jaas.config can contain multiple keytabs in one string. Therefore, we must assume - // that there can - // also be multiple keytab secrets in the same string. If the currently matched secret does - // not equal - // the secret that we are processing (passed in via secretId) then we do not want to create - // a keytab file and overwrite it. - continue; - } - String filename = "kafka-client-" + UUID.randomUUID().toString() + ".keytab"; - - localFileString = super.getBaseDirectory() + "/" + LOCAL_FACTORY_TYPE + "/" + filename; - Path localFilePath = Paths.get(localFileString); - Path parentDir = localFilePath.getParent(); - try { - if (parentDir != null) { - Files.createDirectories(parentDir); - } - Files.write(localFilePath, secretValue); - if (!new File(localFileString).canRead()) { - LOG.warn("The file is not readable"); - } - LOG.info("Successfully wrote file to path: {}", localFilePath); - } catch (IOException e) { - throw new RuntimeException("Unable to create the keytab file for the provided secret."); - } - } - // if no localFile was created, then we can assume that the secret is meant to be kept as a - // value. - return localFileString.isEmpty() - ? new String(secretValue, StandardCharsets.UTF_8) - : localFileString; - } -} diff --git a/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/package-info.java b/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/package-info.java deleted file mode 100644 index 885134dc578e..000000000000 --- a/sdks/java/io/kafka/file-aware-factories/src/main/java/org/apache/beam/sdk/io/kafka/file/aware/factories/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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. - */ - -/** ConsumerFactoryFns for file paths that exist in GCS or Google SecretManager. */ -package org.apache.beam.sdk.io.kafka.file.aware.factories; diff --git a/settings.gradle.kts b/settings.gradle.kts index 7e599cd38ac9..7161d0c87af5 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -18,14 +18,14 @@ import com.gradle.enterprise.gradleplugin.internal.extension.BuildScanExtensionWithHiddenFeatures pluginManagement { - plugins { - id("org.javacc.javacc") version "3.0.3" // enable the JavaCC parser generator - } + plugins { + id("org.javacc.javacc") version "3.0.3" // enable the JavaCC parser generator + } } plugins { - id("com.gradle.develocity") version "3.19" - id("com.gradle.common-custom-user-data-gradle-plugin") version "2.4.0" + id("com.gradle.develocity") version "3.19" + id("com.gradle.common-custom-user-data-gradle-plugin") version "2.2.1" } @@ -36,32 +36,32 @@ val isGithubActionsBuild = arrayOf("GITHUB_REPOSITORY", "GITHUB_RUN_ID").all { S val isCi = isJenkinsBuild || isGithubActionsBuild develocity { - server = "https://develocity.apache.org" - projectId = "beam" + server = "https://develocity.apache.org" + projectId = "beam" - buildScan { - uploadInBackground = !isCi - publishing.onlyIf { it.isAuthenticated } - obfuscation { - ipAddresses { addresses -> addresses.map { "0.0.0.0" } } + buildScan { + uploadInBackground = !isCi + publishing.onlyIf { it.isAuthenticated } + obfuscation { + ipAddresses { addresses -> addresses.map { "0.0.0.0" } } + } } - } } buildCache { - local { - isEnabled = true - } - remote { - url = uri("https://beam-cache.apache.org/cache/") - isAllowUntrustedServer = false - credentials { - username = System.getenv("GRADLE_ENTERPRISE_CACHE_USERNAME") - password = System.getenv("GRADLE_ENTERPRISE_CACHE_PASSWORD") + local { + isEnabled = true + } + remote { + url = uri("https://beam-cache.apache.org/cache/") + isAllowUntrustedServer = false + credentials { + username = System.getenv("GRADLE_ENTERPRISE_CACHE_USERNAME") + password = System.getenv("GRADLE_ENTERPRISE_CACHE_PASSWORD") + } + isEnabled = !System.getenv("GRADLE_ENTERPRISE_CACHE_USERNAME").isNullOrBlank() + isPush = isCi && !System.getenv("GRADLE_ENTERPRISE_CACHE_USERNAME").isNullOrBlank() } - isEnabled = !System.getenv("GRADLE_ENTERPRISE_CACHE_USERNAME").isNullOrBlank() - isPush = isCi && !System.getenv("GRADLE_ENTERPRISE_CACHE_USERNAME").isNullOrBlank() - } } rootProject.name = "beam" From 48a084e24b8e9504be4e45a17b86a949398b3a5d Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Tue, 23 Sep 2025 22:34:27 +0000 Subject: [PATCH 08/14] Modify the base class to use GCS client instead of GCS FileSystems. This is a more lightweight dependency for the expansion service. --- .../factories/FileAwareFactoryFnTest.java | 269 ------------------ .../KerberosConsumerFactoryFnTest.java | 148 ---------- 2 files changed, 417 deletions(-) delete mode 100644 sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java delete mode 100644 sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java diff --git a/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java b/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java deleted file mode 100644 index 4f3ce7989502..000000000000 --- a/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java +++ /dev/null @@ -1,269 +0,0 @@ -/* - * 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.beam.sdk.io.kafka.file.aware.factories; - -import java.io.ByteArrayInputStream; -import java.io.File; -import java.io.IOException; -import java.nio.channels.Channels; -import java.nio.channels.FileChannel; -import java.nio.channels.ReadableByteChannel; -import java.nio.charset.StandardCharsets; -import java.nio.file.Path; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import org.apache.beam.sdk.io.FileSystems; -import org.apache.beam.sdk.io.fs.MatchResult; -import org.apache.beam.sdk.io.fs.ResourceId; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.mockito.ArgumentMatchers; -import org.mockito.MockedStatic; -import org.mockito.Mockito; - -@RunWith(JUnit4.class) -public class FileAwareFactoryFnTest { - - @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - - private TestFactoryFn factory; - private String baseDir; - private static final String TEST_FACTORY_TYPE = "test-factory"; - - // A concrete implementation for testing the abstract FileAwareFactoryFn - static class TestFactoryFn extends FileAwareFactoryFn { - public TestFactoryFn() { - super(TEST_FACTORY_TYPE); - } - - @Override - protected Object createObject(Map config) { - // Return the processed config for easy assertion - return config; - } - } - - @Before - public void setup() throws IOException { - baseDir = "/tmp/" + TEST_FACTORY_TYPE; - factory = Mockito.spy(new TestFactoryFn()); - Mockito.doReturn(baseDir).when(factory).getBaseDirectory(); - } - - @Test - public void testHappyPathReplacesGcsPath() { - // Arrange - String gcsPath = "gs://test-bucket/config-file.json"; - String expectedLocalPath = - FileAwareFactoryFn.DIRECTORY_PREFIX - + "/" - + TEST_FACTORY_TYPE - + "/test-bucket/config-file.json"; - Map config = new HashMap<>(); - config.put("config.file.path", gcsPath); - - // Act & Assert - // Use try-with-resources to manage the scope of the static mock on FileSystems - try (MockedStatic mockedFileSystems = Mockito.mockStatic(FileSystems.class)) { - // 1. Mock the underlying static FileSystems calls to avoid real network I/O - MatchResult.Metadata metadata = Mockito.mock(MatchResult.Metadata.class); - ResourceId resourceId = Mockito.mock(ResourceId.class); - Mockito.when(metadata.resourceId()).thenReturn(resourceId); - mockedFileSystems.when(() -> FileSystems.matchSingleFileSpec(gcsPath)).thenReturn(metadata); - - // 2. Mock 'open' to return a channel with no data, simulating a successful download - ReadableByteChannel channel = Channels.newChannel(new ByteArrayInputStream(new byte[0])); - mockedFileSystems.when(() -> FileSystems.open(resourceId)).thenReturn(channel); - - // Act - Map processedConfig = (Map) factory.apply(config); - - // Assert - Assert.assertEquals(expectedLocalPath, processedConfig.get("config.file.path")); - Assert.assertTrue( - "Local file should have been created", new File(expectedLocalPath).exists()); - } - } - - @Test - public void testApplyFailurePathThrowsRuntimeExceptionOnDownloadFailure() { - // Arrange - String gcsPath = "gs://test-bucket/failing-file.txt"; - Map config = new HashMap<>(); - config.put("critical.file", gcsPath); - - // Mock the static FileSystems.matchSingleFileSpec to throw an exception - try (MockedStatic mockedFileSystems = Mockito.mockStatic(FileSystems.class)) { - mockedFileSystems - .when(() -> FileSystems.matchSingleFileSpec(gcsPath)) - .thenThrow(new IOException("GCS file not found")); - - // Act & Assert - RuntimeException exception = - Assert.assertThrows(RuntimeException.class, () -> factory.apply(config)); - Assert.assertTrue(exception.getMessage().contains("Failed trying to process value")); - Assert.assertTrue(exception.getCause() instanceof IOException); - Assert.assertTrue(exception.getCause().getMessage().contains("Failed to download file")); - } - } - - @Test - public void testApplyHappyPathIgnoresNonGcsValues() { - // Arrange - Map config = new HashMap<>(); - config.put("some.string", "/local/path/file.txt"); - config.put("some.number", 42); - config.put("some.boolean", false); - - // Act - Map processedConfig = (Map) factory.apply(config); - - // Assert - Assert.assertEquals(config, processedConfig); - } - - @Test - public void testApplyEdgeCaseMultipleGcsPathsInSingleValue() { - // Arrange - String gcsPath1 = "gs://bucket/keytab.keytab"; - String gcsPath2 = "gs://bucket/trust.jks"; - String originalValue = - "jaas_config keyTab=\"" + gcsPath1 + "\" trustStore=\"" + gcsPath2 + "\""; - - String expectedLocalPath1 = - FileAwareFactoryFn.DIRECTORY_PREFIX + "/" + TEST_FACTORY_TYPE + "/bucket/keytab.keytab"; - String expectedLocalPath2 = - FileAwareFactoryFn.DIRECTORY_PREFIX + "/" + TEST_FACTORY_TYPE + "/bucket/trust.jks"; - String expectedProcessedValue = - "jaas_config keyTab=\"" - + expectedLocalPath1 - + "\" trustStore=\"" - + expectedLocalPath2 - + "\""; - - Map config = new HashMap<>(); - config.put("jaas.config", originalValue); - - try (MockedStatic mockedFileSystems = Mockito.mockStatic(FileSystems.class)) { - // Mock GCS calls for both paths - mockSuccessfulDownload(mockedFileSystems, gcsPath1); - mockSuccessfulDownload(mockedFileSystems, gcsPath2); - - // Act - Map processedConfig = (Map) factory.apply(config); - - // Assert - Assert.assertEquals(expectedProcessedValue, processedConfig.get("jaas.config")); - } - } - - @Test - public void testApplyEdgeCaseLocalFileWriteFails() throws IOException { - // Arrange - String gcsPath = "gs://test-bucket/some-file.txt"; - Map config = new HashMap<>(); - config.put("a.file", gcsPath); - - // Mock GCS part to succeed - try (MockedStatic mockedFileSystems = Mockito.mockStatic(FileSystems.class); - MockedStatic mockedFileChannel = Mockito.mockStatic(FileChannel.class)) { - mockSuccessfulDownload(mockedFileSystems, gcsPath); - - // Mock the local file writing part to fail - mockedFileChannel - .when( - () -> - FileChannel.open( - ArgumentMatchers.any(Path.class), ArgumentMatchers.any(Set.class))) - .thenThrow(new IOException("Permission denied")); - - // Act & Assert - RuntimeException exception = - Assert.assertThrows(RuntimeException.class, () -> factory.apply(config)); - Assert.assertTrue(exception.getMessage().contains("Failed trying to process value")); - Assert.assertTrue(exception.getCause() instanceof IOException); - // Check that the root cause is our "Permission denied" mock - Assert.assertTrue(exception.getCause().getCause().getMessage().contains("Permission denied")); - } - } - - @Test - public void testApplyHappyPathResolvesSecretValue() { - // Arrange - String secretVersion = "secretValue:projects/p/secrets/s/versions/v"; - String secretVersionParsed = "projects/p/secrets/s/versions/v"; - String secretValue = "my-secret-password"; - String originalValue = "password=" + secretVersion; - String expectedProcessedValue = "password=" + secretValue; - - Map config = new HashMap<>(); - config.put("db.password", originalValue); - - TestFactoryFn factoryWithMockedSecret = - new TestFactoryFn() { - @Override - public byte[] getSecret(String secretIdentifier) { - // Assert that the correct identifier is passed - Assert.assertEquals(secretVersionParsed, secretIdentifier); - // Return a predictable, hardcoded value for the test - return secretValue.getBytes(StandardCharsets.UTF_8); - } - }; - - // Act - @SuppressWarnings("unchecked") - Map processedConfig = - (Map) factoryWithMockedSecret.apply(config); - - // Assert - Assert.assertEquals(expectedProcessedValue, processedConfig.get("db.password")); - } - - @Test - public void testApplyFailurePathThrowsExceptionForInvalidSecretFormat() { - // Arrange - String invalidSecret = "secretValue:not-a-valid-secret-path"; - Map config = new HashMap<>(); - config.put("db.password", "password=" + invalidSecret); - - // Act & Assert - RuntimeException ex = Assert.assertThrows(RuntimeException.class, () -> factory.apply(config)); - Assert.assertEquals(IllegalArgumentException.class, ex.getCause().getClass()); - } - - // Helper method to reduce boilerplate in mocking successful GCS downloads - private void mockSuccessfulDownload(MockedStatic mockedFileSystems, String gcsPath) { - MatchResult.Metadata metadata = Mockito.mock(MatchResult.Metadata.class); - ResourceId resourceId = Mockito.mock(ResourceId.class); - Mockito.when(metadata.resourceId()).thenReturn(resourceId); - mockedFileSystems - .when(() -> FileSystems.matchSingleFileSpec(ArgumentMatchers.eq(gcsPath))) - .thenReturn(metadata); - - ReadableByteChannel channel = Channels.newChannel(new ByteArrayInputStream(new byte[0])); - mockedFileSystems - .when(() -> FileSystems.open(ArgumentMatchers.eq(resourceId))) - .thenReturn(channel); - } -} diff --git a/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java b/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java deleted file mode 100644 index b52c08137920..000000000000 --- a/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * 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.beam.sdk.io.kafka.file.aware.factories; - -import static org.mockito.Mockito.spy; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.stream.Stream; -import javax.security.auth.login.Configuration; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.mockito.ArgumentCaptor; -import org.mockito.ArgumentMatchers; -import org.mockito.MockedConstruction; -import org.mockito.MockedStatic; -import org.mockito.Mockito; - -@RunWith(JUnit4.class) -public class KerberosConsumerFactoryFnTest { - - private KerberosConsumerFactoryFn factory; - private String originalKrb5Conf; - private static final String KRB5_GCS_PATH = "gs://sec-bucket/kerberos/krb5.conf"; - private static final String LOCAL_FACTORY_TYPE = "kerberos"; - - @Before - public void setup() { - try { - java.lang.reflect.Field field = - KerberosConsumerFactoryFn.class.getDeclaredField("localKrb5ConfPath"); - field.setAccessible(true); - field.set(null, ""); - } catch (Exception e) { - throw new RuntimeException(e); - } - - factory = spy(new KerberosConsumerFactoryFn(KRB5_GCS_PATH)); - originalKrb5Conf = System.getProperty("java.security.krb5.conf"); - } - - @After - public void tearDown() throws IOException { - // Clean up system property to avoid affecting other tests - if (originalKrb5Conf != null) { - System.setProperty("java.security.krb5.conf", originalKrb5Conf); - } else { - System.clearProperty("java.security.krb5.conf"); - } - - // Clean up the directory created outside of the JUnit TemporaryFolder rule. - Path pathToDelete = Paths.get(FileAwareFactoryFn.DIRECTORY_PREFIX, LOCAL_FACTORY_TYPE); - if (Files.exists(pathToDelete)) { - try (Stream walk = Files.walk(pathToDelete)) { - walk.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); - } - } - } - - @Test - @SuppressWarnings("rawtypes") - public void testHappyPath() { - // Arrange - String keytabGcsPath = "gs://sec-bucket/keytabs/my.keytab"; - String expectedKrb5LocalPath = "/tmp/kerberos/krb5.conf"; - String expectedKeytabLocalPath = "/tmp/kerberos/sec-bucket/keytabs/my.keytab"; - - Map config = new HashMap<>(); - config.put( - "sasl.jaas.config", - "com.sun.security.auth.module.Krb5LoginModule required keyTab=\"" - + keytabGcsPath - + "\" principal=\"user@REALM\";"); - - try (MockedStatic mockedStaticFactory = - Mockito.mockStatic(FileAwareFactoryFn.class); - MockedStatic mockedConfiguration = Mockito.mockStatic(Configuration.class); - MockedStatic mockedFiles = Mockito.mockStatic(Files.class); - MockedConstruction mockedConsumer = - Mockito.mockConstruction(KafkaConsumer.class)) { - - Assert.assertNotNull(mockedConsumer); - // Mock the static downloadGcsFile method to prevent any GCS interaction - mockedStaticFactory - .when(() -> FileAwareFactoryFn.downloadGcsFile(KRB5_GCS_PATH, expectedKrb5LocalPath)) - .thenReturn(expectedKrb5LocalPath); - mockedStaticFactory - .when(() -> FileAwareFactoryFn.downloadGcsFile(keytabGcsPath, expectedKeytabLocalPath)) - .thenReturn(expectedKeytabLocalPath); - - Configuration mockConf = Mockito.mock(Configuration.class); - mockedConfiguration.when(Configuration::getConfiguration).thenReturn(mockConf); - mockedFiles - .when( - () -> - Files.setPosixFilePermissions( - ArgumentMatchers.any(Path.class), ArgumentMatchers.any(Set.class))) - .thenReturn(null); - mockedFiles - .when(() -> Files.createDirectories(ArgumentMatchers.any(Path.class))) - .thenReturn(null); - - // Act - factory.apply(config); - - // Assert - // 1. Verify that the krb5.conf system property was set correctly. - Assert.assertEquals(expectedKrb5LocalPath, System.getProperty("java.security.krb5.conf")); - - // 2. Capture the config passed to createObject and verify the keytab path was replaced. - ArgumentCaptor> configCaptor = ArgumentCaptor.forClass(Map.class); - Mockito.verify(factory).createObject(configCaptor.capture()); - Map capturedConfig = configCaptor.getValue(); - String processedJaasConfig = (String) capturedConfig.get("sasl.jaas.config"); - Assert.assertTrue(processedJaasConfig.contains("keyTab=\"" + expectedKeytabLocalPath + "\"")); - - // 3. Verify that the JAAS configuration was refreshed. - Mockito.verify(mockConf).refresh(); - } - } -} From 4e34e0c5823dcd2caed4c7ea347c3ca714c626ec Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Tue, 9 Sep 2025 15:19:45 +0000 Subject: [PATCH 09/14] Migrate to a new module such that kafka remains GCP Agnostic. --- .../sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java | 0 .../kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java | 0 2 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java create mode 100644 sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java diff --git a/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java b/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java b/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java new file mode 100644 index 000000000000..e69de29bb2d1 From ba188e183625ad3f2e46164ff75d748e0e4b621d Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Wed, 17 Sep 2025 17:54:19 +0000 Subject: [PATCH 10/14] Move the existing module files to the extensions repo. This module will contain the factory functions to be utilized by users and the cross lang expansion service. --- settings.gradle.kts | 1 - 1 file changed, 1 deletion(-) diff --git a/settings.gradle.kts b/settings.gradle.kts index 7161d0c87af5..b83782a29e3f 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -246,7 +246,6 @@ include(":sdks:java:io:json") include(":sdks:java:io:kafka") include(":sdks:java:io:kafka:jmh") include(":sdks:java:io:kafka:upgrade") -include(":sdks:java:io:kafka:file-aware-factories") include(":sdks:java:io:kudu") include(":sdks:java:io:mongodb") include(":sdks:java:io:mqtt") From 43d731c5821180bbc389caa083c124523207b293 Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Fri, 19 Sep 2025 15:44:03 +0000 Subject: [PATCH 11/14] Add plumbing for python use case. --- sdks/java/io/expansion-service/build.gradle | 2 + .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 32 ++++++++++ .../io/external/xlang_kafkaio_it_test.py | 62 ++++++++++++++++++- sdks/python/apache_beam/io/kafka.py | 17 ++++- 4 files changed, 110 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/expansion-service/build.gradle b/sdks/java/io/expansion-service/build.gradle index f1366817db22..dbd6e279846b 100644 --- a/sdks/java/io/expansion-service/build.gradle +++ b/sdks/java/io/expansion-service/build.gradle @@ -76,6 +76,8 @@ dependencies { permitUnusedDeclared project(":sdks:java:io:kafka") // BEAM-11761 implementation project(":sdks:java:io:kafka:upgrade") permitUnusedDeclared project(":sdks:java:io:kafka:upgrade") // BEAM-11761 + implementation project(":sdks:java:extensions:kafka-factories") + permitUnusedDeclared project(":sdks:java:extensions:kafka-factories") if (JavaVersion.current().compareTo(JavaVersion.VERSION_11) >= 0 && project.findProperty('testJavaVersion') != '8') { // iceberg ended support for Java 8 in 1.7.0 diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 568fe49217b3..43b286e87e08 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -27,6 +27,7 @@ import io.confluent.kafka.serializers.KafkaAvroDeserializer; import java.io.InputStream; import java.io.OutputStream; +import java.lang.reflect.Constructor; import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Arrays; @@ -94,6 +95,8 @@ import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.Manual; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.MonotonicallyIncreasing; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.WallTime; +import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.util.construction.PTransformMatchers; import org.apache.beam.sdk.util.construction.ReplacementOutputs; @@ -930,6 +933,25 @@ static void setupExternalBuilder( builder.setOffsetDeduplication(false); builder.setRedistributeByRecordKey(false); } + + if (config.consumerFactoryFnClass != null) { + if (config.consumerFactoryFnClass.contains("KerberosConsumerFactoryFn")) { + try { + if (!config.consumerFactoryFnClass.contains("krb5Location")) { + throw new IllegalArgumentException("The KerberosConsumerFactoryFn requires a location for the krb5.conf file. " + + "Please provide either a GCS location or Google Secret Manager location for this file."); + } + builder.setConsumerFactoryFn(InstanceBuilder.ofType(new TypeDescriptor, Consumer>>() {}) + .fromClassName(config.consumerFactoryFnClass) + .withArg( + String.class, + config.consumerFactoryFnParams.get("krb5Location")) + .build()); + } catch (Exception e) { + throw new RuntimeException("Unable to construct FactoryFn " + config.consumerFactoryFnClass + ": " + e.getMessage(), e); + } + } + } } private static Coder resolveCoder(Class> deserializer) { @@ -1000,6 +1022,8 @@ public static class Configuration { private Boolean offsetDeduplication; private Boolean redistributeByRecordKey; private Long dynamicReadPollIntervalSeconds; + private String consumerFactoryFnClass; + private Map consumerFactoryFnParams; public void setConsumerConfig(Map consumerConfig) { this.consumerConfig = consumerConfig; @@ -1068,6 +1092,14 @@ public void setRedistributeByRecordKey(Boolean redistributeByRecordKey) { public void setDynamicReadPollIntervalSeconds(Long dynamicReadPollIntervalSeconds) { this.dynamicReadPollIntervalSeconds = dynamicReadPollIntervalSeconds; } + + public void setConsumerFactoryFnClass(String consumerFactoryFnClass) { + this.consumerFactoryFnClass = consumerFactoryFnClass; + } + + public void setConsumerFactoryFnParams(Map consumerFactoryFnParams) { + this.consumerFactoryFnParams = consumerFactoryFnParams; + } } } diff --git a/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py b/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py index 23178b0ee363..4b4e145cb2f4 100644 --- a/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py +++ b/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py @@ -41,6 +41,10 @@ from apache_beam.transforms.userstate import BagStateSpec from apache_beam.transforms.userstate import CombiningValueStateSpec from apache_beam.utils import subprocess_server +from apache_beam.options.pipeline_options import ( + PipelineOptions, + GoogleCloudOptions +) NUM_RECORDS = 1000 @@ -68,7 +72,7 @@ def process( class CrossLanguageKafkaIO(object): def __init__( - self, bootstrap_servers, topic, null_key, expansion_service=None): + self, bootstrap_servers=None, topic=None, null_key=None, expansion_service=None): self.bootstrap_servers = bootstrap_servers self.topic = topic self.null_key = null_key @@ -107,6 +111,34 @@ def build_read_pipeline(self, pipeline, max_num_records=None): | 'CalculateSum' >> beam.ParDo(CollectingFn()) | 'SetSumCounter' >> beam.Map(self.sum_counter.inc)) + def build_read_pipeline_with_kerberos(self, p, max_num_records=None): + + jaas_config = ( + f'com.sun.security.auth.module.Krb5LoginModule required ' + f'useKeyTab=true storeKey=true ' + f'keyTab="secretValue:projects/dataflow-testing-311516/secrets/kafka-client-keytab/versions/latest" ' + f'principal="kafka-client@US-CENTRAL1-B.C.DATAFLOW-TESTING-311516.INTERNAL";' + ) + + kafka_records = ( + p + | 'ReadFromKafka' >> ReadFromKafka( + consumer_config={ + 'bootstrap.servers': 'fozzie-test-kafka-broker.us-central1-c.c.dataflow-testing-311516.internal:9092', + 'auto.offset.reset': 'earliest', + 'max_num_records': max_num_records, + 'security.protocol': 'SASL_PLAINTEXT', + 'sasl.mechanism': 'GSSAPI', + 'sasl.kerberos.service.name': 'kafka', + 'sasl.jaas.config': jaas_config + }, + topics=['fozzie_test_kerberos_topic'], + key_deserializer='org.apache.kafka.common.serialization.StringDeserializer', + value_deserializer='org.apache.kafka.common.serialization.StringDeserializer', + consumer_factory_fn_class='org.apache.beam.sdk.extensions.kafka.factories.KerberosConsumerFactoryFn', + consumer_factory_fn_params={'krb5Location': 'gs://fozzie_testing_bucket/kerberos/krb5.conf'})) + return kafka_records + def run_xlang_kafkaio(self, pipeline): self.build_write_pipeline(pipeline) self.build_read_pipeline(pipeline) @@ -182,6 +214,17 @@ def test_hosted_kafkaio_null_key(self): self.run_kafka_write(pipeline_creator) self.run_kafka_read(pipeline_creator, None) + def test_hosted_kafkaio_null_key_kerberos(self): + kafka_topic = 'xlang_kafkaio_test_null_key_{}'.format(uuid.uuid4()) + bootstrap_servers = 'fozzie-test-kafka-broker.us-central1-c.c.dataflow-testing-311516.internal:9092' + pipeline_creator = CrossLanguageKafkaIO( + bootstrap_servers, + kafka_topic, + True, + 'localhost:%s' % os.environ.get('EXPANSION_PORT')) + + self.run_kafka_read_with_kerberos(pipeline_creator) + def run_kafka_write(self, pipeline_creator): with TestPipeline() as pipeline: pipeline.not_use_test_runner_api = True @@ -196,6 +239,23 @@ def run_kafka_read(self, pipeline_creator, expected_key): equal_to([(expected_key, str(i).encode()) for i in range(NUM_RECORDS)])) + def run_kafka_read_with_kerberos(self, pipeline_creator): + options_dict = { + 'runner': 'DataflowRunner', + 'project': 'dataflow-testing-311516', + 'region': 'us-central1', + 'streaming': False + } + options = PipelineOptions.from_dictionary(options_dict) + expected_records = [f'test{i}' for i in range(1, 12)] + with beam.Pipeline(options=options) as p: + pipeline.not_use_test_runner_api = True + result = pipeline_creator.build_read_pipeline_with_kerberos(p, max_num_records=11) + assert_that( + result, + equal_to(expected_records) + ) + def get_platform_localhost(self): if sys.platform == 'darwin': return 'host.docker.internal' diff --git a/sdks/python/apache_beam/io/kafka.py b/sdks/python/apache_beam/io/kafka.py index f3e6c39cfda4..1fbafda26274 100644 --- a/sdks/python/apache_beam/io/kafka.py +++ b/sdks/python/apache_beam/io/kafka.py @@ -124,7 +124,9 @@ ('redistribute', typing.Optional[bool]), ('redistribute_num_keys', typing.Optional[np.int32]), ('allow_duplicates', typing.Optional[bool]), - ('dynamic_read_poll_interval_seconds', typing.Optional[int]), + ('dynamic_read_poll_interval_seconds', typing.Optional[int]) + ('consumer_factory_fn', typing.Optional[str]), + ('consumer_factory_fn_params', typing.Optional[typing.Mapping[str, str]]) ]) @@ -173,6 +175,8 @@ def __init__( redistribute_num_keys=np.int32(0), allow_duplicates=False, dynamic_read_poll_interval_seconds: typing.Optional[int] = None, + consumer_factory_fn_class=None, + consumer_factory_fn_params=None ): """ Initializes a read operation from Kafka. @@ -216,6 +220,13 @@ def __init__( :param dynamic_read_poll_interval_seconds: The interval in seconds at which to check for new partitions. If not None, dynamic partition discovery is enabled. + :param consumer_factory_fn_class: A fully qualified classpath to an existing provided + consumerFactoryFn. If not None, this will construct Kafka consumers with a custom + configuration. + :param consumer_factory_fn_params: A map which specifies the parameters for the provided + consumer_factory_fn_class. IF not None, the values in this map will be used when + constructing the consumer_factory_fn_class object. This cannot be null + if the consumer_factory_fn_class is not null. """ if timestamp_policy not in [ReadFromKafka.processing_time_policy, ReadFromKafka.create_time_policy, @@ -242,7 +253,9 @@ def __init__( redistribute_num_keys=redistribute_num_keys, allow_duplicates=allow_duplicates, dynamic_read_poll_interval_seconds= - dynamic_read_poll_interval_seconds)), + dynamic_read_poll_interval_seconds, + consumer_factory_fn_class=consumer_factory_fn_class, + consumer_factory_fn_params=consumer_factory_fn_params)), expansion_service or default_io_expansion_service()) From 470a751231abaedd8ec49913ef1149e5b2221b82 Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Mon, 29 Sep 2025 21:31:21 +0000 Subject: [PATCH 12/14] Remove accidentally committed python modules --- .../factories/FileAwareFactoryFnTest.java | 0 .../KerberosConsumerFactoryFnTest.java | 0 .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 38 +++++++----- .../io/external/xlang_kafkaio_it_test.py | 62 +------------------ sdks/python/apache_beam/io/kafka.py | 48 +++++++------- 5 files changed, 45 insertions(+), 103 deletions(-) delete mode 100644 sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java delete mode 100644 sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java diff --git a/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java b/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/FileAwareFactoryFnTest.java deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java b/sdks/java/extensions/kafka-factories/src/test/java/org/apache/beam/sdk/io/kafka/file/aware/factories/KerberosConsumerFactoryFnTest.java deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 43b286e87e08..48e4ae2317ac 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -27,7 +27,6 @@ import io.confluent.kafka.serializers.KafkaAvroDeserializer; import java.io.InputStream; import java.io.OutputStream; -import java.lang.reflect.Constructor; import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Arrays; @@ -36,6 +35,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.regex.Pattern; @@ -96,7 +96,6 @@ import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.MonotonicallyIncreasing; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.WallTime; import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.util.construction.PTransformMatchers; import org.apache.beam.sdk.util.construction.ReplacementOutputs; @@ -937,18 +936,27 @@ static void setupExternalBuilder( if (config.consumerFactoryFnClass != null) { if (config.consumerFactoryFnClass.contains("KerberosConsumerFactoryFn")) { try { - if (!config.consumerFactoryFnClass.contains("krb5Location")) { - throw new IllegalArgumentException("The KerberosConsumerFactoryFn requires a location for the krb5.conf file. " + - "Please provide either a GCS location or Google Secret Manager location for this file."); + if (!config.consumerFactoryFnParams.containsKey("krb5Location")) { + throw new IllegalArgumentException( + "The KerberosConsumerFactoryFn requires a location for the krb5.conf file. " + + "Please provide either a GCS location or Google Secret Manager location for this file."); } - builder.setConsumerFactoryFn(InstanceBuilder.ofType(new TypeDescriptor, Consumer>>() {}) - .fromClassName(config.consumerFactoryFnClass) - .withArg( - String.class, - config.consumerFactoryFnParams.get("krb5Location")) - .build()); + String krb5Location = config.consumerFactoryFnParams.get("krb5Location"); + builder.setConsumerFactoryFn( + InstanceBuilder.ofType( + new TypeDescriptor< + SerializableFunction< + Map, Consumer>>() {}) + .fromClassName(config.consumerFactoryFnClass) + .withArg(String.class, Objects.requireNonNull(krb5Location)) + .build()); } catch (Exception e) { - throw new RuntimeException("Unable to construct FactoryFn " + config.consumerFactoryFnClass + ": " + e.getMessage(), e); + throw new RuntimeException( + "Unable to construct FactoryFn " + + config.consumerFactoryFnClass + + ": " + + e.getMessage(), + e); } } } @@ -1094,12 +1102,12 @@ public void setDynamicReadPollIntervalSeconds(Long dynamicReadPollIntervalSecond } public void setConsumerFactoryFnClass(String consumerFactoryFnClass) { - this.consumerFactoryFnClass = consumerFactoryFnClass; + this.consumerFactoryFnClass = consumerFactoryFnClass; } public void setConsumerFactoryFnParams(Map consumerFactoryFnParams) { - this.consumerFactoryFnParams = consumerFactoryFnParams; - } + this.consumerFactoryFnParams = consumerFactoryFnParams; + } } } diff --git a/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py b/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py index 4b4e145cb2f4..23178b0ee363 100644 --- a/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py +++ b/sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py @@ -41,10 +41,6 @@ from apache_beam.transforms.userstate import BagStateSpec from apache_beam.transforms.userstate import CombiningValueStateSpec from apache_beam.utils import subprocess_server -from apache_beam.options.pipeline_options import ( - PipelineOptions, - GoogleCloudOptions -) NUM_RECORDS = 1000 @@ -72,7 +68,7 @@ def process( class CrossLanguageKafkaIO(object): def __init__( - self, bootstrap_servers=None, topic=None, null_key=None, expansion_service=None): + self, bootstrap_servers, topic, null_key, expansion_service=None): self.bootstrap_servers = bootstrap_servers self.topic = topic self.null_key = null_key @@ -111,34 +107,6 @@ def build_read_pipeline(self, pipeline, max_num_records=None): | 'CalculateSum' >> beam.ParDo(CollectingFn()) | 'SetSumCounter' >> beam.Map(self.sum_counter.inc)) - def build_read_pipeline_with_kerberos(self, p, max_num_records=None): - - jaas_config = ( - f'com.sun.security.auth.module.Krb5LoginModule required ' - f'useKeyTab=true storeKey=true ' - f'keyTab="secretValue:projects/dataflow-testing-311516/secrets/kafka-client-keytab/versions/latest" ' - f'principal="kafka-client@US-CENTRAL1-B.C.DATAFLOW-TESTING-311516.INTERNAL";' - ) - - kafka_records = ( - p - | 'ReadFromKafka' >> ReadFromKafka( - consumer_config={ - 'bootstrap.servers': 'fozzie-test-kafka-broker.us-central1-c.c.dataflow-testing-311516.internal:9092', - 'auto.offset.reset': 'earliest', - 'max_num_records': max_num_records, - 'security.protocol': 'SASL_PLAINTEXT', - 'sasl.mechanism': 'GSSAPI', - 'sasl.kerberos.service.name': 'kafka', - 'sasl.jaas.config': jaas_config - }, - topics=['fozzie_test_kerberos_topic'], - key_deserializer='org.apache.kafka.common.serialization.StringDeserializer', - value_deserializer='org.apache.kafka.common.serialization.StringDeserializer', - consumer_factory_fn_class='org.apache.beam.sdk.extensions.kafka.factories.KerberosConsumerFactoryFn', - consumer_factory_fn_params={'krb5Location': 'gs://fozzie_testing_bucket/kerberos/krb5.conf'})) - return kafka_records - def run_xlang_kafkaio(self, pipeline): self.build_write_pipeline(pipeline) self.build_read_pipeline(pipeline) @@ -214,17 +182,6 @@ def test_hosted_kafkaio_null_key(self): self.run_kafka_write(pipeline_creator) self.run_kafka_read(pipeline_creator, None) - def test_hosted_kafkaio_null_key_kerberos(self): - kafka_topic = 'xlang_kafkaio_test_null_key_{}'.format(uuid.uuid4()) - bootstrap_servers = 'fozzie-test-kafka-broker.us-central1-c.c.dataflow-testing-311516.internal:9092' - pipeline_creator = CrossLanguageKafkaIO( - bootstrap_servers, - kafka_topic, - True, - 'localhost:%s' % os.environ.get('EXPANSION_PORT')) - - self.run_kafka_read_with_kerberos(pipeline_creator) - def run_kafka_write(self, pipeline_creator): with TestPipeline() as pipeline: pipeline.not_use_test_runner_api = True @@ -239,23 +196,6 @@ def run_kafka_read(self, pipeline_creator, expected_key): equal_to([(expected_key, str(i).encode()) for i in range(NUM_RECORDS)])) - def run_kafka_read_with_kerberos(self, pipeline_creator): - options_dict = { - 'runner': 'DataflowRunner', - 'project': 'dataflow-testing-311516', - 'region': 'us-central1', - 'streaming': False - } - options = PipelineOptions.from_dictionary(options_dict) - expected_records = [f'test{i}' for i in range(1, 12)] - with beam.Pipeline(options=options) as p: - pipeline.not_use_test_runner_api = True - result = pipeline_creator.build_read_pipeline_with_kerberos(p, max_num_records=11) - assert_that( - result, - equal_to(expected_records) - ) - def get_platform_localhost(self): if sys.platform == 'darwin': return 'host.docker.internal' diff --git a/sdks/python/apache_beam/io/kafka.py b/sdks/python/apache_beam/io/kafka.py index 1fbafda26274..97be602ed5ea 100644 --- a/sdks/python/apache_beam/io/kafka.py +++ b/sdks/python/apache_beam/io/kafka.py @@ -110,24 +110,19 @@ ReadFromKafkaSchema = typing.NamedTuple( 'ReadFromKafkaSchema', - [ - ('consumer_config', typing.Mapping[str, str]), - ('topics', typing.List[str]), - ('key_deserializer', str), - ('value_deserializer', str), - ('start_read_time', typing.Optional[int]), - ('max_num_records', typing.Optional[int]), - ('max_read_time', typing.Optional[int]), - ('commit_offset_in_finalize', bool), - ('timestamp_policy', str), - ('consumer_polling_timeout', typing.Optional[int]), - ('redistribute', typing.Optional[bool]), - ('redistribute_num_keys', typing.Optional[np.int32]), - ('allow_duplicates', typing.Optional[bool]), - ('dynamic_read_poll_interval_seconds', typing.Optional[int]) - ('consumer_factory_fn', typing.Optional[str]), - ('consumer_factory_fn_params', typing.Optional[typing.Mapping[str, str]]) - ]) + [('consumer_config', typing.Mapping[str, str]), + ('topics', typing.List[str]), ('key_deserializer', str), + ('value_deserializer', str), ('start_read_time', typing.Optional[int]), + ('max_num_records', typing.Optional[int]), + ('max_read_time', typing.Optional[int]), + ('commit_offset_in_finalize', bool), ('timestamp_policy', str), + ('consumer_polling_timeout', typing.Optional[int]), + ('redistribute', typing.Optional[bool]), + ('redistribute_num_keys', typing.Optional[np.int32]), + ('allow_duplicates', typing.Optional[bool]), + ('dynamic_read_poll_interval_seconds', typing.Optional[int]), + ('consumer_factory_fn_class', typing.Optional[str]), + ('consumer_factory_fn_params', typing.Optional[typing.Mapping[str, str]])]) def default_io_expansion_service(append_args=None): @@ -176,8 +171,7 @@ def __init__( allow_duplicates=False, dynamic_read_poll_interval_seconds: typing.Optional[int] = None, consumer_factory_fn_class=None, - consumer_factory_fn_params=None - ): + consumer_factory_fn_params=None): """ Initializes a read operation from Kafka. @@ -220,13 +214,13 @@ def __init__( :param dynamic_read_poll_interval_seconds: The interval in seconds at which to check for new partitions. If not None, dynamic partition discovery is enabled. - :param consumer_factory_fn_class: A fully qualified classpath to an existing provided - consumerFactoryFn. If not None, this will construct Kafka consumers with a custom - configuration. - :param consumer_factory_fn_params: A map which specifies the parameters for the provided - consumer_factory_fn_class. IF not None, the values in this map will be used when - constructing the consumer_factory_fn_class object. This cannot be null - if the consumer_factory_fn_class is not null. + :param consumer_factory_fn_class: A fully qualified classpath to an + existing provided consumerFactoryFn. If not None, this will construct + Kafka consumers with a custom configuration. + :param consumer_factory_fn_params: A map which specifies the parameters for + the provided consumer_factory_fn_class. IF not None, the values in this + map will be used when constructing the consumer_factory_fn_class object. + This cannot be null if the consumer_factory_fn_class is not null. """ if timestamp_policy not in [ReadFromKafka.processing_time_policy, ReadFromKafka.create_time_policy, From 78b8a074ed3a00902de0f78373cda35d769ae8b9 Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Mon, 3 Nov 2025 18:40:30 +0000 Subject: [PATCH 13/14] Trigger CI build From c7c583d146950b5bd394af8f5a32bdce1cb52b0e Mon Sep 17 00:00:00 2001 From: fozzie15 Date: Wed, 5 Nov 2025 21:40:07 +0000 Subject: [PATCH 14/14] Clean up typing. --- sdks/python/apache_beam/io/kafka.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/io/kafka.py b/sdks/python/apache_beam/io/kafka.py index 97be602ed5ea..09063fb0828f 100644 --- a/sdks/python/apache_beam/io/kafka.py +++ b/sdks/python/apache_beam/io/kafka.py @@ -100,6 +100,7 @@ # pytype: skip-file +import collections import typing import numpy as np @@ -122,7 +123,9 @@ ('allow_duplicates', typing.Optional[bool]), ('dynamic_read_poll_interval_seconds', typing.Optional[int]), ('consumer_factory_fn_class', typing.Optional[str]), - ('consumer_factory_fn_params', typing.Optional[typing.Mapping[str, str]])]) + ( + 'consumer_factory_fn_params', + typing.Optional[collections.abc.Mapping[str, str]])]) def default_io_expansion_service(append_args=None): @@ -170,8 +173,9 @@ def __init__( redistribute_num_keys=np.int32(0), allow_duplicates=False, dynamic_read_poll_interval_seconds: typing.Optional[int] = None, - consumer_factory_fn_class=None, - consumer_factory_fn_params=None): + consumer_factory_fn_class: typing.Optional[str] = None, + consumer_factory_fn_params: typing.Optional[ + collections.abc.Mapping] = None): """ Initializes a read operation from Kafka. @@ -218,7 +222,7 @@ def __init__( existing provided consumerFactoryFn. If not None, this will construct Kafka consumers with a custom configuration. :param consumer_factory_fn_params: A map which specifies the parameters for - the provided consumer_factory_fn_class. IF not None, the values in this + the provided consumer_factory_fn_class. If not None, the values in this map will be used when constructing the consumer_factory_fn_class object. This cannot be null if the consumer_factory_fn_class is not null. """