diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java index 62022b219c2a..989e3a1e3193 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java @@ -420,7 +420,7 @@ public Long create(PipelineOptions options) { *
Beam will infer the secret type and value based on the secret itself. This guarantees that * any data at rest during the performing a GBK, so this can be used to guarantee that data is not * unencrypted. Runners with this behavior include the Dataflow, Flink, and Spark runners. The - * option should be structured like: + * secret should be a url safe base64 encoded 32 byte value. The option should be structured like: * *
* --gbek=type:;:
@@ -432,14 +432,19 @@ public Long create(PipelineOptions options) {
* --gbek=type:GcpSecret;version_name:my_secret/versions/latest"
*
*
- * All variables should use snake case to allow consistency across languages.
+ * All variables should use snake case to allow consistency across languages. For an example of
+ * generating a properly formatted secret, see
+ * https://github.com/apache/beam/blob/c8df4da229da49d533491857e1bb4ab5dbf4fd37/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyIT.java#L82
*/
@Description(
"When set, will replace all GroupByKey transforms in the pipeline the option. Beam will"
+ " infer the secret type and value based on the secret itself. This guarantees that"
+ " any data at rest during the performing a GBK, so this can be used to guarantee"
+ " that data is not unencrypted. Runners with this behavior include the Dataflow,"
- + " Flink, and Spark runners. The option should be structured like:"
+ + " Flink, and Spark runners. The secret should be a url safe base64 encoded 32 byte"
+ + " value. For an example of generating a properly formatted secret, see"
+ + " https://github.com/apache/beam/blob/c8df4da229da49d533491857e1bb4ab5dbf4fd37/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyIT.java#L82"
+ + " When passing in the gbek option, it should be structured like:"
+ " --gbek=type:The transform requires a {@link Secret} which returns a 32 byte secret which can be used to - * generate a {@link SecretKeySpec} object using the HmacSHA256 algorithm. + *
The transform requires a {@link Secret} which returns a base64 encoded 32 byte secret which + * can be used to generate a {@link SecretKeySpec} object using the HmacSHA256 algorithm. * *
Note the following caveats: 1) Runners can implement arbitrary materialization steps, so this
* does not guarantee that the whole pipeline will not have unencrypted data at rest by itself. 2)
@@ -153,7 +153,7 @@ private static class EncryptMessage