From 45ee19f27f10f5d287585decc48fd1686a939594 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Mon, 11 Jan 2021 12:33:26 +0300 Subject: [PATCH 01/29] [WIP] Transfer from DataflowTemplates to Beam --- .../ProtegrityDataTokenization.java | 226 +++++++++++ .../ProtegrityDataTokenizationOptions.java | 74 ++++ .../options/package-info.java | 18 + .../datatokenization/package-info.java | 18 + .../transforms/ProtegrityDataProtectors.java | 376 ++++++++++++++++++ .../transforms/io/BigQueryIO.java | 99 +++++ .../transforms/io/BigTableIO.java | 160 ++++++++ .../datatokenization/transforms/io/GcsIO.java | 233 +++++++++++ .../transforms/io/package-info.java | 18 + .../transforms/package-info.java | 18 + .../datatokenization/utils/RowToCsv.java | 35 ++ .../datatokenization/utils/SchemasUtils.java | 136 +++++++ .../datatokenization/utils/package-info.java | 18 + 13 files changed, 1429 insertions(+) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/ProtegrityDataTokenizationOptions.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/package-info.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/package-info.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/package-info.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/package-info.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/RowToCsv.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/package-info.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java new file mode 100644 index 000000000000..7cd22dd2cfab --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java @@ -0,0 +1,226 @@ +/* + * 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.examples.complete.datatokenization; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Map; +import org.apache.beam.examples.complete.datatokenization.options.ProtegrityDataTokenizationOptions; +import org.apache.beam.examples.complete.datatokenization.transforms.ProtegrityDataProtectors.RowToTokenizedRow; +import org.apache.beam.examples.complete.datatokenization.transforms.io.BigQueryIO; +import org.apache.beam.examples.complete.datatokenization.transforms.io.BigTableIO; +import org.apache.beam.examples.complete.datatokenization.transforms.io.GcsIO; +import org.apache.beam.examples.complete.datatokenization.utils.SchemasUtils; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.gcp.bigquery.WriteResult; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.JsonToRow; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link ProtegrityDataTokenization} pipeline. + */ +public class ProtegrityDataTokenization { + + /** + * Logger for class. + */ + private static final Logger LOG = LoggerFactory.getLogger(ProtegrityDataTokenization.class); + + /** + * The default suffix for error tables if dead letter table is not specified. + */ + private static final String DEFAULT_DEADLETTER_TABLE_SUFFIX = "_error_records"; + + /** + * The tag for the main output for the UDF. + */ + private static final TupleTag TOKENIZATION_OUT = new TupleTag() { + }; + + /** + * Main entry point for pipeline execution. + * + * @param args Command line arguments to the pipeline. + */ + public static void main(String[] args) { + ProtegrityDataTokenizationOptions options = + PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(ProtegrityDataTokenizationOptions.class); + FileSystems.setDefaultPipelineOptions(options); + + run(options); + } + + /** + * Runs the pipeline to completion with the specified options. + * + * @param options The execution options. + * @return The pipeline result. + */ + public static PipelineResult run( + ProtegrityDataTokenizationOptions options) { + SchemasUtils schema = null; + try { + schema = new SchemasUtils(options.getDataSchemaGcsPath(), StandardCharsets.UTF_8); + } catch (IOException e) { + LOG.error("Failed to retrieve schema for data.", e); + } + checkArgument(schema != null, "Data schema is mandatory."); + + Map dataElements = + schema.getDataElementsToTokenize(options.getPayloadConfigGcsPath()); + + // Create the pipeline + Pipeline pipeline = Pipeline.create(options); + // Register the coder for pipeline + CoderRegistry coderRegistry = pipeline.getCoderRegistry(); + coderRegistry.registerCoderForType( + RowCoder.of(schema.getBeamSchema()).getEncodedTypeDescriptor(), + RowCoder.of(schema.getBeamSchema())); + + PCollection jsons; + if (options.getInputGcsFilePattern() != null) { + jsons = new GcsIO(options).read(pipeline, schema.getJsonBeamSchema()); + } else if (options.getPubsubTopic() != null) { + jsons = + pipeline.apply( + "ReadMessagesFromPubsub", PubsubIO.readStrings().fromTopic(options.getPubsubTopic())); + } else { + throw new IllegalStateException("No source is provided, please configure GCS or Pub/Sub"); + } + + JsonToRow.ParseResult rows = + jsons.apply( + "JsonToRow", + JsonToRow.withExceptionReporting(schema.getBeamSchema()).withExtendedErrorInfo()); + +// if (options.getNonTokenizedDeadLetterGcsPath() != null) { +// /* +// * Write Row conversion errors to filesystem specified path +// */ +// rows.getFailedToParseLines() +// .apply( +// "ToFailsafeElement", +// MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor()) +// .via( +// (Row errRow) -> +// FailsafeElement.of(errRow.getString("line"), errRow.getString("line")) +// .setErrorMessage(errRow.getString("err")))) +// .apply( +// "WriteCsvConversionErrorsToGcs", +// ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder() +// .setCsvDelimiter(options.getCsvDelimiter()) +// .setErrorWritePath(options.getNonTokenizedDeadLetterGcsPath()) +// .build()); +// } + /* + Tokenize data using remote API call + */ + PCollectionTuple tokenizedRows = + rows.getResults() + .setRowSchema(schema.getBeamSchema()) + .apply( + MapElements.into( + TypeDescriptors.kvs(TypeDescriptors.integers(), TypeDescriptors.rows())) + .via((Row row) -> KV.of(0, row))) + .setCoder(KvCoder.of(VarIntCoder.of(), RowCoder.of(schema.getBeamSchema()))) + .apply( + "DsgTokenization", + RowToTokenizedRow.newBuilder() + .setBatchSize(options.getBatchSize()) + .setDsgURI(options.getDsgUri()) + .setSchema(schema.getBeamSchema()) + .setDataElements(dataElements) + .setSuccessTag(TOKENIZATION_OUT) + .setFailureTag(TOKENIZATION_DEADLETTER_OUT) + .build()); + +// String csvDelimiter = options.getCsvDelimiter(); +// if (options.getNonTokenizedDeadLetterGcsPath() != null) { +// /* +// Write tokenization errors to dead-letter sink +// */ +// tokenizedRows +// .get(TOKENIZATION_DEADLETTER_OUT) +// .apply( +// "ConvertToCSV", +// MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor()) +// .via( +// (FailsafeElement fse) -> +// FailsafeElement.of( +// new RowToCsv(csvDelimiter).getCsvFromRow(fse.getOriginalPayload()), +// new RowToCsv(csvDelimiter).getCsvFromRow(fse.getPayload())))) +// .apply( +// "WriteTokenizationErrorsToGcs", +// ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder() +// .setCsvDelimiter(options.getCsvDelimiter()) +// .setErrorWritePath(options.getNonTokenizedDeadLetterGcsPath()) +// .build()); +// } + + if (options.getOutputGcsDirectory() != null) { + new GcsIO(options).write(tokenizedRows.get(TOKENIZATION_OUT), schema.getBeamSchema()); + } else if (options.getBigQueryTableName() != null) { + WriteResult writeResult = + BigQueryIO.write( + tokenizedRows.get(TOKENIZATION_OUT), + options.getBigQueryTableName(), + schema.getBigQuerySchema()); +// writeResult +// .getFailedInsertsWithErr() +// .apply( +// "WrapInsertionErrors", +// MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor()) +// .via(BigQueryIO::wrapBigQueryInsertError)) +// .setCoder(FAILSAFE_ELEMENT_CODER) +// .apply( +// "WriteInsertionFailedRecords", +// ErrorConverters.WriteStringMessageErrors.newBuilder() +// .setErrorRecordsTable( +// options.getBigQueryTableName() + DEFAULT_DEADLETTER_TABLE_SUFFIX) +// .setErrorRecordsTableSchema(SchemaUtils.DEADLETTER_SCHEMA) +// .build()); + } else if (options.getBigTableInstanceId() != null) { + new BigTableIO(options).write(tokenizedRows.get(TOKENIZATION_OUT), schema.getBeamSchema()); + } else { + throw new IllegalStateException( + "No sink is provided, please configure BigQuery or BigTable."); + } + + return pipeline.run(); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/ProtegrityDataTokenizationOptions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/ProtegrityDataTokenizationOptions.java new file mode 100644 index 000000000000..1b4a258b7a76 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/ProtegrityDataTokenizationOptions.java @@ -0,0 +1,74 @@ +/* + * 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.examples.complete.datatokenization.options; + +import org.apache.beam.examples.complete.datatokenization.transforms.io.BigTableIO; +import org.apache.beam.examples.complete.datatokenization.transforms.io.GcsIO; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; + +/** + * The {@link ProtegrityDataTokenizationOptions} interface provides the custom execution options + * passed by the executor at the command-line. + */ +public interface ProtegrityDataTokenizationOptions + extends PipelineOptions, GcsIO.GcsPipelineOptions, BigTableIO.BigTableOptions { + + @Description("Path to data schema (JSON format) in GCS compatible with BigQuery.") + String getDataSchemaGcsPath(); + + void setDataSchemaGcsPath(String dataSchemaGcsPath); + + @Description( + "The Cloud Pub/Sub topic to read from." + + "The name should be in the format of " + + "projects//topics/.") + String getPubsubTopic(); + + void setPubsubTopic(String pubsubTopic); + + @Description("Cloud BigQuery table name to write into.") + String getBigQueryTableName(); + + void setBigQueryTableName(String bigQueryTableName); + + // Protegrity specific parameters + @Description("URI for the API calls to DSG.") + String getDsgUri(); + + void setDsgUri(String dsgUri); + + @Description("Size of the batch to send to DSG per request.") + @Default.Integer(10) + Integer getBatchSize(); + + void setBatchSize(Integer batchSize); + + @Description( + "GCS path to the payload configuration file with an array of fields " + + "to extract for tokenization.") + String getPayloadConfigGcsPath(); + + void setPayloadConfigGcsPath(String payloadConfigGcsPath); + + @Description("Dead-Letter GCS path to store not-tokenized data") + String getNonTokenizedDeadLetterGcsPath(); + + void setNonTokenizedDeadLetterGcsPath(String nonTokenizedDeadLetterGcsPath); +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/package-info.java new file mode 100644 index 000000000000..b61fce151910 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2020 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** Protegrity Data Tokenization template for Google Cloud Teleport. */ +package org.apache.beam.examples.complete.datatokenization.options; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/package-info.java new file mode 100644 index 000000000000..09d4df153c3b --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2020 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** Protegrity Data Tokenization template for Google Cloud Teleport. */ +package org.apache.beam.examples.complete.datatokenization; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java new file mode 100644 index 000000000000..f7919afd5909 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java @@ -0,0 +1,376 @@ +/* + * 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.examples.complete.datatokenization.transforms; + +import static org.apache.beam.sdk.util.RowJsonUtils.rowToJson; +import static org.apache.beam.vendor.grpc.v1p26p0.com.google.common.base.MoreObjects.firstNonNull; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.services.bigquery.model.TableRow; +import com.google.auto.value.AutoValue; +import com.google.gson.reflect.TypeToken; +import java.io.IOException; +import java.lang.reflect.Type; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.RowJson; +import org.apache.beam.sdk.util.RowJsonUtils; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.Gson; +import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.JsonArray; +import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.JsonObject; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables; +import org.apache.commons.io.IOUtils; +import org.apache.http.HttpEntity; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.ByteArrayEntity; +import org.apache.http.entity.ContentType; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * TODO: Add javadoc. + */ +public class ProtegrityDataProtectors { + + /** + * Logger for class. + */ + private static final Logger LOG = LoggerFactory.getLogger(ProtegrityDataProtectors.class); + + /** + * The {@link RowToTokenizedRow} transform converts {@link Row} to {@link TableRow} objects. The + * transform accepts a {@link FailsafeElement} object so the original payload of the incoming + * record can be maintained across multiple series of transforms. + */ + @AutoValue + public abstract static class RowToTokenizedRow + extends PTransform>, PCollectionTuple> { + + public static Builder newBuilder() { + return new AutoValue_ProtegrityDataProtectors_RowToTokenizedRow.Builder<>(); + } + + public abstract TupleTag successTag(); + + public abstract TupleTag> failureTag(); + + public abstract Schema schema(); + + public abstract int batchSize(); + + public abstract Map dataElements(); + + public abstract String dsgURI(); + + @Override + public PCollectionTuple expand(PCollection> inputRows) { + FailsafeElementCoder coder = + FailsafeElementCoder.of(RowCoder.of(schema()), RowCoder.of(schema())); + PCollectionTuple pCollectionTuple = + inputRows.apply( + "TokenizeUsingDsg", + ParDo.of( + new DSGTokenizationFn( + schema(), + batchSize(), + dataElements(), + dsgURI(), + failureTag())) + .withOutputTags(successTag(), TupleTagList.of(failureTag()))); + return PCollectionTuple.of( + successTag(), pCollectionTuple.get(successTag()).setRowSchema(schema())) + .and(failureTag(), pCollectionTuple.get(failureTag()).setCoder(coder)); + } + + /** + * Builder for {@link RowToTokenizedRow}. + */ + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setSuccessTag(TupleTag successTag); + + public abstract Builder setFailureTag(TupleTag> failureTag); + + public abstract Builder setSchema(Schema schema); + + public abstract Builder setBatchSize(int batchSize); + + public abstract Builder setDataElements(Map fieldsDataElements); + + public abstract Builder setDsgURI(String dsgURI); + + public abstract RowToTokenizedRow build(); + } + } + + /** + * Class for data tokenization using DSG. + */ + public static class DSGTokenizationFn extends DoFn, Row> { + + public static final String ID_TOKEN_NAME = "ID"; + + private static Schema schemaToDsg; + private static CloseableHttpClient httpclient; + private static ObjectMapper objectMapperSerializerForDSG; + private static ObjectMapper objectMapperDeserializerForDSG; + + private final Schema schema; + private final int batchSize; + private final Map dataElements; + private final String dsgURI; + private final TupleTag> failureTag; + + @StateId("buffer") + private final StateSpec> bufferedEvents; + + @StateId("count") + private final StateSpec> countState = StateSpecs.value(); + + @TimerId("expiry") + private final TimerSpec expirySpec = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + private boolean hasIdInInputs = true; + private String idFieldName; + private Map inputRowsWithIds; + + public DSGTokenizationFn( + Schema schema, + int batchSize, + Map dataElements, + String dsgURI, + TupleTag> failureTag) { + this.schema = schema; + this.batchSize = batchSize; + this.dataElements = dataElements; + this.dsgURI = dsgURI; + bufferedEvents = StateSpecs.bag(RowCoder.of(schema)); + this.failureTag = failureTag; + } + + @Setup + public void setup() { + + List idFieldList = + dataElements.entrySet().stream() + .filter(map -> ID_TOKEN_NAME.equals(map.getValue())) + .map(Entry::getKey) + .collect(Collectors.toList()); + + // If we have more than 1 ID fields, we will choose the first. + if (idFieldList.size() > 0) { + idFieldName = idFieldList.get(0); + } + + if (idFieldName == null || !schema.hasField(idFieldName)) { + this.hasIdInInputs = false; + } + + ArrayList fields = new ArrayList<>(); + for (String field : dataElements.keySet()) { + if (schema.hasField(field)) { + fields.add(schema.getField(field)); + } + } + if (!hasIdInInputs) { + idFieldName = ID_TOKEN_NAME; + fields.add(Field.of(ID_TOKEN_NAME, FieldType.STRING)); + dataElements.put(ID_TOKEN_NAME, ID_TOKEN_NAME); + } + schemaToDsg = new Schema(fields); + objectMapperSerializerForDSG = + RowJsonUtils.newObjectMapperWith(RowJson.RowJsonSerializer.forSchema(schemaToDsg)); + + objectMapperDeserializerForDSG = + RowJsonUtils.newObjectMapperWith(RowJson.RowJsonDeserializer.forSchema(schemaToDsg)); + + httpclient = HttpClients.createDefault(); + } + + @Teardown + public void close() { + try { + httpclient.close(); + } catch (IOException exception) { + LOG.warn("Can't close connection: {}", exception.getMessage()); + } + } + + @OnTimer("expiry") + public void onExpiry(OnTimerContext context, @StateId("buffer") BagState bufferState) { + boolean isEmpty = firstNonNull(bufferState.isEmpty().read(), true); + if (!isEmpty) { + processBufferedRows(bufferState.read(), context); + bufferState.clear(); + } + } + + @ProcessElement + public void process( + ProcessContext context, + BoundedWindow window, + @StateId("buffer") BagState bufferState, + @StateId("count") ValueState countState, + @TimerId("expiry") Timer expiryTimer) { + + expiryTimer.set(window.maxTimestamp()); + + int count = firstNonNull(countState.read(), 0); + count++; + countState.write(count); + bufferState.add(context.element().getValue()); + + if (count >= batchSize) { + processBufferedRows(bufferState.read(), context); + bufferState.clear(); + countState.clear(); + } + } + + private void processBufferedRows(Iterable rows, WindowedContext context) { + + try { + for (Row outputRow : getTokenizedRow(rows)) { + context.output(outputRow); + } + } catch (Exception e) { + for (Row outputRow : rows) { + context.output( + failureTag, + FailsafeElement.of(outputRow, outputRow) + .setErrorMessage(e.getMessage()) + .setStacktrace(Throwables.getStackTraceAsString(e))); + } + } + } + + private ArrayList rowsToJsons(Iterable inputRows) { + ArrayList jsons = new ArrayList<>(); + Map inputRowsWithIds = new HashMap<>(); + for (Row inputRow : inputRows) { + + Row.Builder builder = Row.withSchema(schemaToDsg); + for (Schema.Field field : schemaToDsg.getFields()) { + if (inputRow.getSchema().hasField(field.getName())) { + builder = builder.addValue(inputRow.getValue(field.getName())); + } + } + String id; + if (!hasIdInInputs) { + id = UUID.randomUUID().toString(); + builder = builder.addValue(id); + } else { + id = inputRow.getValue(idFieldName); + } + inputRowsWithIds.put(id, inputRow); + + Row row = builder.build(); + + jsons.add(rowToJson(objectMapperSerializerForDSG, row)); + } + this.inputRowsWithIds = inputRowsWithIds; + return jsons; + } + + private String formatJsonsToDsgBatch(Iterable jsons) { + StringBuilder stringBuilder = new StringBuilder(String.join(",", jsons)); + Gson gson = new Gson(); + Type gsonType = new TypeToken>() { + }.getType(); + String dataElementsJson = gson.toJson(dataElements, gsonType); + stringBuilder + .append("]") + .insert(0, "{\"data\": [") + .append(",\"data_elements\":") + .append(dataElementsJson) + .append("}"); + return stringBuilder.toString(); + } + + private ArrayList getTokenizedRow(Iterable inputRows) throws IOException { + ArrayList outputRows = new ArrayList<>(); + + CloseableHttpResponse response = + sendToDsg(formatJsonsToDsgBatch(rowsToJsons(inputRows)).getBytes()); + + String tokenizedData = + IOUtils.toString(response.getEntity().getContent(), StandardCharsets.UTF_8); + + Gson gson = new Gson(); + JsonArray jsonTokenizedRows = + gson.fromJson(tokenizedData, JsonObject.class).getAsJsonArray("data"); + + for (int i = 0; i < jsonTokenizedRows.size(); i++) { + Row tokenizedRow = + RowJsonUtils.jsonToRow( + objectMapperDeserializerForDSG, jsonTokenizedRows.get(i).toString()); + Row.FieldValueBuilder rowBuilder = + Row.fromRow(this.inputRowsWithIds.get(tokenizedRow.getString(idFieldName))); + for (Schema.Field field : schemaToDsg.getFields()) { + if (!hasIdInInputs && field.getName().equals(idFieldName)) { + continue; + } + rowBuilder = + rowBuilder.withFieldValue(field.getName(), tokenizedRow.getValue(field.getName())); + } + outputRows.add(rowBuilder.build()); + } + + return outputRows; + } + + private CloseableHttpResponse sendToDsg(byte[] data) throws IOException { + HttpPost httpPost = new HttpPost(dsgURI); + HttpEntity stringEntity = new ByteArrayEntity(data, ContentType.APPLICATION_JSON); + httpPost.setEntity(stringEntity); + return httpclient.execute(httpPost); + } + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java new file mode 100644 index 000000000000..d680e3ab5e5d --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java @@ -0,0 +1,99 @@ +/* + * 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.examples.complete.datatokenization.transforms.io; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; +import org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy; +import org.apache.beam.sdk.io.gcp.bigquery.WriteResult; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link BigQueryIO} class for writing data from template to BigTable. + */ +public class BigQueryIO { + + /** + * Logger for class. + */ + private static final Logger LOG = LoggerFactory.getLogger(BigQueryIO.class); + + public static WriteResult write( + PCollection input, String bigQueryTableName, TableSchema schema) { + return input + .apply("RowToTableRow", ParDo.of(new RowToTableRowFn())) + .apply( + "WriteSuccessfulRecords", + org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.writeTableRows() + .withCreateDisposition( + org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition + .CREATE_IF_NEEDED) + .withWriteDisposition( + org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition + .WRITE_APPEND) + .withExtendedErrorInfo() + .withMethod( + org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.Method.STREAMING_INSERTS) + .withFailedInsertRetryPolicy(InsertRetryPolicy.retryTransientErrors()) + .withSchema(schema) + .to(bigQueryTableName)); + } + +// /** +// * Method to wrap a {@link BigQueryInsertError} into a {@link FailsafeElement}. +// * +// * @param insertError BigQueryInsert error. +// * @return FailsafeElement object. +// */ +// public static FailsafeElement wrapBigQueryInsertError( +// BigQueryInsertError insertError) { +// +// FailsafeElement failsafeElement; +// try { +// +// failsafeElement = +// FailsafeElement.of( +// insertError.getRow().toPrettyString(), insertError.getRow().toPrettyString()); +// failsafeElement.setErrorMessage(insertError.getError().toPrettyString()); +// +// } catch (IOException e) { +// BigQueryIO.LOG.error("Failed to wrap BigQuery insert error."); +// throw new RuntimeException(e); +// } +// return failsafeElement; +// } + + /** + * The {@link RowToTableRowFn} class converts a row to tableRow using {@link + * BigQueryUtils#toTableRow()}. + */ + public static class RowToTableRowFn extends DoFn { + + @ProcessElement + public void processElement(ProcessContext context) { + Row row = context.element(); + context.output(BigQueryUtils.toTableRow(row)); + } + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java new file mode 100644 index 000000000000..99e1acd4241e --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java @@ -0,0 +1,160 @@ +/* + * 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.examples.complete.datatokenization.transforms.io; + +import com.google.bigtable.v2.Mutation; +import com.google.protobuf.ByteString; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.beam.examples.complete.datatokenization.options.ProtegrityDataTokenizationOptions; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteResult; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.Row; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link BigTableIO} class for writing data from template to BigTable. + */ +public class BigTableIO { + + /** + * Logger for class. + */ + private static final Logger LOG = LoggerFactory.getLogger(BigTableIO.class); + + private final ProtegrityDataTokenizationOptions options; + + public BigTableIO(ProtegrityDataTokenizationOptions options) { + this.options = options; + } + + public PDone write(PCollection input, Schema schema) { + return input + .apply("ConvertToBigTableFormat", ParDo.of(new TransformToBigTableFormat(schema))) + .apply( + "WriteToBigTable", + BigtableIO.write() + .withProjectId(options.getBigTableProjectId()) + .withInstanceId(options.getBigTableInstanceId()) + .withTableId(options.getBigTableTableId()) + .withWriteResults()) + .apply("LogRowCount", new LogSuccessfulRows()); + } + + static class TransformToBigTableFormat extends DoFn>> { + + private final Schema schema; + + TransformToBigTableFormat(Schema schema) { + this.schema = schema; + } + + @ProcessElement + public void processElement( + @Element Row in, OutputReceiver>> out, ProcessContext c) { + ProtegrityDataTokenizationOptions options = + c.getPipelineOptions().as(ProtegrityDataTokenizationOptions.class); + // Mapping every field in provided Row to Mutation.SetCell, which will create/update + // cell content with provided data + Set mutations = + schema.getFields().stream() + .map(Schema.Field::getName) + // Ignoring key field, otherwise it will be added as regular column + .filter(fieldName -> !Objects.equals(fieldName, options.getBigTableKeyColumnName())) + .map(fieldName -> Pair.of(fieldName, in.getString(fieldName))) + .map( + pair -> + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setFamilyName(options.getBigTableColumnFamilyName()) + .setColumnQualifier(ByteString.copyFrom(pair.getKey().getBytes())) + .setValue(ByteString.copyFrom(pair.getValue().getBytes())) + .setTimestampMicros(System.currentTimeMillis() * 1000) + .build()) + .build()) + .collect(Collectors.toSet()); + // Converting key value to BigTable format + // TODO ramazan@akvelon.com check that please (NPE) + ByteString key = + ByteString.copyFrom( + Objects.requireNonNull(in.getString(options.getBigTableKeyColumnName())).getBytes()); + out.output(KV.of(key, mutations)); + } + } + + static class LogSuccessfulRows extends PTransform, PDone> { + + @Override + public PDone expand(PCollection input) { + input.apply( + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(@Element BigtableWriteResult in) { + LOG.info("Successfully wrote {} rows.", in.getRowsWritten()); + } + })); + return PDone.in(input.getPipeline()); + } + } + + /** + * Necessary {@link PipelineOptions} options for Pipelines that perform write operations to + * BigTable. + */ + public interface BigTableOptions extends PipelineOptions { + + @Description("Id of the project where the Cloud BigTable instance to write into is located.") + String getBigTableProjectId(); + + void setBigTableProjectId(String bigTableProjectId); + + @Description("Id of the Cloud BigTable instance to write into.") + String getBigTableInstanceId(); + + void setBigTableInstanceId(String bigTableInstanceId); + + @Description("Id of the Cloud BigTable table to write into.") + String getBigTableTableId(); + + void setBigTableTableId(String bigTableTableId); + + @Description("Column name to use as a key in Cloud BigTable.") + String getBigTableKeyColumnName(); + + void setBigTableKeyColumnName(String bigTableKeyColumnName); + + @Description("Column family name to use in Cloud BigTable.") + String getBigTableColumnFamilyName(); + + void setBigTableColumnFamilyName(String bigTableColumnFamilyName); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java new file mode 100644 index 000000000000..dea28ebe7a0d --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java @@ -0,0 +1,233 @@ +/* + * 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.examples.complete.datatokenization.transforms.io; + +import org.apache.beam.examples.complete.datatokenization.options.ProtegrityDataTokenizationOptions; +import org.apache.beam.examples.complete.datatokenization.utils.RowToCsv; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.ToJson; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link GcsIO} class to read/write data from/into Google Cloud Storage. + */ +public class GcsIO { + + /** + * The tag for the headers of the CSV if required. + */ + static final TupleTag CSV_HEADERS = new TupleTag() { + }; + + /** + * The tag for the lines of the CSV. + */ + static final TupleTag CSV_LINES = new TupleTag() { + }; + + /** + * The tag for the dead-letter output. + */ + static final TupleTag> PROCESSING_DEADLETTER_OUT = + new TupleTag>() { + }; + + /** + * The tag for the main output. + */ + static final TupleTag> PROCESSING_OUT = + new TupleTag>() { + }; + + /* Logger for class. */ + private static final Logger LOG = LoggerFactory.getLogger(GcsIO.class); + + public static final String DEAD_LETTER_PREFIX = "CSV_CONVERTOR"; + + /** + * String/String Coder for FailsafeElement. + */ + private static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER = + FailsafeElementCoder.of( + NullableCoder.of(StringUtf8Coder.of()), NullableCoder.of(StringUtf8Coder.of())); + + /** + * Supported format to read from GCS. + */ + public enum FORMAT { + JSON, + CSV + } + + /** + * Necessary {@link PipelineOptions} options for Pipelines that operate with JSON/CSV data in + * GCS. + */ + public interface GcsPipelineOptions extends PipelineOptions { + + @Description("GCS filepattern for files in bucket to read data from") + String getInputGcsFilePattern(); + + void setOutputGcsDirectory(String outputGcsDirectory); + + @Description("File format of input files. Supported formats: JSON, CSV") + @Default.Enum("JSON") + GcsIO.FORMAT getInputGcsFileFormat(); + + void setInputGcsFilePattern(String inputGcsFilePattern); + + @Description("GCS directory in bucket to write data to") + String getOutputGcsDirectory(); + + void setInputGcsFileFormat(FORMAT inputGcsFileFormat); + + @Description("File format of output files. Supported formats: JSON, CSV") + @Default.Enum("JSON") + GcsIO.FORMAT getOutputGcsFileFormat(); + + void setOutputGcsFileFormat(FORMAT outputGcsFileFormat); + + // CSV parameters + @Description("If file(s) contain headers") + Boolean getCsvContainsHeaders(); + + void setCsvContainsHeaders(Boolean csvContainsHeaders); + + @Description("Delimiting character in CSV. Default: use delimiter provided in csvFormat") + @Default.InstanceFactory(CsvConverters.DelimiterFactory.class) + String getCsvDelimiter(); + + void setCsvDelimiter(String csvDelimiter); + + @Description( + "Csv format according to Apache Commons CSV format. Default is: Apache Commons CSV" + + " default\n" + + "https://static.javadoc.io/org.apache.commons/commons-csv/1.7/org/apache/commons/csv/CSVFormat.html#DEFAULT\n" + + "Must match format names exactly found at: " + + "https://static.javadoc.io/org.apache.commons/commons-csv/1.7/org/apache/commons/csv/CSVFormat.Predefined.html") + @Default.String("Default") + String getCsvFormat(); + + void setCsvFormat(String csvFormat); + } + + private final ProtegrityDataTokenizationOptions options; + + public GcsIO(ProtegrityDataTokenizationOptions options) { + this.options = options; + } + + public PCollection read(Pipeline pipeline, String schema) { + if (options.getInputGcsFileFormat() == FORMAT.JSON) { + return pipeline.apply( + "ReadJsonFromGCSFiles", TextIO.read().from(options.getInputGcsFilePattern())); + } else if (options.getInputGcsFileFormat() == FORMAT.CSV) { + PCollectionTuple jsons = + pipeline + /* + * Step 1: Read CSV file(s) from Cloud Storage using {@link CsvConverters.ReadCsv}. + */ + .apply( + "ReadCsvFromGcsFiles", + CsvConverters.ReadCsv.newBuilder() + .setCsvFormat(options.getCsvFormat()) + .setDelimiter(options.getCsvDelimiter()) + .setHasHeaders(options.getCsvContainsHeaders()) + .setInputFileSpec(options.getInputGcsFilePattern()) + .setHeaderTag(CSV_HEADERS) + .setLineTag(CSV_LINES) + .build()) + /* + * Step 2: Convert lines to Json. + */ + .apply( + "LineToJson", + CsvConverters.LineToFailsafeJson.newBuilder() + .setDelimiter(options.getCsvDelimiter()) + .setJsonSchema(schema) + .setHeaderTag(CSV_HEADERS) + .setLineTag(CSV_LINES) + .setUdfOutputTag(PROCESSING_OUT) + .setUdfDeadletterTag(PROCESSING_DEADLETTER_OUT) + .build()); + + if (options.getNonTokenizedDeadLetterGcsPath() != null) { + /* + * Step 3: Write jsons to dead-letter gcs that were successfully processed. + */ + jsons + .get(PROCESSING_DEADLETTER_OUT) + .apply( + "WriteCsvConversionErrorsToGcs", + ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder() + .setCsvDelimiter(options.getCsvDelimiter()) + .setErrorWritePath(options.getNonTokenizedDeadLetterGcsPath()) + .build()); + } + + /* + * Step 4: Get jsons that were successfully processed. + */ + return jsons + .get(PROCESSING_OUT) + .apply( + "GetJson", + MapElements.into(TypeDescriptors.strings()).via(FailsafeElement::getPayload)); + } else { + throw new IllegalStateException( + "No valid format for input data is provided. Please, choose JSON or CSV."); + } + } + + public PDone write(PCollection input, Schema schema) { + if (options.getOutputGcsFileFormat() == FORMAT.JSON) { + return input + .apply("RowsToJSON", ToJson.of()) + .apply("WriteToGCS", TextIO.write().to(options.getOutputGcsDirectory())); + } else if (options.getOutputGcsFileFormat() == FORMAT.CSV) { + String header = String.join(options.getCsvDelimiter(), schema.getFieldNames()); + String csvDelimiter = options.getCsvDelimiter(); + return input + .apply( + "ConvertToCSV", + MapElements.into(TypeDescriptors.strings()) + .via((Row inputRow) -> new RowToCsv(csvDelimiter).getCsvFromRow(inputRow))) + .apply( + "WriteToGCS", TextIO.write().to(options.getOutputGcsDirectory()).withHeader(header)); + + } else { + throw new IllegalStateException( + "No valid format for output data is provided. Please, choose JSON or CSV."); + } + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/package-info.java new file mode 100644 index 000000000000..0c1c03c0295c --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2020 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** Protegrity Data Tokenization template for Google Cloud Teleport. */ +package org.apache.beam.examples.complete.datatokenization.transforms.io; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/package-info.java new file mode 100644 index 000000000000..5530e633d727 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2020 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** Protegrity Data Tokenization template for Google Cloud Teleport. */ +package org.apache.beam.examples.complete.datatokenization.transforms; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/RowToCsv.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/RowToCsv.java new file mode 100644 index 000000000000..2b4f3b32469b --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/RowToCsv.java @@ -0,0 +1,35 @@ +/* + * 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.examples.complete.datatokenization.utils; + +import java.util.stream.Collectors; +import org.apache.beam.sdk.values.Row; + +/** The {@link RowToCsv} class to convert Beam Rows into strings in CSV format. */ +public class RowToCsv { + + private final String csvDelimiter; + + public RowToCsv(String csvDelimiter) { + this.csvDelimiter = csvDelimiter; + } + + public String getCsvFromRow(Row row) { + return row.getValues().stream().map(Object::toString).collect(Collectors.joining(csvDelimiter)); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java new file mode 100644 index 000000000000..938e80ac5e6f --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java @@ -0,0 +1,136 @@ +/* + * 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.examples.complete.datatokenization.utils; + +import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.fromTableSchema; + +import com.google.api.services.bigquery.model.TableSchema; +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Type; +import java.nio.channels.Channels; +import java.nio.channels.ReadableByteChannel; +import java.nio.charset.Charset; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; +import org.apache.beam.vendor.grpc.v1p26p0.com.google.common.io.ByteStreams; +import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.Gson; +import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.reflect.TypeToken; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link SchemasUtils} Class to read JSON based schema. Is there available to read from file or + * from string. Currently supported local File System and GCS. + */ +public class SchemasUtils { + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(SchemasUtils.class); + + private TableSchema bigQuerySchema; + private Schema beamSchema; + private String jsonBeamSchema; + + public SchemasUtils(String schema) { + parseJson(schema); + } + + public SchemasUtils(String path, Charset encoding) throws IOException { + if (path.startsWith("gs://")) { + parseJson(new String(readGcsFile(path), encoding)); + } else { + byte[] encoded = Files.readAllBytes(Paths.get(path)); + parseJson(new String(encoded, encoding)); + } + LOG.info("Extracted schema: " + bigQuerySchema.toPrettyString()); + } + + public TableSchema getBigQuerySchema() { + return bigQuerySchema; + } + + private void parseJson(String jsonSchema) throws UnsupportedOperationException { + TableSchema schema = BigQueryHelpers.fromJsonString(jsonSchema, TableSchema.class); + validateSchemaTypes(schema); + bigQuerySchema = schema; + jsonBeamSchema = BigQueryHelpers.toJsonString(schema.getFields()); + } + + private void validateSchemaTypes(TableSchema bigQuerySchema) { + try { + beamSchema = fromTableSchema(bigQuerySchema); + } catch (UnsupportedOperationException exception) { + LOG.error("Check json schema, {}", exception.getMessage()); + } catch (NullPointerException npe) { + LOG.error("Missing schema keywords, please check what all required fields presented"); + } + } + + /** + * Method to read a schema file from GCS and return the file contents as a string. + * + * @param gcsFilePath path to file in GCS in format "gs://your-bucket/path/to/file" + * @return byte array with file contents + * @throws IOException thrown if not able to read file + */ + public static byte[] readGcsFile(String gcsFilePath) throws IOException { + LOG.info("Reading contents from GCS file: {}", gcsFilePath); + // Read the GCS file into byte array and will throw an I/O exception in case file not found. + try (ReadableByteChannel readerChannel = + FileSystems.open(FileSystems.matchSingleFileSpec(gcsFilePath).resourceId())) { + try (InputStream stream = Channels.newInputStream(readerChannel)) { + return ByteStreams.toByteArray(stream); + } + } + } + + public Map getDataElementsToTokenize(String payloadConfigGcsPath) { + Map dataElements; + try { + String rawJsonWithDataElements = + new String(readGcsFile(payloadConfigGcsPath), Charset.defaultCharset()); + Gson gson = new Gson(); + Type type = new TypeToken>() {}.getType(); + dataElements = gson.fromJson(rawJsonWithDataElements, type); + } catch (IOException | NullPointerException exception) { + LOG.error( + "Cant parse fields to tokenize, or input parameter payloadConfigGcsPath was not specified." + + " All fields will be sent to the protectors"); + dataElements = + this.getBeamSchema().getFields().stream() + .collect(Collectors.toMap(Field::getName, e -> "")); + } + return dataElements; + } + + public Schema getBeamSchema() { + return beamSchema; + } + + public String getJsonBeamSchema() { + return jsonBeamSchema; + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/package-info.java new file mode 100644 index 000000000000..e53a4f7ce046 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2020 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** Protegrity Data Tokenization template for Google Cloud Teleport. */ +package org.apache.beam.examples.complete.datatokenization.utils; From ba046cbfcfe35b6e3463233960b4e0fc8369e54f Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Mon, 11 Jan 2021 16:31:21 +0300 Subject: [PATCH 02/29] Move to beam repo --- examples/java/build.gradle | 1 + .../ProtegrityDataTokenization.java | 162 +++-- .../transforms/CsvConverters.java | 657 ++++++++++++++++++ .../transforms/ErrorConverters.java | 205 ++++++ .../transforms/ProtegrityDataProtectors.java | 4 +- .../transforms/io/BigQueryIO.java | 49 +- .../datatokenization/transforms/io/GcsIO.java | 2 + .../utils/FailsafeElement.java | 113 +++ .../utils/FailsafeElementCoder.java | 107 +++ .../datatokenization/utils/SchemasUtils.java | 48 ++ 10 files changed, 1260 insertions(+), 88 deletions(-) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/CsvConverters.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ErrorConverters.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElementCoder.java diff --git a/examples/java/build.gradle b/examples/java/build.gradle index a1b6827a98ec..19b224eab2fd 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -74,6 +74,7 @@ dependencies { compile library.java.proto_google_cloud_datastore_v1 compile library.java.slf4j_api compile library.java.slf4j_jdk14 + provided library.java.commons_io runtime project(path: ":runners:direct-java", configuration: "shadow") testCompile project(":sdks:java:io:google-cloud-platform") testCompile project(":sdks:java:extensions:ml") diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java index 7cd22dd2cfab..0756a7e2cd97 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java @@ -17,22 +17,29 @@ */ package org.apache.beam.examples.complete.datatokenization; +import static org.apache.beam.examples.complete.datatokenization.utils.SchemasUtils.DEADLETTER_SCHEMA; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Map; import org.apache.beam.examples.complete.datatokenization.options.ProtegrityDataTokenizationOptions; +import org.apache.beam.examples.complete.datatokenization.transforms.ErrorConverters; import org.apache.beam.examples.complete.datatokenization.transforms.ProtegrityDataProtectors.RowToTokenizedRow; import org.apache.beam.examples.complete.datatokenization.transforms.io.BigQueryIO; import org.apache.beam.examples.complete.datatokenization.transforms.io.BigTableIO; import org.apache.beam.examples.complete.datatokenization.transforms.io.GcsIO; +import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement; +import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; +import org.apache.beam.examples.complete.datatokenization.utils.RowToCsv; import org.apache.beam.examples.complete.datatokenization.utils.SchemasUtils; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.gcp.bigquery.WriteResult; @@ -59,6 +66,13 @@ public class ProtegrityDataTokenization { */ private static final Logger LOG = LoggerFactory.getLogger(ProtegrityDataTokenization.class); + /** + * String/String Coder for FailsafeElement. + */ + private static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER = + FailsafeElementCoder.of( + NullableCoder.of(StringUtf8Coder.of()), NullableCoder.of(StringUtf8Coder.of())); + /** * The default suffix for error tables if dead letter table is not specified. */ @@ -70,6 +84,13 @@ public class ProtegrityDataTokenization { private static final TupleTag TOKENIZATION_OUT = new TupleTag() { }; + /** + * The tag for the dead-letter output of the udf. + */ + static final TupleTag> TOKENIZATION_DEADLETTER_OUT = + new TupleTag>() { + }; + /** * Main entry point for pipeline execution. * @@ -109,44 +130,54 @@ public static PipelineResult run( // Register the coder for pipeline CoderRegistry coderRegistry = pipeline.getCoderRegistry(); coderRegistry.registerCoderForType( - RowCoder.of(schema.getBeamSchema()).getEncodedTypeDescriptor(), - RowCoder.of(schema.getBeamSchema())); + FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor(), FAILSAFE_ELEMENT_CODER); + coderRegistry + .registerCoderForType(RowCoder.of(schema.getBeamSchema()).getEncodedTypeDescriptor(), + RowCoder.of(schema.getBeamSchema())); + + /* + * Row/Row Coder for FailsafeElement. + */ + FailsafeElementCoder coder = FailsafeElementCoder.of( + RowCoder.of(schema.getBeamSchema()), + RowCoder.of(schema.getBeamSchema()) + ); + + coderRegistry + .registerCoderForType(coder.getEncodedTypeDescriptor(), coder); PCollection jsons; if (options.getInputGcsFilePattern() != null) { jsons = new GcsIO(options).read(pipeline, schema.getJsonBeamSchema()); } else if (options.getPubsubTopic() != null) { - jsons = - pipeline.apply( - "ReadMessagesFromPubsub", PubsubIO.readStrings().fromTopic(options.getPubsubTopic())); + jsons = pipeline + .apply("ReadMessagesFromPubsub", + PubsubIO.readStrings().fromTopic(options.getPubsubTopic())); } else { throw new IllegalStateException("No source is provided, please configure GCS or Pub/Sub"); } - JsonToRow.ParseResult rows = - jsons.apply( - "JsonToRow", + JsonToRow.ParseResult rows = jsons + .apply("JsonToRow", JsonToRow.withExceptionReporting(schema.getBeamSchema()).withExtendedErrorInfo()); -// if (options.getNonTokenizedDeadLetterGcsPath() != null) { -// /* -// * Write Row conversion errors to filesystem specified path -// */ -// rows.getFailedToParseLines() -// .apply( -// "ToFailsafeElement", -// MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor()) -// .via( -// (Row errRow) -> -// FailsafeElement.of(errRow.getString("line"), errRow.getString("line")) -// .setErrorMessage(errRow.getString("err")))) -// .apply( -// "WriteCsvConversionErrorsToGcs", -// ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder() -// .setCsvDelimiter(options.getCsvDelimiter()) -// .setErrorWritePath(options.getNonTokenizedDeadLetterGcsPath()) -// .build()); -// } + if (options.getNonTokenizedDeadLetterGcsPath() != null) { + /* + * Write Row conversion errors to filesystem specified path + */ + rows.getFailedToParseLines() + .apply("ToFailsafeElement", + MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor()) + .via((Row errRow) -> FailsafeElement + .of(errRow.getString("line"), errRow.getString("line")) + .setErrorMessage(errRow.getString("err")) + )) + .apply("WriteCsvConversionErrorsToGcs", + ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder() + .setCsvDelimiter(options.getCsvDelimiter()) + .setErrorWritePath(options.getNonTokenizedDeadLetterGcsPath()) + .build()); + } /* Tokenize data using remote API call */ @@ -169,51 +200,54 @@ public static PipelineResult run( .setFailureTag(TOKENIZATION_DEADLETTER_OUT) .build()); -// String csvDelimiter = options.getCsvDelimiter(); -// if (options.getNonTokenizedDeadLetterGcsPath() != null) { -// /* -// Write tokenization errors to dead-letter sink -// */ -// tokenizedRows -// .get(TOKENIZATION_DEADLETTER_OUT) -// .apply( -// "ConvertToCSV", -// MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor()) -// .via( -// (FailsafeElement fse) -> -// FailsafeElement.of( -// new RowToCsv(csvDelimiter).getCsvFromRow(fse.getOriginalPayload()), -// new RowToCsv(csvDelimiter).getCsvFromRow(fse.getPayload())))) -// .apply( -// "WriteTokenizationErrorsToGcs", -// ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder() -// .setCsvDelimiter(options.getCsvDelimiter()) -// .setErrorWritePath(options.getNonTokenizedDeadLetterGcsPath()) -// .build()); -// } + String csvDelimiter = options.getCsvDelimiter(); + if (options.getNonTokenizedDeadLetterGcsPath() != null) { + /* + Write tokenization errors to dead-letter sink + */ + tokenizedRows + .get(TOKENIZATION_DEADLETTER_OUT) + .apply( + "ConvertToCSV", + MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor()) + .via( + (FailsafeElement fse) -> + FailsafeElement.of( + new RowToCsv(csvDelimiter).getCsvFromRow(fse.getOriginalPayload()), + new RowToCsv(csvDelimiter).getCsvFromRow(fse.getPayload())))) + .apply( + "WriteTokenizationErrorsToGcs", + ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder() + .setCsvDelimiter(options.getCsvDelimiter()) + .setErrorWritePath(options.getNonTokenizedDeadLetterGcsPath()) + .build()); + } if (options.getOutputGcsDirectory() != null) { - new GcsIO(options).write(tokenizedRows.get(TOKENIZATION_OUT), schema.getBeamSchema()); + new GcsIO(options).write( + tokenizedRows.get(TOKENIZATION_OUT), + schema.getBeamSchema() + ); } else if (options.getBigQueryTableName() != null) { WriteResult writeResult = BigQueryIO.write( tokenizedRows.get(TOKENIZATION_OUT), options.getBigQueryTableName(), schema.getBigQuerySchema()); -// writeResult -// .getFailedInsertsWithErr() -// .apply( -// "WrapInsertionErrors", -// MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor()) -// .via(BigQueryIO::wrapBigQueryInsertError)) -// .setCoder(FAILSAFE_ELEMENT_CODER) -// .apply( -// "WriteInsertionFailedRecords", -// ErrorConverters.WriteStringMessageErrors.newBuilder() -// .setErrorRecordsTable( -// options.getBigQueryTableName() + DEFAULT_DEADLETTER_TABLE_SUFFIX) -// .setErrorRecordsTableSchema(SchemaUtils.DEADLETTER_SCHEMA) -// .build()); + writeResult + .getFailedInsertsWithErr() + .apply( + "WrapInsertionErrors", + MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor()) + .via(BigQueryIO::wrapBigQueryInsertError)) + .setCoder(FAILSAFE_ELEMENT_CODER) + .apply( + "WriteInsertionFailedRecords", + ErrorConverters.WriteStringMessageErrors.newBuilder() + .setErrorRecordsTable( + options.getBigQueryTableName() + DEFAULT_DEADLETTER_TABLE_SUFFIX) + .setErrorRecordsTableSchema(DEADLETTER_SCHEMA) + .build()); } else if (options.getBigTableInstanceId() != null) { new BigTableIO(options).write(tokenizedRows.get(TOKENIZATION_OUT), schema.getBeamSchema()); } else { diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/CsvConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/CsvConverters.java new file mode 100644 index 000000000000..b01e7beed132 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/CsvConverters.java @@ -0,0 +1,657 @@ +/* + * Copyright (C) 2019 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.datatokenization.transforms; + +import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.value.AutoValue; +import com.google.cloud.teleport.v2.transforms.JavascriptTextTransformer.FailsafeJavascriptUdf; +import com.google.cloud.teleport.v2.transforms.JavascriptTextTransformer.JavascriptTextTransformerOptions; +import com.google.cloud.teleport.v2.utils.SchemaUtils; +import com.google.cloud.teleport.v2.values.FailsafeElement; +import com.google.gson.JsonArray; +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; +import com.google.gson.stream.JsonWriter; +import java.io.IOException; +import java.io.StringReader; +import java.io.StringWriter; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.io.FileIO; +import org.apache.beam.sdk.io.FileIO.ReadableFile; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.DefaultValueFactory; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Sample; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Splitter; +import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Throwables; +import org.apache.commons.csv.CSVFormat; +import org.apache.commons.csv.CSVParser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Common transforms for Csv files. */ +public class CsvConverters { + + /* Logger for class. */ + private static final Logger LOG = LoggerFactory.getLogger(CsvConverters.class); + + private static final String SUCCESSFUL_TO_JSON_COUNTER = "SuccessfulToJsonCounter"; + + private static final String FAILED_TO_JSON_COUNTER = "FailedToJsonCounter"; + + private static JsonParser jsonParser = new JsonParser(); + + /** + * Builds Json string from list of values and headers or values and schema if schema is provided. + * + * @param headers optional list of strings which is the header of the Csv file. + * @param values list of strings which are combined with header or json schema to create Json + * string. + * @param jsonSchemaString + * @return Json string containing object. + * @throws IOException thrown if Json object is not able to be written. + * @throws NumberFormatException thrown if value cannot be parsed into type successfully. + */ + static String buildJsonString( + @Nullable List headers, List values, @Nullable String jsonSchemaString) + throws Exception { + + StringWriter stringWriter = new StringWriter(); + JsonWriter writer = new JsonWriter(stringWriter); + + if (jsonSchemaString != null) { + JsonArray jsonSchema = jsonParser.parse(jsonSchemaString).getAsJsonArray(); + writer.beginObject(); + + for (int i = 0; i < jsonSchema.size(); i++) { + JsonObject jsonObject = jsonSchema.get(i).getAsJsonObject(); + String type = jsonObject.get("type").getAsString().toUpperCase(); + writer.name(jsonObject.get("name").getAsString()); + + switch (type) { + case "LONG": + writer.value(Long.parseLong(values.get(i))); + break; + + case "DOUBLE": + writer.value(Double.parseDouble(values.get(i))); + break; + + case "INTEGER": + writer.value(Integer.parseInt(values.get(i))); + break; + + case "SHORT": + writer.value(Short.parseShort(values.get(i))); + break; + + case "BYTE": + writer.value(Byte.parseByte(values.get(i))); + break; + + case "FLOAT": + writer.value(Float.parseFloat(values.get(i))); + break; + + case "TEXT": + case "KEYWORD": + case "STRING": + writer.value(values.get(i)); + break; + + default: + LOG.error("Invalid data type, got: " + type); + throw new RuntimeException("Invalid data type, got: " + type); + } + } + writer.endObject(); + writer.close(); + return stringWriter.toString(); + + } else if (headers != null) { + + writer.beginObject(); + + for (int i = 0; i < headers.size(); i++) { + writer.name(headers.get(i)); + writer.value(values.get(i)); + } + + writer.endObject(); + writer.close(); + return stringWriter.toString(); + + } else { + LOG.error("No headers or schema specified"); + throw new RuntimeException("No headers or schema specified"); + } + } + + /** + * Gets Csv format accoring to Apache Commons CSV. If user + * passed invalid format error is thrown. + */ + public static CSVFormat getCsvFormat(String formatString, @Nullable String delimiter) { + + CSVFormat format = CSVFormat.Predefined.valueOf(formatString).getFormat(); + + // If a delimiter has been passed set it here. + if (delimiter != null) { + return format.withDelimiter(delimiter.charAt(0)); + } + return format; + } + + /** Necessary {@link PipelineOptions} options for Csv Pipelines. */ + public interface CsvPipelineOptions extends PipelineOptions, JavascriptTextTransformerOptions { + @Description("Pattern to where data lives, ex: gs://mybucket/somepath/*.csv") + String getInputFileSpec(); + + void setInputFileSpec(String inputFileSpec); + + @Description("If file(s) contain headers") + Boolean getContainsHeaders(); + + void setContainsHeaders(Boolean containsHeaders); + + @Description("Deadletter table for failed inserts in form: :.") + String getDeadletterTable(); + + void setDeadletterTable(String deadletterTable); + + @Description("Delimiting character. Default: use delimiter provided in csvFormat") + @Default.InstanceFactory(DelimiterFactory.class) + String getDelimiter(); + + void setDelimiter(String delimiter); + + @Description( + "Csv format according to Apache Commons CSV format. Default is: Apache Commons CSV" + + " default\n" + + "https://static.javadoc.io/org.apache.commons/commons-csv/1.7/org/apache/commons/csv/CSVFormat.html#DEFAULT\n" + + "Must match format names exactly found at: " + + "https://static.javadoc.io/org.apache.commons/commons-csv/1.7/org/apache/commons/csv/CSVFormat.Predefined.html") + @Default.String("Default") + String getCsvFormat(); + + void setCsvFormat(String csvFormat); + + @Description("Optional: Path to JSON schema, ex gs://path/to/schema. ") + String getJsonSchemaPath(); + + void setJsonSchemaPath(String jsonSchemaPath); + + @Description("Set to true if number of files is in the tens of thousands. Default: false") + @Default.Boolean(false) + Boolean getLargeNumFiles(); + + void setLargeNumFiles(Boolean largeNumFiles); + } + + /** + * Default value factory to get delimiter from Csv format so that if the user does not pass one + * in, it matches the supplied {@link CsvPipelineOptions#getCsvFormat()}. + */ + public static class DelimiterFactory implements DefaultValueFactory { + + @Override + public String create(PipelineOptions options) { + CSVFormat csvFormat = getCsvFormat(options.as(CsvPipelineOptions.class).getCsvFormat(), null); + return String.valueOf(csvFormat.getDelimiter()); + } + } + + /** + * The {@link LineToFailsafeJson} interface converts a line from a Csv file into a Json string. + * Uses either: Javascript Udf, Json schema or the headers of the file to create the Json object + * which is then added to the {@link FailsafeElement} as the new payload. + */ + @AutoValue + public abstract static class LineToFailsafeJson + extends PTransform { + + public static Builder newBuilder() { + return new AutoValue_CsvConverters_LineToFailsafeJson.Builder(); + } + + public abstract String delimiter(); + + @Nullable + public abstract String udfFileSystemPath(); + + @Nullable + public abstract String udfFunctionName(); + + @Nullable + public abstract String jsonSchemaPath(); + + @Nullable + public abstract String jsonSchema(); + + public abstract TupleTag headerTag(); + + public abstract TupleTag lineTag(); + + public abstract TupleTag> udfOutputTag(); + + public abstract TupleTag> udfDeadletterTag(); + + @Override + public PCollectionTuple expand(PCollectionTuple lines) { + + PCollectionView headersView = null; + + // Convert csv lines into Failsafe elements so that we can recover over multiple transforms. + PCollection> lineFailsafeElements = + lines + .get(lineTag()) + .apply("LineToFailsafeElement", ParDo.of(new LineToFailsafeElementFn())); + + // If UDF is specified then use that to parse csv lines. + if (udfFileSystemPath() != null) { + + return lineFailsafeElements.apply( + "LineToDocumentUsingUdf", + FailsafeJavascriptUdf.newBuilder() + .setFileSystemPath(udfFileSystemPath()) + .setFunctionName(udfFunctionName()) + .setSuccessTag(udfOutputTag()) + .setFailureTag(udfDeadletterTag()) + .build()); + } + + // If no udf then use json schema + if (jsonSchemaPath() != null || jsonSchema() != null) { + + String schema; + if (jsonSchemaPath() != null) { + schema = SchemaUtils.getGcsFileAsString(jsonSchemaPath()); + } else { + schema = jsonSchema(); + } + + return lineFailsafeElements.apply( + "LineToDocumentUsingSchema", + ParDo.of( + new FailsafeElementToJsonFn( + headersView, schema, delimiter(), udfDeadletterTag())) + .withOutputTags(udfOutputTag(), TupleTagList.of(udfDeadletterTag()))); + } + + // Run if using headers + headersView = lines.get(headerTag()).apply(Sample.any(1)).apply(View.asSingleton()); + + PCollectionView finalHeadersView = headersView; + lines + .get(headerTag()) + .apply( + "CheckHeaderConsistency", + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) { + String headers = c.sideInput(finalHeadersView); + if (!c.element().equals(headers)) { + LOG.error("Headers do not match, consistency cannot be guaranteed"); + throw new RuntimeException( + "Headers do not match, consistency cannot be guaranteed"); + } + } + }) + .withSideInputs(finalHeadersView)); + + return lineFailsafeElements.apply( + "LineToDocumentWithHeaders", + ParDo.of( + new FailsafeElementToJsonFn( + headersView, jsonSchemaPath(), delimiter(), udfDeadletterTag())) + .withSideInputs(headersView) + .withOutputTags(udfOutputTag(), TupleTagList.of(udfDeadletterTag()))); + } + + /** Builder for {@link LineToFailsafeJson}. */ + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setDelimiter(String delimiter); + + public abstract Builder setUdfFileSystemPath(String udfFileSystemPath); + + public abstract Builder setUdfFunctionName(String udfFunctionName); + + public abstract Builder setJsonSchemaPath(String jsonSchemaPath); + + public abstract Builder setJsonSchema(String jsonSchema); + + public abstract Builder setHeaderTag(TupleTag headerTag); + + public abstract Builder setLineTag(TupleTag lineTag); + + public abstract Builder setUdfOutputTag( + TupleTag> udfOutputTag); + + public abstract Builder setUdfDeadletterTag( + TupleTag> udfDeadletterTag); + + public abstract LineToFailsafeJson build(); + } + } + + /** + * The {@link FailsafeElementToJsonFn} class creates a Json string from a failsafe element. + * + *

{@link FailsafeElementToJsonFn#FailsafeElementToJsonFn(PCollectionView, String, String, + * TupleTag)} + */ + public static class FailsafeElementToJsonFn + extends DoFn, FailsafeElement> { + + @Nullable public final String jsonSchema; + public final String delimiter; + public final TupleTag> udfDeadletterTag; + @Nullable private final PCollectionView headersView; + private Counter successCounter = + Metrics.counter(FailsafeElementToJsonFn.class, SUCCESSFUL_TO_JSON_COUNTER); + private Counter failedCounter = + Metrics.counter(FailsafeElementToJsonFn.class, FAILED_TO_JSON_COUNTER); + + FailsafeElementToJsonFn( + PCollectionView headersView, + String jsonSchema, + String delimiter, + TupleTag> udfDeadletterTag) { + this.headersView = headersView; + this.jsonSchema = jsonSchema; + this.delimiter = delimiter; + this.udfDeadletterTag = udfDeadletterTag; + } + + @ProcessElement + public void processElement(ProcessContext context) { + FailsafeElement element = context.element(); + List header = null; + + if (this.headersView != null) { + header = Arrays.asList(context.sideInput(this.headersView).split(this.delimiter)); + } + + List record = Arrays.asList(element.getOriginalPayload().split(this.delimiter)); + + try { + String json = buildJsonString(header, record, this.jsonSchema); + context.output(FailsafeElement.of(element.getOriginalPayload(), json)); + successCounter.inc(); + } catch (Exception e) { + failedCounter.inc(); + context.output( + this.udfDeadletterTag, + FailsafeElement.of(element) + .setErrorMessage(e.getMessage()) + .setStacktrace(Throwables.getStackTraceAsString(e))); + } + } + } + + /** + * The {@link LineToFailsafeElementFn} wraps an csv line with the {@link FailsafeElement} class so + * errors can be recovered from and the original message can be output to a error records table. + */ + static class LineToFailsafeElementFn extends DoFn> { + + @ProcessElement + public void processElement(ProcessContext context) { + String message = context.element(); + context.output(FailsafeElement.of(message, message)); + } + } + + /** + * The {@link ReadCsv} class is a {@link PTransform} that reads from one for more Csv files. The + * transform returns a {@link PCollectionTuple} consisting of the following {@link PCollection}: + * + *

    + *
  • {@link ReadCsv#headerTag()} - Contains headers found in files if read with headers, + * contains empty {@link PCollection} if no headers. + *
  • {@link ReadCsv#lineTag()} - Contains Csv lines as a {@link PCollection} of strings. + *
+ */ + @AutoValue + public abstract static class ReadCsv extends PTransform { + + public static Builder newBuilder() { + return new AutoValue_CsvConverters_ReadCsv.Builder(); + } + + public abstract String csvFormat(); + + @Nullable + public abstract String delimiter(); + + public abstract Boolean hasHeaders(); + + public abstract String inputFileSpec(); + + public abstract TupleTag headerTag(); + + public abstract TupleTag lineTag(); + + @Override + public PCollectionTuple expand(PBegin input) { + + if (hasHeaders()) { + return input + .apply("MatchFilePattern", FileIO.match().filepattern(inputFileSpec())) + .apply("ReadMatches", FileIO.readMatches()) + .apply( + "ReadCsvWithHeaders", + ParDo.of(new GetCsvHeadersFn(headerTag(), lineTag(), csvFormat(), delimiter())) + .withOutputTags(headerTag(), TupleTagList.of(lineTag()))); + } + + return PCollectionTuple.of( + lineTag(), input.apply("ReadCsvWithoutHeaders", TextIO.read().from(inputFileSpec()))); + } + + /** Builder for {@link ReadCsv}. */ + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setCsvFormat(String csvFormat); + + public abstract Builder setDelimiter(@Nullable String delimiter); + + public abstract Builder setHasHeaders(Boolean hasHeaders); + + public abstract Builder setInputFileSpec(String inputFileSpec); + + public abstract Builder setHeaderTag(TupleTag headerTag); + + public abstract Builder setLineTag(TupleTag lineTag); + + abstract ReadCsv autoBuild(); + + public ReadCsv build() { + + ReadCsv readCsv = autoBuild(); + + checkArgument(readCsv.inputFileSpec() != null, "Input file spec must be provided."); + + checkArgument(readCsv.csvFormat() != null, "Csv format must not be null."); + + checkArgument(readCsv.hasHeaders() != null, "Header information must be provided."); + + return readCsv; + } + } + } + + /** + * The {@link GetCsvHeadersFn} class gets the header of a Csv file and outputs it as a string. The + * csv format provided in {@link CsvConverters#getCsvFormat(String, String)} is used to get the + * header. + */ + static class GetCsvHeadersFn extends DoFn { + + private final TupleTag headerTag; + private final TupleTag linesTag; + private CSVFormat csvFormat; + + GetCsvHeadersFn( + TupleTag headerTag, TupleTag linesTag, String csvFormat, String delimiter) { + this.headerTag = headerTag; + this.linesTag = linesTag; + this.csvFormat = getCsvFormat(csvFormat, delimiter); + } + + @ProcessElement + public void processElement(ProcessContext context, MultiOutputReceiver outputReceiver) { + ReadableFile f = context.element(); + String headers; + List records = null; + String delimiter = String.valueOf(this.csvFormat.getDelimiter()); + try { + String csvFileString = f.readFullyAsUTF8String(); + StringReader reader = new StringReader(csvFileString); + CSVParser parser = CSVParser.parse(reader, this.csvFormat.withFirstRecordAsHeader()); + records = + parser.getRecords().stream() + .map(i -> String.join(delimiter, i)) + .collect(Collectors.toList()); + headers = String.join(delimiter, parser.getHeaderNames()); + } catch (IOException ioe) { + LOG.error("Headers do not match, consistency cannot be guaranteed"); + throw new RuntimeException("Could not read Csv headers: " + ioe.getMessage()); + } + outputReceiver.get(this.headerTag).output(headers); + records.forEach(r -> outputReceiver.get(this.linesTag).output(r)); + } + } + + /** + * The {@link StringToGenericRecordFn} class takes in a String as input and outputs a {@link + * GenericRecord}. + */ + public static class StringToGenericRecordFn extends DoFn { + String schemaLocation; + String delimiter; + Schema schema; + + public StringToGenericRecordFn(String schemaLocation, String delimiter) { + this.schemaLocation = schemaLocation; + this.delimiter = delimiter; + } + + @Setup + public void setup() { + schema = SchemaUtils.getAvroSchema(schemaLocation); + } + + @ProcessElement + public void processElement(ProcessContext context) throws IllegalArgumentException { + GenericRecord genericRecord = new GenericData.Record(schema); + String[] rowValue = + Splitter.on(delimiter).splitToList(context.element()).toArray(new String[0]); + List fields = schema.getFields(); + + try { + for (int index = 0; index < fields.size(); ++index) { + Schema.Field field = fields.get(index); + String fieldType = field.schema().getType().getName().toLowerCase(); + + // Handle null values to be added in generic records, if present in Csv data. + if (fieldType.equals("union")) { + String dataType1 = field.schema().getTypes().get(0).getType().getName().toLowerCase(); + String dataType2 = field.schema().getTypes().get(1).getType().getName().toLowerCase(); + + // Check if Csv data is null. + if ((dataType1.equals("null") || dataType2.equals("null")) + && rowValue[index].length() == 0) { + genericRecord.put(field.name(), null); + } else { + // Add valid data type to generic record. + if (dataType1.equals("null")) { + populateGenericRecord(genericRecord, dataType2, rowValue[index], field.name()); + } else { + populateGenericRecord(genericRecord, dataType1, rowValue[index], field.name()); + } + } + } else { + populateGenericRecord(genericRecord, fieldType, rowValue[index], field.name()); + } + } + } catch (ArrayIndexOutOfBoundsException e) { + LOG.error("Number of fields in the Avro schema and number of Csv headers do not match."); + throw new RuntimeException( + "Number of fields in the Avro schema and number of Csv headers do not match."); + } + context.output(genericRecord); + } + + private void populateGenericRecord( + GenericRecord genericRecord, String fieldType, String data, String fieldName) { + + try { + switch (fieldType) { + case "string": + genericRecord.put(fieldName, data); + break; + case "int": + genericRecord.put(fieldName, Integer.valueOf(data)); + break; + case "long": + genericRecord.put(fieldName, Long.valueOf(data)); + break; + case "float": + genericRecord.put(fieldName, Float.valueOf(data)); + break; + case "double": + genericRecord.put(fieldName, Double.valueOf(data)); + break; + case "boolean": + genericRecord.put(fieldName, Boolean.valueOf(data)); + break; + default: + LOG.error(fieldType + " field type is not supported."); + throw new IllegalArgumentException(fieldType + " field type is not supported."); + } + } catch (Exception e) { + LOG.error("Failed to convert Strings to Generic Record."); + throw new RuntimeException("Failed to convert Strings to Generic Record."); + } + } + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ErrorConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ErrorConverters.java new file mode 100644 index 000000000000..4dd05ed166a0 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ErrorConverters.java @@ -0,0 +1,205 @@ +/* + * Copyright (C) 2019 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.datatokenization.transforms; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.auto.value.AutoValue; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; +import org.apache.beam.sdk.io.gcp.bigquery.WriteResult; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; + +/** + * Transforms & DoFns & Options for Teleport Error logging. + */ +public class ErrorConverters { + + /** + * Writes all Errors to GCS, place at the end of your pipeline. + */ + @AutoValue + public abstract static class WriteStringMessageErrorsAsCsv extends + PTransform>, PDone> { + + public static Builder newBuilder() { + return new AutoValue_ErrorConverters_WriteStringMessageErrorsAsCsv.Builder(); + } + + public abstract String errorWritePath(); + + public abstract String csvDelimiter(); + + @Override + public PDone expand(PCollection> pCollection) { + + return pCollection + .apply("GetFormattedErrorRow", + ParDo.of(new FailedStringToCsvRowFn(csvDelimiter()))) + .apply(TextIO.write().to(errorWritePath()).withNumShards(1)); + } + + /** + * Builder for {@link WriteStringMessageErrorsAsCsv}. + */ + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setErrorWritePath(String errorWritePath); + + public abstract Builder setCsvDelimiter(String csvDelimiter); + + public abstract WriteStringMessageErrorsAsCsv build(); + } + } + + /** + * The {@link FailedStringToCsvRowFn} converts string objects which have failed processing into + * {@link String} objects contained CSV which can be output to a filesystem. + */ + public static class FailedStringToCsvRowFn + extends DoFn, String> { + + /** + * The formatter used to convert timestamps into a BigQuery compatible format. + */ + private static final DateTimeFormatter TIMESTAMP_FORMATTER = + DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSS"); + + private final String csvDelimiter; + + public FailedStringToCsvRowFn(String csvDelimiter) { + this.csvDelimiter = csvDelimiter; + } + + public FailedStringToCsvRowFn() { + this.csvDelimiter = ","; + } + + @ProcessElement + public void processElement(ProcessContext context) { + FailsafeElement failsafeElement = context.element(); + ArrayList outputRow = new ArrayList<>(); + final String message = failsafeElement.getOriginalPayload(); + + // Format the timestamp for insertion + String timestamp = + TIMESTAMP_FORMATTER.print(context.timestamp().toDateTime(DateTimeZone.UTC)); + + outputRow.add(timestamp); + outputRow.add(failsafeElement.getErrorMessage()); + + // Only set the payload if it's populated on the message. + if (message != null) { + outputRow.add(message); + } + + context.output(String.join(csvDelimiter, outputRow)); + } + } + + public abstract static class WriteStringMessageErrors + extends PTransform>, WriteResult> { + + public static Builder newBuilder() { + return new AutoValue_ErrorConverters_WriteStringMessageErrors.Builder(); + } + + public abstract String getErrorRecordsTable(); + + public abstract String getErrorRecordsTableSchema(); + + @Override + public WriteResult expand(PCollection> failedRecords) { + + return failedRecords + .apply("FailedRecordToTableRow", ParDo.of(new FailedStringToTableRowFn())) + .apply( + "WriteFailedRecordsToBigQuery", + BigQueryIO.writeTableRows() + .to(getErrorRecordsTable()) + .withJsonSchema(getErrorRecordsTableSchema()) + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(WriteDisposition.WRITE_APPEND)); + } + + /** + * Builder for {@link WriteStringMessageErrors}. + */ + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setErrorRecordsTable(String errorRecordsTable); + + public abstract Builder setErrorRecordsTableSchema(String errorRecordsTableSchema); + + public abstract WriteStringMessageErrors build(); + } + } + + /** + * The {@link FailedStringToTableRowFn} converts string objects which have failed processing into + * {@link TableRow} objects which can be output to a dead-letter table. + */ + public static class FailedStringToTableRowFn + extends DoFn, TableRow> { + + /** + * The formatter used to convert timestamps into a BigQuery compatible format. + */ + private static final DateTimeFormatter TIMESTAMP_FORMATTER = + DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSSSS"); + + @ProcessElement + public void processElement(ProcessContext context) { + FailsafeElement failsafeElement = context.element(); + final String message = failsafeElement.getOriginalPayload(); + + // Format the timestamp for insertion + String timestamp = + TIMESTAMP_FORMATTER.print(context.timestamp().toDateTime(DateTimeZone.UTC)); + + // Build the table row + final TableRow failedRow = + new TableRow() + .set("timestamp", timestamp) + .set("errorMessage", failsafeElement.getErrorMessage()) + .set("stacktrace", failsafeElement.getStacktrace()); + + // Only set the payload if it's populated on the message. + if (message != null) { + failedRow + .set("payloadString", message) + .set("payloadBytes", message.getBytes(StandardCharsets.UTF_8)); + } + + context.output(failedRow); + } + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java index f7919afd5909..05c699ccca3e 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java @@ -34,6 +34,8 @@ import java.util.Map.Entry; import java.util.UUID; import java.util.stream.Collectors; +import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement; +import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; @@ -143,7 +145,7 @@ public abstract static class Builder { public abstract Builder setBatchSize(int batchSize); public abstract Builder setDataElements(Map fieldsDataElements); - + public abstract Builder setDsgURI(String dsgURI); public abstract RowToTokenizedRow build(); diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java index d680e3ab5e5d..d61ed8ccfa12 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java @@ -19,6 +19,9 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; +import java.io.IOException; +import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryInsertError; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; import org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy; import org.apache.beam.sdk.io.gcp.bigquery.WriteResult; @@ -60,29 +63,29 @@ public static WriteResult write( .to(bigQueryTableName)); } -// /** -// * Method to wrap a {@link BigQueryInsertError} into a {@link FailsafeElement}. -// * -// * @param insertError BigQueryInsert error. -// * @return FailsafeElement object. -// */ -// public static FailsafeElement wrapBigQueryInsertError( -// BigQueryInsertError insertError) { -// -// FailsafeElement failsafeElement; -// try { -// -// failsafeElement = -// FailsafeElement.of( -// insertError.getRow().toPrettyString(), insertError.getRow().toPrettyString()); -// failsafeElement.setErrorMessage(insertError.getError().toPrettyString()); -// -// } catch (IOException e) { -// BigQueryIO.LOG.error("Failed to wrap BigQuery insert error."); -// throw new RuntimeException(e); -// } -// return failsafeElement; -// } + /** + * Method to wrap a {@link BigQueryInsertError} into a {@link FailsafeElement}. + * + * @param insertError BigQueryInsert error. + * @return FailsafeElement object. + */ + public static FailsafeElement wrapBigQueryInsertError( + BigQueryInsertError insertError) { + + FailsafeElement failsafeElement; + try { + + failsafeElement = + FailsafeElement.of( + insertError.getRow().toPrettyString(), insertError.getRow().toPrettyString()); + failsafeElement.setErrorMessage(insertError.getError().toPrettyString()); + + } catch (IOException e) { + BigQueryIO.LOG.error("Failed to wrap BigQuery insert error."); + throw new RuntimeException(e); + } + return failsafeElement; + } /** * The {@link RowToTableRowFn} class converts a row to tableRow using {@link diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java index dea28ebe7a0d..8ce76448ac82 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java @@ -18,6 +18,8 @@ package org.apache.beam.examples.complete.datatokenization.transforms.io; import org.apache.beam.examples.complete.datatokenization.options.ProtegrityDataTokenizationOptions; +import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement; +import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; import org.apache.beam.examples.complete.datatokenization.utils.RowToCsv; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.NullableCoder; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java new file mode 100644 index 000000000000..69121ff41b22 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java @@ -0,0 +1,113 @@ +/* + * Copyright (C) 2019 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.datatokenization.utils; + +import java.util.Objects; +import org.apache.avro.reflect.Nullable; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; + +/** + * The {@link FailsafeElement} class holds the current value and original value of a record within a + * pipeline. This class allows pipelines to not lose valuable information about an incoming record + * throughout the processing of that record. The use of this class allows for more robust + * dead-letter strategies as the original record information is not lost throughout the pipeline and + * can be output to a dead-letter in the event of a failure during one of the pipelines transforms. + */ +@DefaultCoder(FailsafeElementCoder.class) +public class FailsafeElement { + + private final OriginalT originalPayload; + private final CurrentT payload; + @Nullable + private String errorMessage; + @Nullable + private String stacktrace; + + private FailsafeElement(OriginalT originalPayload, CurrentT payload) { + this.originalPayload = originalPayload; + this.payload = payload; + } + + public static FailsafeElement of( + OriginalT originalPayload, CurrentT currentPayload) { + return new FailsafeElement<>(originalPayload, currentPayload); + } + + public static FailsafeElement of( + FailsafeElement other) { + return new FailsafeElement<>(other.originalPayload, other.payload) + .setErrorMessage(other.getErrorMessage()) + .setStacktrace(other.getStacktrace()); + } + + public OriginalT getOriginalPayload() { + return originalPayload; + } + + public CurrentT getPayload() { + return payload; + } + + public String getErrorMessage() { + return errorMessage; + } + + public FailsafeElement setErrorMessage(String errorMessage) { + this.errorMessage = errorMessage; + return this; + } + + public String getStacktrace() { + return stacktrace; + } + + public FailsafeElement setStacktrace(String stacktrace) { + this.stacktrace = stacktrace; + return this; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + final FailsafeElement other = (FailsafeElement) obj; + return Objects.deepEquals(this.originalPayload, other.getOriginalPayload()) + && Objects.deepEquals(this.payload, other.getPayload()) + && Objects.deepEquals(this.errorMessage, other.getErrorMessage()) + && Objects.deepEquals(this.stacktrace, other.getStacktrace()); + } + + @Override + public int hashCode() { + return Objects.hash(originalPayload, payload, errorMessage, stacktrace); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("originalPayload", originalPayload) + .add("payload", payload) + .add("errorMessage", errorMessage) + .add("stacktrace", stacktrace) + .toString(); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElementCoder.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElementCoder.java new file mode 100644 index 000000000000..8c971050da01 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElementCoder.java @@ -0,0 +1,107 @@ +/* + * Copyright (C) 2019 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.datatokenization.utils; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeParameter; + +/** + * The {@link FailsafeElementCoder} encodes and decodes {@link FailsafeElement} objects. + * + *

This coder is necessary until Avro supports parameterized types (AVRO-1571) without requiring to + * explicitly specifying the schema for the type. + * + * @param The type of the original payload to be encoded. + * @param The type of the current payload to be encoded. + */ +public class FailsafeElementCoder + extends CustomCoder> { + + private static final NullableCoder STRING_CODER = NullableCoder.of(StringUtf8Coder.of()); + private final Coder originalPayloadCoder; + private final Coder currentPayloadCoder; + + private FailsafeElementCoder( + Coder originalPayloadCoder, Coder currentPayloadCoder) { + this.originalPayloadCoder = originalPayloadCoder; + this.currentPayloadCoder = currentPayloadCoder; + } + + public Coder getOriginalPayloadCoder() { + return originalPayloadCoder; + } + + public Coder getCurrentPayloadCoder() { + return currentPayloadCoder; + } + + public static FailsafeElementCoder of( + Coder originalPayloadCoder, Coder currentPayloadCoder) { + return new FailsafeElementCoder<>(originalPayloadCoder, currentPayloadCoder); + } + + @Override + public void encode(FailsafeElement value, OutputStream outStream) + throws IOException { + if (value == null) { + throw new CoderException("The FailsafeElementCoder cannot encode a null object!"); + } + + originalPayloadCoder.encode(value.getOriginalPayload(), outStream); + currentPayloadCoder.encode(value.getPayload(), outStream); + STRING_CODER.encode(value.getErrorMessage(), outStream); + STRING_CODER.encode(value.getStacktrace(), outStream); + } + + @Override + public FailsafeElement decode(InputStream inStream) throws IOException { + + OriginalT originalPayload = originalPayloadCoder.decode(inStream); + CurrentT currentPayload = currentPayloadCoder.decode(inStream); + String errorMessage = STRING_CODER.decode(inStream); + String stacktrace = STRING_CODER.decode(inStream); + + return FailsafeElement.of(originalPayload, currentPayload) + .setErrorMessage(errorMessage) + .setStacktrace(stacktrace); + } + + @Override + public List> getCoderArguments() { + return Arrays.asList(originalPayloadCoder, currentPayloadCoder); + } + + @Override + public TypeDescriptor> getEncodedTypeDescriptor() { + return new TypeDescriptor>() { + }.where( + new TypeParameter() { + }, originalPayloadCoder.getEncodedTypeDescriptor()) + .where(new TypeParameter() { + }, currentPayloadCoder.getEncodedTypeDescriptor()); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java index 938e80ac5e6f..e6f0e7594794 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java @@ -133,4 +133,52 @@ public Schema getBeamSchema() { public String getJsonBeamSchema() { return jsonBeamSchema; } + + public static final String DEADLETTER_SCHEMA = + "{\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"timestamp\",\n" + + " \"type\": \"TIMESTAMP\",\n" + + " \"mode\": \"REQUIRED\"\n" + + " },\n" + + " {\n" + + " \"name\": \"payloadString\",\n" + + " \"type\": \"STRING\",\n" + + " \"mode\": \"REQUIRED\"\n" + + " },\n" + + " {\n" + + " \"name\": \"payloadBytes\",\n" + + " \"type\": \"BYTES\",\n" + + " \"mode\": \"REQUIRED\"\n" + + " },\n" + + " {\n" + + " \"name\": \"attributes\",\n" + + " \"type\": \"RECORD\",\n" + + " \"mode\": \"REPEATED\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"key\",\n" + + " \"type\": \"STRING\",\n" + + " \"mode\": \"NULLABLE\"\n" + + " },\n" + + " {\n" + + " \"name\": \"value\",\n" + + " \"type\": \"STRING\",\n" + + " \"mode\": \"NULLABLE\"\n" + + " }\n" + + " ]\n" + + " },\n" + + " {\n" + + " \"name\": \"errorMessage\",\n" + + " \"type\": \"STRING\",\n" + + " \"mode\": \"NULLABLE\"\n" + + " },\n" + + " {\n" + + " \"name\": \"stacktrace\",\n" + + " \"type\": \"STRING\",\n" + + " \"mode\": \"NULLABLE\"\n" + + " }\n" + + " ]\n" + + "}"; } From df5881fca38d1fff6173114300f6bc72a6339f72 Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Tue, 12 Jan 2021 10:45:46 +0300 Subject: [PATCH 03/29] moved convertors for GCSio --- examples/java/build.gradle | 1 + .../ProtegrityDataTokenization.java | 2 +- .../datatokenization/transforms/io/GcsIO.java | 2 + .../{transforms => utils}/CsvConverters.java | 114 +------ .../ErrorConverters.java | 2 +- .../utils/JavascriptTextTransformer.java | 292 ++++++++++++++++++ .../datatokenization/utils/SchemasUtils.java | 41 +++ 7 files changed, 345 insertions(+), 109 deletions(-) rename examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/{transforms => utils}/CsvConverters.java (80%) rename examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/{transforms => utils}/ErrorConverters.java (99%) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java diff --git a/examples/java/build.gradle b/examples/java/build.gradle index 19b224eab2fd..86990d9bfa74 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -75,6 +75,7 @@ dependencies { compile library.java.slf4j_api compile library.java.slf4j_jdk14 provided library.java.commons_io + provided library.java.commons_csv runtime project(path: ":runners:direct-java", configuration: "shadow") testCompile project(":sdks:java:io:google-cloud-platform") testCompile project(":sdks:java:extensions:ml") diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java index 0756a7e2cd97..9cfd13e34ad5 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java @@ -24,7 +24,7 @@ import java.nio.charset.StandardCharsets; import java.util.Map; import org.apache.beam.examples.complete.datatokenization.options.ProtegrityDataTokenizationOptions; -import org.apache.beam.examples.complete.datatokenization.transforms.ErrorConverters; +import org.apache.beam.examples.complete.datatokenization.utils.ErrorConverters; import org.apache.beam.examples.complete.datatokenization.transforms.ProtegrityDataProtectors.RowToTokenizedRow; import org.apache.beam.examples.complete.datatokenization.transforms.io.BigQueryIO; import org.apache.beam.examples.complete.datatokenization.transforms.io.BigTableIO; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java index 8ce76448ac82..41ddfc0649c8 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java @@ -18,6 +18,8 @@ package org.apache.beam.examples.complete.datatokenization.transforms.io; import org.apache.beam.examples.complete.datatokenization.options.ProtegrityDataTokenizationOptions; +import org.apache.beam.examples.complete.datatokenization.utils.CsvConverters; +import org.apache.beam.examples.complete.datatokenization.utils.ErrorConverters; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; import org.apache.beam.examples.complete.datatokenization.utils.RowToCsv; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/CsvConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java similarity index 80% rename from examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/CsvConverters.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java index b01e7beed132..b0f2a0595960 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/CsvConverters.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java @@ -13,15 +13,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.complete.datatokenization.transforms; +package org.apache.beam.examples.complete.datatokenization.utils; -import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.examples.complete.datatokenization.utils.SchemasUtils.getGcsFileAsString; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import com.google.auto.value.AutoValue; -import com.google.cloud.teleport.v2.transforms.JavascriptTextTransformer.FailsafeJavascriptUdf; -import com.google.cloud.teleport.v2.transforms.JavascriptTextTransformer.JavascriptTextTransformerOptions; -import com.google.cloud.teleport.v2.utils.SchemaUtils; -import com.google.cloud.teleport.v2.values.FailsafeElement; import com.google.gson.JsonArray; import com.google.gson.JsonObject; import com.google.gson.JsonParser; @@ -33,9 +30,7 @@ import java.util.List; import java.util.stream.Collectors; import javax.annotation.Nullable; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; +import org.apache.beam.examples.complete.datatokenization.utils.JavascriptTextTransformer.FailsafeJavascriptUdf; import org.apache.beam.sdk.io.FileIO; import org.apache.beam.sdk.io.FileIO.ReadableFile; import org.apache.beam.sdk.io.TextIO; @@ -56,8 +51,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Splitter; -import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Throwables; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables; import org.apache.commons.csv.CSVFormat; import org.apache.commons.csv.CSVParser; import org.slf4j.Logger; @@ -178,7 +172,7 @@ public static CSVFormat getCsvFormat(String formatString, @Nullable String delim } /** Necessary {@link PipelineOptions} options for Csv Pipelines. */ - public interface CsvPipelineOptions extends PipelineOptions, JavascriptTextTransformerOptions { + public interface CsvPipelineOptions extends PipelineOptions { @Description("Pattern to where data lives, ex: gs://mybucket/somepath/*.csv") String getInputFileSpec(); @@ -300,7 +294,7 @@ public PCollectionTuple expand(PCollectionTuple lines) { String schema; if (jsonSchemaPath() != null) { - schema = SchemaUtils.getGcsFileAsString(jsonSchemaPath()); + schema = getGcsFileAsString(jsonSchemaPath()); } else { schema = jsonSchema(); } @@ -560,98 +554,4 @@ public void processElement(ProcessContext context, MultiOutputReceiver outputRec records.forEach(r -> outputReceiver.get(this.linesTag).output(r)); } } - - /** - * The {@link StringToGenericRecordFn} class takes in a String as input and outputs a {@link - * GenericRecord}. - */ - public static class StringToGenericRecordFn extends DoFn { - String schemaLocation; - String delimiter; - Schema schema; - - public StringToGenericRecordFn(String schemaLocation, String delimiter) { - this.schemaLocation = schemaLocation; - this.delimiter = delimiter; - } - - @Setup - public void setup() { - schema = SchemaUtils.getAvroSchema(schemaLocation); - } - - @ProcessElement - public void processElement(ProcessContext context) throws IllegalArgumentException { - GenericRecord genericRecord = new GenericData.Record(schema); - String[] rowValue = - Splitter.on(delimiter).splitToList(context.element()).toArray(new String[0]); - List fields = schema.getFields(); - - try { - for (int index = 0; index < fields.size(); ++index) { - Schema.Field field = fields.get(index); - String fieldType = field.schema().getType().getName().toLowerCase(); - - // Handle null values to be added in generic records, if present in Csv data. - if (fieldType.equals("union")) { - String dataType1 = field.schema().getTypes().get(0).getType().getName().toLowerCase(); - String dataType2 = field.schema().getTypes().get(1).getType().getName().toLowerCase(); - - // Check if Csv data is null. - if ((dataType1.equals("null") || dataType2.equals("null")) - && rowValue[index].length() == 0) { - genericRecord.put(field.name(), null); - } else { - // Add valid data type to generic record. - if (dataType1.equals("null")) { - populateGenericRecord(genericRecord, dataType2, rowValue[index], field.name()); - } else { - populateGenericRecord(genericRecord, dataType1, rowValue[index], field.name()); - } - } - } else { - populateGenericRecord(genericRecord, fieldType, rowValue[index], field.name()); - } - } - } catch (ArrayIndexOutOfBoundsException e) { - LOG.error("Number of fields in the Avro schema and number of Csv headers do not match."); - throw new RuntimeException( - "Number of fields in the Avro schema and number of Csv headers do not match."); - } - context.output(genericRecord); - } - - private void populateGenericRecord( - GenericRecord genericRecord, String fieldType, String data, String fieldName) { - - try { - switch (fieldType) { - case "string": - genericRecord.put(fieldName, data); - break; - case "int": - genericRecord.put(fieldName, Integer.valueOf(data)); - break; - case "long": - genericRecord.put(fieldName, Long.valueOf(data)); - break; - case "float": - genericRecord.put(fieldName, Float.valueOf(data)); - break; - case "double": - genericRecord.put(fieldName, Double.valueOf(data)); - break; - case "boolean": - genericRecord.put(fieldName, Boolean.valueOf(data)); - break; - default: - LOG.error(fieldType + " field type is not supported."); - throw new IllegalArgumentException(fieldType + " field type is not supported."); - } - } catch (Exception e) { - LOG.error("Failed to convert Strings to Generic Record."); - throw new RuntimeException("Failed to convert Strings to Generic Record."); - } - } - } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ErrorConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java similarity index 99% rename from examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ErrorConverters.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java index 4dd05ed166a0..cf12f2909ee7 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ErrorConverters.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.complete.datatokenization.transforms; +package org.apache.beam.examples.complete.datatokenization.utils; import com.google.api.services.bigquery.model.TableRow; import com.google.auto.value.AutoValue; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java new file mode 100644 index 000000000000..893a4ccfc886 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java @@ -0,0 +1,292 @@ +/* + * Copyright (C) 2019 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.datatokenization.utils; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.io.Reader; +import java.io.UncheckedIOException; +import java.nio.channels.Channels; +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import javax.script.Invocable; +import javax.script.ScriptEngine; +import javax.script.ScriptEngineManager; +import javax.script.ScriptException; +import jdk.nashorn.api.scripting.ScriptObjectMirror; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.io.fs.MatchResult.Metadata; +import org.apache.beam.sdk.io.fs.MatchResult.Status; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.CharStreams; + +/** A Text UDF Transform Function. Note that this class's implementation is not threadsafe */ +@AutoValue +public abstract class JavascriptTextTransformer { + /** + * The {@link FailsafeJavascriptUdf} class processes user-defined functions is a fail-safe manner + * by maintaining the original payload post-transformation and outputting to a dead-letter on + * failure. + */ + @AutoValue + public abstract static class FailsafeJavascriptUdf + extends PTransform>, PCollectionTuple> { + public abstract @Nullable String fileSystemPath(); + + public abstract @Nullable String functionName(); + + public abstract TupleTag> successTag(); + + public abstract TupleTag> failureTag(); + + public static Builder newBuilder() { + return new AutoValue_JavascriptTextTransformer_FailsafeJavascriptUdf.Builder<>(); + } + + private Counter successCounter = Metrics.counter(FailsafeJavascriptUdf.class, "udf-transform-success-count"); + + private Counter failedCounter = Metrics.counter(FailsafeJavascriptUdf.class, "udf-transform-failed-count"); + + /** Builder for {@link FailsafeJavascriptUdf}. */ + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setFileSystemPath(@Nullable String fileSystemPath); + + public abstract Builder setFunctionName(@Nullable String functionName); + + public abstract Builder setSuccessTag(TupleTag> successTag); + + public abstract Builder setFailureTag(TupleTag> failureTag); + + public abstract FailsafeJavascriptUdf build(); + } + + @Override + public PCollectionTuple expand(PCollection> elements) { + return elements.apply( + "ProcessUdf", + ParDo.of( + new DoFn, FailsafeElement>() { + private JavascriptRuntime javascriptRuntime; + + @Setup + public void setup() { + if (fileSystemPath() != null && functionName() != null) { + javascriptRuntime = + getJavascriptRuntime(fileSystemPath(), functionName()); + } + } + + @ProcessElement + public void processElement(ProcessContext context) { + FailsafeElement element = context.element(); + String payloadStr = element.getPayload(); + + try { + if (javascriptRuntime != null) { + payloadStr = javascriptRuntime.invoke(payloadStr); + } + + if (!Strings.isNullOrEmpty(payloadStr)) { + context.output( + FailsafeElement.of(element.getOriginalPayload(), payloadStr)); + successCounter.inc(); + } + } catch (Exception e) { + context.output( + failureTag(), + FailsafeElement.of(element) + .setErrorMessage(e.getMessage()) + .setStacktrace(Throwables.getStackTraceAsString(e))); + failedCounter.inc(); + } + } + }) + .withOutputTags(successTag(), TupleTagList.of(failureTag()))); + } + } + + /** + * Grabs code from a FileSystem, loads it into the Nashorn Javascript Engine, and executes + * Javascript Functions. + */ + @AutoValue + public abstract static class JavascriptRuntime { + @Nullable + public abstract String fileSystemPath(); + + @Nullable + public abstract String functionName(); + + private Invocable invocable; + + /** Builder for {@link JavascriptTextTransformer}. */ + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setFileSystemPath(@Nullable String fileSystemPath); + + public abstract Builder setFunctionName(@Nullable String functionName); + + public abstract JavascriptRuntime build(); + } + + /** + * Factory method for generating a JavascriptTextTransformer.Builder. + * + * @return a JavascriptTextTransformer builder + */ + public static Builder newBuilder() { + return new AutoValue_JavascriptTextTransformer_JavascriptRuntime.Builder(); + } + + /** + * Gets a cached Javascript Invocable, if fileSystemPath() not set, returns null. + * + * @return a Javascript Invocable or null + */ + @Nullable + public Invocable getInvocable() throws ScriptException, IOException { + + // return null if no UDF path specified. + if (Strings.isNullOrEmpty(fileSystemPath())) { + return null; + } + + if (invocable == null) { + Collection scripts = getScripts(fileSystemPath()); + invocable = newInvocable(scripts); + } + return invocable; + } + + /** + * Factory method for making a new Invocable. + * + * @param scripts a collection of javascript scripts encoded with UTF8 to load in + */ + @Nullable + private static Invocable newInvocable(Collection scripts) throws ScriptException { + ScriptEngineManager manager = new ScriptEngineManager(); + ScriptEngine engine = manager.getEngineByName("JavaScript"); + + for (String script : scripts) { + engine.eval(script); + } + + return (Invocable) engine; + } + + /** + * Invokes the UDF with specified data. + * + * @param data data to pass to the invocable function + * @return The data transformed by the UDF in String format + */ + @Nullable + public String invoke(String data) throws ScriptException, IOException, NoSuchMethodException { + Invocable invocable = getInvocable(); + if (invocable == null) { + throw new RuntimeException("No udf was loaded"); + } + + Object result = getInvocable().invokeFunction(functionName(), data); + if (result == null || ScriptObjectMirror.isUndefined(result)) { + return null; + + } else if (result instanceof String) { + return (String) result; + + } else { + String className = result.getClass().getName(); + throw new RuntimeException( + "UDF Function did not return a String. Instead got: " + className); + } + } + + /** + * Loads into memory scripts from a File System from a given path. Supports any file system that + * {@link FileSystems} supports. + * + * @return a collection of scripts loaded as UF8 Strings + */ + private static Collection getScripts(String path) throws IOException { + MatchResult result = FileSystems.match(path); + checkArgument( + result.status() == Status.OK && !result.metadata().isEmpty(), + "Failed to match any files with the pattern: " + path); + + List scripts = + result + .metadata() + .stream() + .filter(metadata -> metadata.resourceId().getFilename().endsWith(".js")) + .map(Metadata::resourceId) + .map( + resourceId -> { + try (Reader reader = + Channels.newReader( + FileSystems.open(resourceId), StandardCharsets.UTF_8.name())) { + return CharStreams.toString(reader); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }) + .collect(Collectors.toList()); + + return scripts; + } + } + + /** + * Retrieves a {@link JavascriptRuntime} configured to invoke the specified function within the + * script. If either the fileSystemPath or functionName is null or empty, this method will return + * null indicating that a runtime was unable to be created within the given parameters. + * + * @param fileSystemPath The file path to the JavaScript file to execute. + * @param functionName The function name which will be invoked within the JavaScript script. + * @return The {@link JavascriptRuntime} instance. + */ + private static JavascriptRuntime getJavascriptRuntime( + String fileSystemPath, String functionName) { + JavascriptRuntime javascriptRuntime = null; + + if (!Strings.isNullOrEmpty(fileSystemPath) && !Strings.isNullOrEmpty(functionName)) { + javascriptRuntime = + JavascriptRuntime.newBuilder() + .setFunctionName(functionName) + .setFileSystemPath(fileSystemPath) + .build(); + } + + return javascriptRuntime; + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java index e6f0e7594794..fe24a8aadcc1 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java @@ -18,26 +18,33 @@ package org.apache.beam.examples.complete.datatokenization.utils; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.fromTableSchema; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import com.google.api.services.bigquery.model.TableSchema; import java.io.IOException; import java.io.InputStream; +import java.io.Reader; import java.lang.reflect.Type; import java.nio.channels.Channels; import java.nio.channels.ReadableByteChannel; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.stream.Collectors; 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.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.vendor.grpc.v1p26p0.com.google.common.io.ByteStreams; import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.Gson; import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.reflect.TypeToken; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.CharStreams; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -134,6 +141,40 @@ public String getJsonBeamSchema() { return jsonBeamSchema; } + /** + * Reads a file from GCS and returns it as a + * string. + * + * @param filePath path to file in GCS + * @return contents of the file as a string + * @throws IOException thrown if not able to read file + */ + public static String getGcsFileAsString(String filePath) { + MatchResult result; + try { + result = FileSystems.match(filePath); + checkArgument( + result.status() == MatchResult.Status.OK && !result.metadata().isEmpty(), + "Failed to match any files with the pattern: " + filePath); + + List rId = + result.metadata().stream() + .map(MatchResult.Metadata::resourceId) + .collect(Collectors.toList()); + + checkArgument(rId.size() == 1, "Expected exactly 1 file, but got " + rId.size() + " files."); + + Reader reader = + Channels.newReader(FileSystems.open(rId.get(0)), StandardCharsets.UTF_8.name()); + + return CharStreams.toString(reader); + + } catch (IOException ioe) { + LOG.error("File system i/o error: " + ioe.getMessage()); + throw new RuntimeException(ioe); + } + } + public static final String DEADLETTER_SCHEMA = "{\n" + " \"fields\": [\n" From 1e9c3361ddb9dd12ddc8b8b39a1c16cd46a894c1 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Tue, 12 Jan 2021 17:15:52 +0300 Subject: [PATCH 04/29] Renaming + readme --- ...okenization.java => DataTokenization.java} | 96 ++++------ .../complete/datatokenization/README.md | 178 ++++++++++++++++++ ...ataProtectors.java => DataProtectors.java} | 124 ++++-------- .../transforms/io/BigQueryIO.java | 8 +- .../transforms/io/BigTableIO.java | 8 +- .../datatokenization/transforms/io/GcsIO.java | 43 ++--- .../datatokenization/utils/CsvConverters.java | 12 +- .../utils/ErrorConverters.java | 39 ++-- .../utils/FailsafeElement.java | 18 +- .../utils/FailsafeElementCoder.java | 23 ++- .../utils/JavascriptTextTransformer.java | 89 ++++----- .../datatokenization/utils/SchemasUtils.java | 3 +- 12 files changed, 354 insertions(+), 287 deletions(-) rename examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/{ProtegrityDataTokenization.java => DataTokenization.java} (80%) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md rename examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/{ProtegrityDataProtectors.java => DataProtectors.java} (74%) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java similarity index 80% rename from examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java index 9cfd13e34ad5..10fd43fb9cf2 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java @@ -22,13 +22,12 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.Map; import org.apache.beam.examples.complete.datatokenization.options.ProtegrityDataTokenizationOptions; -import org.apache.beam.examples.complete.datatokenization.utils.ErrorConverters; -import org.apache.beam.examples.complete.datatokenization.transforms.ProtegrityDataProtectors.RowToTokenizedRow; +import org.apache.beam.examples.complete.datatokenization.transforms.DataProtectors.RowToTokenizedRow; import org.apache.beam.examples.complete.datatokenization.transforms.io.BigQueryIO; import org.apache.beam.examples.complete.datatokenization.transforms.io.BigTableIO; import org.apache.beam.examples.complete.datatokenization.transforms.io.GcsIO; +import org.apache.beam.examples.complete.datatokenization.utils.ErrorConverters; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; import org.apache.beam.examples.complete.datatokenization.utils.RowToCsv; @@ -56,40 +55,26 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * The {@link ProtegrityDataTokenization} pipeline. - */ -public class ProtegrityDataTokenization { +/** The {@link DataTokenization} pipeline. */ +public class DataTokenization { - /** - * Logger for class. - */ - private static final Logger LOG = LoggerFactory.getLogger(ProtegrityDataTokenization.class); + /** Logger for class. */ + private static final Logger LOG = LoggerFactory.getLogger(DataTokenization.class); - /** - * String/String Coder for FailsafeElement. - */ + /** String/String Coder for FailsafeElement. */ private static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER = FailsafeElementCoder.of( NullableCoder.of(StringUtf8Coder.of()), NullableCoder.of(StringUtf8Coder.of())); - /** - * The default suffix for error tables if dead letter table is not specified. - */ + /** The default suffix for error tables if dead letter table is not specified. */ private static final String DEFAULT_DEADLETTER_TABLE_SUFFIX = "_error_records"; - /** - * The tag for the main output for the UDF. - */ - private static final TupleTag TOKENIZATION_OUT = new TupleTag() { - }; + /** The tag for the main output for the UDF. */ + private static final TupleTag TOKENIZATION_OUT = new TupleTag() {}; - /** - * The tag for the dead-letter output of the udf. - */ + /** The tag for the dead-letter output of the udf. */ static final TupleTag> TOKENIZATION_DEADLETTER_OUT = - new TupleTag>() { - }; + new TupleTag>() {}; /** * Main entry point for pipeline execution. @@ -112,8 +97,7 @@ public static void main(String[] args) { * @param options The execution options. * @return The pipeline result. */ - public static PipelineResult run( - ProtegrityDataTokenizationOptions options) { + public static PipelineResult run(ProtegrityDataTokenizationOptions options) { SchemasUtils schema = null; try { schema = new SchemasUtils(options.getDataSchemaGcsPath(), StandardCharsets.UTF_8); @@ -122,43 +106,39 @@ public static PipelineResult run( } checkArgument(schema != null, "Data schema is mandatory."); - Map dataElements = - schema.getDataElementsToTokenize(options.getPayloadConfigGcsPath()); - // Create the pipeline Pipeline pipeline = Pipeline.create(options); // Register the coder for pipeline CoderRegistry coderRegistry = pipeline.getCoderRegistry(); coderRegistry.registerCoderForType( FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor(), FAILSAFE_ELEMENT_CODER); - coderRegistry - .registerCoderForType(RowCoder.of(schema.getBeamSchema()).getEncodedTypeDescriptor(), - RowCoder.of(schema.getBeamSchema())); + coderRegistry.registerCoderForType( + RowCoder.of(schema.getBeamSchema()).getEncodedTypeDescriptor(), + RowCoder.of(schema.getBeamSchema())); /* * Row/Row Coder for FailsafeElement. */ - FailsafeElementCoder coder = FailsafeElementCoder.of( - RowCoder.of(schema.getBeamSchema()), - RowCoder.of(schema.getBeamSchema()) - ); + FailsafeElementCoder coder = + FailsafeElementCoder.of( + RowCoder.of(schema.getBeamSchema()), RowCoder.of(schema.getBeamSchema())); - coderRegistry - .registerCoderForType(coder.getEncodedTypeDescriptor(), coder); + coderRegistry.registerCoderForType(coder.getEncodedTypeDescriptor(), coder); PCollection jsons; if (options.getInputGcsFilePattern() != null) { jsons = new GcsIO(options).read(pipeline, schema.getJsonBeamSchema()); } else if (options.getPubsubTopic() != null) { - jsons = pipeline - .apply("ReadMessagesFromPubsub", - PubsubIO.readStrings().fromTopic(options.getPubsubTopic())); + jsons = + pipeline.apply( + "ReadMessagesFromPubsub", PubsubIO.readStrings().fromTopic(options.getPubsubTopic())); } else { throw new IllegalStateException("No source is provided, please configure GCS or Pub/Sub"); } - JsonToRow.ParseResult rows = jsons - .apply("JsonToRow", + JsonToRow.ParseResult rows = + jsons.apply( + "JsonToRow", JsonToRow.withExceptionReporting(schema.getBeamSchema()).withExtendedErrorInfo()); if (options.getNonTokenizedDeadLetterGcsPath() != null) { @@ -166,13 +146,15 @@ public static PipelineResult run( * Write Row conversion errors to filesystem specified path */ rows.getFailedToParseLines() - .apply("ToFailsafeElement", + .apply( + "ToFailsafeElement", MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor()) - .via((Row errRow) -> FailsafeElement - .of(errRow.getString("line"), errRow.getString("line")) - .setErrorMessage(errRow.getString("err")) - )) - .apply("WriteCsvConversionErrorsToGcs", + .via( + (Row errRow) -> + FailsafeElement.of(errRow.getString("line"), errRow.getString("line")) + .setErrorMessage(errRow.getString("err")))) + .apply( + "WriteCsvConversionErrorsToGcs", ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder() .setCsvDelimiter(options.getCsvDelimiter()) .setErrorWritePath(options.getNonTokenizedDeadLetterGcsPath()) @@ -186,16 +168,15 @@ public static PipelineResult run( .setRowSchema(schema.getBeamSchema()) .apply( MapElements.into( - TypeDescriptors.kvs(TypeDescriptors.integers(), TypeDescriptors.rows())) + TypeDescriptors.kvs(TypeDescriptors.integers(), TypeDescriptors.rows())) .via((Row row) -> KV.of(0, row))) .setCoder(KvCoder.of(VarIntCoder.of(), RowCoder.of(schema.getBeamSchema()))) .apply( "DsgTokenization", RowToTokenizedRow.newBuilder() .setBatchSize(options.getBatchSize()) - .setDsgURI(options.getDsgUri()) + .setRpcURI(options.getDsgUri()) .setSchema(schema.getBeamSchema()) - .setDataElements(dataElements) .setSuccessTag(TOKENIZATION_OUT) .setFailureTag(TOKENIZATION_DEADLETTER_OUT) .build()); @@ -224,10 +205,7 @@ public static PipelineResult run( } if (options.getOutputGcsDirectory() != null) { - new GcsIO(options).write( - tokenizedRows.get(TOKENIZATION_OUT), - schema.getBeamSchema() - ); + new GcsIO(options).write(tokenizedRows.get(TOKENIZATION_OUT), schema.getBeamSchema()); } else if (options.getBigQueryTableName() != null) { WriteResult writeResult = BigQueryIO.write( diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md new file mode 100644 index 000000000000..ee6b0233554c --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md @@ -0,0 +1,178 @@ + + +# Apache Beam pipeline example to tokenize data using remote RPC server + +This directory contains an Apache Beam example that creates a pipeline to read data from one of +the supported sources, tokenize data with external API calls to remote RPC server, and write data into one of the supported sinks. + +Supported data formats: + +- JSON +- CSV + +Supported input sources: + +- Local file system +- [Google Cloud Storage](https://cloud.google.com/storage) +- [Google Pub/Sub](https://cloud.google.com/pubsub) + +Supported destination sinks: + +- Local file system +- Google Cloud Storage +- [Google Cloud BigQuery](https://cloud.google.com/bigquery) +- [Cloud BigTable](https://cloud.google.com/bigtable) + +Supported data schema format: + +- JSON with an array of fields described in BigQuery format + +In the main scenario, the template will create an Apache Beam pipeline that will read data in CSV or +JSON format from a specified input source, send the data to an external processing server, receive +processed data, and write it into a specified output sink. + +## Requirements + +- Java 8 +- 1 of supported sources to read data from +- 1 of supported destination sinks to write data into +- A configured RPC to tokenize data + +## Getting Started + +This section describes what is needed to get the template up and running. + +- Gradle preparation +- Local execution +- Running as a Dataflow Template + - Setting Up Project Environment + - Build Data Tokenization Dataflow Flex Template + - Creating the Dataflow Flex Template + - Executing Template + + +## Gradle preparation + +To run this example your `build.gradle` file should contain the following task to execute the pipeline: + +``` +task execute (type:JavaExec) { + main = System.getProperty("mainClass") + classpath = sourceSets.main.runtimeClasspath + systemProperties System.getProperties() + args System.getProperty("exec.args", "").split() +} +``` + +This task allows to run the pipeline via the following command: + +```bash +gradle clean execute -DmainClass=org.apache.beam.examples.complete.datatokenization.DataTokenization \ + -Dexec.args="--= --=" +``` + +## Running the pipeline + +To execute this pipeline, specify the parameters: + +**NOTE:** Any parameters called as GSC also supports local file system. + +- Data schema + - **dataSchemaGcsPath**: Path to data schema (JSON format) in GCS compatible with BigQuery. +- 1 specified input source out of these: + - Google Cloud Storage + - **inputGcsFilePattern**: GCS filepattern for files in bucket to read data from + - **inputGcsFileFormat**: File format of input files. Supported formats: JSON, CSV + - In case if input data is in CSV format: + - **csvContainsHeaders**: `true` if file(s) in bucket to read data from contain headers, + and `false` otherwise + - **csvDelimiter**: Delimiting character in CSV. Default: use delimiter provided in + csvFormat + - **csvFormat**: Csv format according to Apache Commons CSV format. Default is: + [Apache Commons CSV default](https://static.javadoc.io/org.apache.commons/commons-csv/1.7/org/apache/commons/csv/CSVFormat.html#DEFAULT) + . Must match format names exactly found + at: https://static.javadoc.io/org.apache.commons/commons-csv/1.7/org/apache/commons/csv/CSVFormat.Predefined.html + - Google Pub/Sub + - **pubsubTopic**: The Cloud Pub/Sub topic to read from, in the format of ' + projects/yourproject/topics/yourtopic' +- 1 specified output sink out of these: + - Google Cloud Storage + - **outputGcsDirectory**: GCS directory in bucket to write data to + - **outputGcsFileFormat**: File format of output files. Supported formats: JSON, CSV + - **windowDuration**: The window duration in which data will be written. Should be specified + only for 'Pub/Sub -> GCS' case. Defaults to 30s. + + Allowed formats are: + - Ns (for seconds, example: 5s), + - Nm (for minutes, example: 12m), + - Nh (for hours, example: 2h). + - Google Cloud BigQuery + - **bigQueryTableName**: Cloud BigQuery table name to write into + - Cloud BigTable + - **bigTableProjectId**: Id of the project where the Cloud BigTable instance to write into + is located + - **bigTableInstanceId**: Id of the Cloud BigTable instance to write into + - **bigTableTableId**: Id of the Cloud BigTable table to write into + - **bigTableKeyColumnName**: Column name to use as a key in Cloud BigTable + - **bigTableColumnFamilyName**: Column family name to use in Cloud BigTable +- DSG parameters + - **dsgUri**: URI for the API calls to DSG + - **batchSize**: Size of the batch to send to DSG per request + - **payloadConfigGcsPath**: GCS path to the payload configuration file with an array of fields + to extract for tokenization + +The template allows for the user to supply the following optional parameter: + +- **nonTokenizedDeadLetterGcsPath**: GCS folder where failed to tokenize data will be stored + + +in the following format: + +```bash +--dataSchemaGcsPath="path-to-data-schema-in-json-format" +--inputGcsFilePattern="path-patternt-to-input-data" +--outputGcsDirectory="path-to-output-directory" +#example to usage csv +--inputGcsFileFormat="CSV" +--outputGcsFileFormat="CSV" +--csvContainsHeaders="true" +--nonTokenizedDeadLetterGcsPath="path-to-errors-rows-writing" +--batchSize=batch-size-number +--rpcUri=http://host:port/tokenize +``` + +By default, this will run the pipeline locally with the DirectRunner. To change the runner, specify: + +```bash +--runner=YOUR_SELECTED_RUNNER +``` + +See the [documentation](http://beam.apache.org/get-started/quickstart/) and +the [Examples README](../../../../../../../../../README.md) for more information about how to run this example. + + +## Running as a Dataflow Template + + +## Running as a Dataflow Template + +This example also exists as Google Dataflow Template, which you can build and run using Google Cloud Platform. See +this template documentation [README.md](https://github.com/GoogleCloudPlatform/DataflowTemplates/blob/master/v2/protegrity-data-tokenization/README.md) for +more information. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java similarity index 74% rename from examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java index 05c699ccca3e..df7a5c613557 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java @@ -29,17 +29,12 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.UUID; -import java.util.stream.Collectors; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.Schema.Field; -import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.state.StateSpecs; @@ -75,15 +70,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * TODO: Add javadoc. - */ -public class ProtegrityDataProtectors { +/** TODO: Add javadoc. */ +public class DataProtectors { - /** - * Logger for class. - */ - private static final Logger LOG = LoggerFactory.getLogger(ProtegrityDataProtectors.class); + /** Logger for class. */ + private static final Logger LOG = LoggerFactory.getLogger(DataProtectors.class); /** * The {@link RowToTokenizedRow} transform converts {@link Row} to {@link TableRow} objects. The @@ -106,9 +97,7 @@ public static Builder newBuilder() { public abstract int batchSize(); - public abstract Map dataElements(); - - public abstract String dsgURI(); + public abstract String rpcURI(); @Override public PCollectionTuple expand(PCollection> inputRows) { @@ -116,23 +105,17 @@ public PCollectionTuple expand(PCollection> inputRows) { FailsafeElementCoder.of(RowCoder.of(schema()), RowCoder.of(schema())); PCollectionTuple pCollectionTuple = inputRows.apply( - "TokenizeUsingDsg", + "Tokenize", ParDo.of( - new DSGTokenizationFn( - schema(), - batchSize(), - dataElements(), - dsgURI(), - failureTag())) + new TokenizationFn( + schema(), batchSize(), rpcURI(), failureTag())) .withOutputTags(successTag(), TupleTagList.of(failureTag()))); return PCollectionTuple.of( - successTag(), pCollectionTuple.get(successTag()).setRowSchema(schema())) + successTag(), pCollectionTuple.get(successTag()).setRowSchema(schema())) .and(failureTag(), pCollectionTuple.get(failureTag()).setCoder(coder)); } - /** - * Builder for {@link RowToTokenizedRow}. - */ + /** Builder for {@link RowToTokenizedRow}. */ @AutoValue.Builder public abstract static class Builder { @@ -144,30 +127,25 @@ public abstract static class Builder { public abstract Builder setBatchSize(int batchSize); - public abstract Builder setDataElements(Map fieldsDataElements); - - public abstract Builder setDsgURI(String dsgURI); + public abstract Builder setRpcURI(String rpcURI); public abstract RowToTokenizedRow build(); } } - /** - * Class for data tokenization using DSG. - */ - public static class DSGTokenizationFn extends DoFn, Row> { + /** Class for data tokenization. */ + public static class TokenizationFn extends DoFn, Row> { public static final String ID_TOKEN_NAME = "ID"; - private static Schema schemaToDsg; + private static Schema schemaToTokenize; private static CloseableHttpClient httpclient; - private static ObjectMapper objectMapperSerializerForDSG; - private static ObjectMapper objectMapperDeserializerForDSG; + private static ObjectMapper objectMapperSerializerForTokenize; + private static ObjectMapper objectMapperDeserializerForTokenize; private final Schema schema; private final int batchSize; - private final Map dataElements; - private final String dsgURI; + private final String rpcURI; private final TupleTag> failureTag; @StateId("buffer") @@ -183,16 +161,14 @@ public static class DSGTokenizationFn extends DoFn, Row> { private String idFieldName; private Map inputRowsWithIds; - public DSGTokenizationFn( + public TokenizationFn( Schema schema, int batchSize, - Map dataElements, - String dsgURI, + String rpcURI, TupleTag> failureTag) { this.schema = schema; this.batchSize = batchSize; - this.dataElements = dataElements; - this.dsgURI = dsgURI; + this.rpcURI = rpcURI; bufferedEvents = StateSpecs.bag(RowCoder.of(schema)); this.failureTag = failureTag; } @@ -200,38 +176,12 @@ public DSGTokenizationFn( @Setup public void setup() { - List idFieldList = - dataElements.entrySet().stream() - .filter(map -> ID_TOKEN_NAME.equals(map.getValue())) - .map(Entry::getKey) - .collect(Collectors.toList()); - - // If we have more than 1 ID fields, we will choose the first. - if (idFieldList.size() > 0) { - idFieldName = idFieldList.get(0); - } - - if (idFieldName == null || !schema.hasField(idFieldName)) { - this.hasIdInInputs = false; - } - - ArrayList fields = new ArrayList<>(); - for (String field : dataElements.keySet()) { - if (schema.hasField(field)) { - fields.add(schema.getField(field)); - } - } - if (!hasIdInInputs) { - idFieldName = ID_TOKEN_NAME; - fields.add(Field.of(ID_TOKEN_NAME, FieldType.STRING)); - dataElements.put(ID_TOKEN_NAME, ID_TOKEN_NAME); - } - schemaToDsg = new Schema(fields); - objectMapperSerializerForDSG = - RowJsonUtils.newObjectMapperWith(RowJson.RowJsonSerializer.forSchema(schemaToDsg)); + schemaToTokenize = new Schema(schema.getFields()); + objectMapperSerializerForTokenize = + RowJsonUtils.newObjectMapperWith(RowJson.RowJsonSerializer.forSchema(schemaToTokenize)); - objectMapperDeserializerForDSG = - RowJsonUtils.newObjectMapperWith(RowJson.RowJsonDeserializer.forSchema(schemaToDsg)); + objectMapperDeserializerForTokenize = + RowJsonUtils.newObjectMapperWith(RowJson.RowJsonDeserializer.forSchema(schemaToTokenize)); httpclient = HttpClients.createDefault(); } @@ -298,8 +248,8 @@ private ArrayList rowsToJsons(Iterable inputRows) { Map inputRowsWithIds = new HashMap<>(); for (Row inputRow : inputRows) { - Row.Builder builder = Row.withSchema(schemaToDsg); - for (Schema.Field field : schemaToDsg.getFields()) { + Row.Builder builder = Row.withSchema(schemaToTokenize); + for (Schema.Field field : schemaToTokenize.getFields()) { if (inputRow.getSchema().hasField(field.getName())) { builder = builder.addValue(inputRow.getValue(field.getName())); } @@ -315,23 +265,19 @@ private ArrayList rowsToJsons(Iterable inputRows) { Row row = builder.build(); - jsons.add(rowToJson(objectMapperSerializerForDSG, row)); + jsons.add(rowToJson(objectMapperSerializerForTokenize, row)); } this.inputRowsWithIds = inputRowsWithIds; return jsons; } - private String formatJsonsToDsgBatch(Iterable jsons) { + private String formatJsonsToRpcBatch(Iterable jsons) { StringBuilder stringBuilder = new StringBuilder(String.join(",", jsons)); Gson gson = new Gson(); - Type gsonType = new TypeToken>() { - }.getType(); - String dataElementsJson = gson.toJson(dataElements, gsonType); + Type gsonType = new TypeToken>() {}.getType(); stringBuilder .append("]") .insert(0, "{\"data\": [") - .append(",\"data_elements\":") - .append(dataElementsJson) .append("}"); return stringBuilder.toString(); } @@ -340,7 +286,7 @@ private ArrayList getTokenizedRow(Iterable inputRows) throws IOExcepti ArrayList outputRows = new ArrayList<>(); CloseableHttpResponse response = - sendToDsg(formatJsonsToDsgBatch(rowsToJsons(inputRows)).getBytes()); + sendRpc(formatJsonsToRpcBatch(rowsToJsons(inputRows)).getBytes()); String tokenizedData = IOUtils.toString(response.getEntity().getContent(), StandardCharsets.UTF_8); @@ -352,10 +298,10 @@ private ArrayList getTokenizedRow(Iterable inputRows) throws IOExcepti for (int i = 0; i < jsonTokenizedRows.size(); i++) { Row tokenizedRow = RowJsonUtils.jsonToRow( - objectMapperDeserializerForDSG, jsonTokenizedRows.get(i).toString()); + objectMapperDeserializerForTokenize, jsonTokenizedRows.get(i).toString()); Row.FieldValueBuilder rowBuilder = Row.fromRow(this.inputRowsWithIds.get(tokenizedRow.getString(idFieldName))); - for (Schema.Field field : schemaToDsg.getFields()) { + for (Schema.Field field : schemaToTokenize.getFields()) { if (!hasIdInInputs && field.getName().equals(idFieldName)) { continue; } @@ -368,8 +314,8 @@ private ArrayList getTokenizedRow(Iterable inputRows) throws IOExcepti return outputRows; } - private CloseableHttpResponse sendToDsg(byte[] data) throws IOException { - HttpPost httpPost = new HttpPost(dsgURI); + private CloseableHttpResponse sendRpc(byte[] data) throws IOException { + HttpPost httpPost = new HttpPost(rpcURI); HttpEntity stringEntity = new ByteArrayEntity(data, ContentType.APPLICATION_JSON); httpPost.setEntity(stringEntity); return httpclient.execute(httpPost); diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java index d61ed8ccfa12..17bcabae025e 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java @@ -32,14 +32,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * The {@link BigQueryIO} class for writing data from template to BigTable. - */ +/** The {@link BigQueryIO} class for writing data from template to BigTable. */ public class BigQueryIO { - /** - * Logger for class. - */ + /** Logger for class. */ private static final Logger LOG = LoggerFactory.getLogger(BigQueryIO.class); public static WriteResult write( diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java index 99e1acd4241e..b597c98443ae 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java @@ -39,14 +39,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * The {@link BigTableIO} class for writing data from template to BigTable. - */ +/** The {@link BigTableIO} class for writing data from template to BigTable. */ public class BigTableIO { - /** - * Logger for class. - */ + /** Logger for class. */ private static final Logger LOG = LoggerFactory.getLogger(BigTableIO.class); private final ProtegrityDataTokenizationOptions options; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java index 41ddfc0649c8..58bdc1510ae0 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java @@ -42,60 +42,41 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * The {@link GcsIO} class to read/write data from/into Google Cloud Storage. - */ +/** The {@link GcsIO} class to read/write data from/into Google Cloud Storage. */ public class GcsIO { - /** - * The tag for the headers of the CSV if required. - */ - static final TupleTag CSV_HEADERS = new TupleTag() { - }; + /** The tag for the headers of the CSV if required. */ + static final TupleTag CSV_HEADERS = new TupleTag() {}; - /** - * The tag for the lines of the CSV. - */ - static final TupleTag CSV_LINES = new TupleTag() { - }; + /** The tag for the lines of the CSV. */ + static final TupleTag CSV_LINES = new TupleTag() {}; - /** - * The tag for the dead-letter output. - */ + /** The tag for the dead-letter output. */ static final TupleTag> PROCESSING_DEADLETTER_OUT = - new TupleTag>() { - }; + new TupleTag>() {}; - /** - * The tag for the main output. - */ + /** The tag for the main output. */ static final TupleTag> PROCESSING_OUT = - new TupleTag>() { - }; + new TupleTag>() {}; /* Logger for class. */ private static final Logger LOG = LoggerFactory.getLogger(GcsIO.class); public static final String DEAD_LETTER_PREFIX = "CSV_CONVERTOR"; - /** - * String/String Coder for FailsafeElement. - */ + /** String/String Coder for FailsafeElement. */ private static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER = FailsafeElementCoder.of( NullableCoder.of(StringUtf8Coder.of()), NullableCoder.of(StringUtf8Coder.of())); - /** - * Supported format to read from GCS. - */ + /** Supported format to read from GCS. */ public enum FORMAT { JSON, CSV } /** - * Necessary {@link PipelineOptions} options for Pipelines that operate with JSON/CSV data in - * GCS. + * Necessary {@link PipelineOptions} options for Pipelines that operate with JSON/CSV data in GCS. */ public interface GcsPipelineOptions extends PipelineOptions { diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java index b0f2a0595960..d39392d2f998 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java @@ -1,9 +1,11 @@ /* - * Copyright (C) 2019 Google Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * 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 * diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java index cf12f2909ee7..bda6b287c9e2 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java @@ -1,9 +1,11 @@ /* - * Copyright (C) 2019 Google Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * 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 * @@ -19,7 +21,6 @@ import com.google.auto.value.AutoValue; import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; @@ -34,17 +35,13 @@ import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -/** - * Transforms & DoFns & Options for Teleport Error logging. - */ +/** Transforms & DoFns & Options for Teleport Error logging. */ public class ErrorConverters { - /** - * Writes all Errors to GCS, place at the end of your pipeline. - */ + /** Writes all Errors to GCS, place at the end of your pipeline. */ @AutoValue - public abstract static class WriteStringMessageErrorsAsCsv extends - PTransform>, PDone> { + public abstract static class WriteStringMessageErrorsAsCsv + extends PTransform>, PDone> { public static Builder newBuilder() { return new AutoValue_ErrorConverters_WriteStringMessageErrorsAsCsv.Builder(); @@ -58,14 +55,11 @@ public static Builder newBuilder() { public PDone expand(PCollection> pCollection) { return pCollection - .apply("GetFormattedErrorRow", - ParDo.of(new FailedStringToCsvRowFn(csvDelimiter()))) + .apply("GetFormattedErrorRow", ParDo.of(new FailedStringToCsvRowFn(csvDelimiter()))) .apply(TextIO.write().to(errorWritePath()).withNumShards(1)); } - /** - * Builder for {@link WriteStringMessageErrorsAsCsv}. - */ + /** Builder for {@link WriteStringMessageErrorsAsCsv}. */ @AutoValue.Builder public abstract static class Builder { @@ -81,8 +75,7 @@ public abstract static class Builder { * The {@link FailedStringToCsvRowFn} converts string objects which have failed processing into * {@link String} objects contained CSV which can be output to a filesystem. */ - public static class FailedStringToCsvRowFn - extends DoFn, String> { + public static class FailedStringToCsvRowFn extends DoFn, String> { /** * The formatter used to convert timestamps into a BigQuery compatible > failedRec .withWriteDisposition(WriteDisposition.WRITE_APPEND)); } - /** - * Builder for {@link WriteStringMessageErrors}. - */ + /** Builder for {@link WriteStringMessageErrors}. */ @AutoValue.Builder public abstract static class Builder { diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java index 69121ff41b22..a3710ba00ed5 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java @@ -1,9 +1,11 @@ /* - * Copyright (C) 2019 Google Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * 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 * @@ -32,10 +34,8 @@ public class FailsafeElement { private final OriginalT originalPayload; private final CurrentT payload; - @Nullable - private String errorMessage; - @Nullable - private String stacktrace; + @Nullable private String errorMessage; + @Nullable private String stacktrace; private FailsafeElement(OriginalT originalPayload, CurrentT payload) { this.originalPayload = originalPayload; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElementCoder.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElementCoder.java index 8c971050da01..151d98a9070e 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElementCoder.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElementCoder.java @@ -1,9 +1,11 @@ /* - * Copyright (C) 2019 Google Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * 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 * @@ -36,7 +38,7 @@ * explicitly specifying the schema for the type. * * @param The type of the original payload to be encoded. - * @param The type of the current payload to be encoded. + * @param The type of the current payload to be encoded. */ public class FailsafeElementCoder extends CustomCoder> { @@ -97,11 +99,8 @@ public List> getCoderArguments() { @Override public TypeDescriptor> getEncodedTypeDescriptor() { - return new TypeDescriptor>() { - }.where( - new TypeParameter() { - }, originalPayloadCoder.getEncodedTypeDescriptor()) - .where(new TypeParameter() { - }, currentPayloadCoder.getEncodedTypeDescriptor()); + return new TypeDescriptor>() {}.where( + new TypeParameter() {}, originalPayloadCoder.getEncodedTypeDescriptor()) + .where(new TypeParameter() {}, currentPayloadCoder.getEncodedTypeDescriptor()); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java index 893a4ccfc886..5401b8d33232 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java @@ -1,9 +1,11 @@ /* - * Copyright (C) 2019 Google Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * 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 * @@ -52,7 +54,7 @@ /** A Text UDF Transform Function. Note that this class's implementation is not threadsafe */ @AutoValue public abstract class JavascriptTextTransformer { - /** + /** * The {@link FailsafeJavascriptUdf} class processes user-defined functions is a fail-safe manner * by maintaining the original payload post-transformation and outputting to a dead-letter on * failure. @@ -72,9 +74,11 @@ public static Builder newBuilder() { return new AutoValue_JavascriptTextTransformer_FailsafeJavascriptUdf.Builder<>(); } - private Counter successCounter = Metrics.counter(FailsafeJavascriptUdf.class, "udf-transform-success-count"); + private Counter successCounter = + Metrics.counter(FailsafeJavascriptUdf.class, "udf-transform-success-count"); - private Counter failedCounter = Metrics.counter(FailsafeJavascriptUdf.class, "udf-transform-failed-count"); + private Counter failedCounter = + Metrics.counter(FailsafeJavascriptUdf.class, "udf-transform-failed-count"); /** Builder for {@link FailsafeJavascriptUdf}. */ @AutoValue.Builder @@ -95,42 +99,41 @@ public PCollectionTuple expand(PCollection> elements) return elements.apply( "ProcessUdf", ParDo.of( - new DoFn, FailsafeElement>() { - private JavascriptRuntime javascriptRuntime; - - @Setup - public void setup() { - if (fileSystemPath() != null && functionName() != null) { - javascriptRuntime = - getJavascriptRuntime(fileSystemPath(), functionName()); - } - } - - @ProcessElement - public void processElement(ProcessContext context) { - FailsafeElement element = context.element(); - String payloadStr = element.getPayload(); - - try { - if (javascriptRuntime != null) { - payloadStr = javascriptRuntime.invoke(payloadStr); + new DoFn, FailsafeElement>() { + private JavascriptRuntime javascriptRuntime; + + @Setup + public void setup() { + if (fileSystemPath() != null && functionName() != null) { + javascriptRuntime = getJavascriptRuntime(fileSystemPath(), functionName()); + } } - if (!Strings.isNullOrEmpty(payloadStr)) { - context.output( - FailsafeElement.of(element.getOriginalPayload(), payloadStr)); - successCounter.inc(); + @ProcessElement + public void processElement(ProcessContext context) { + FailsafeElement element = context.element(); + String payloadStr = element.getPayload(); + + try { + if (javascriptRuntime != null) { + payloadStr = javascriptRuntime.invoke(payloadStr); + } + + if (!Strings.isNullOrEmpty(payloadStr)) { + context.output( + FailsafeElement.of(element.getOriginalPayload(), payloadStr)); + successCounter.inc(); + } + } catch (Exception e) { + context.output( + failureTag(), + FailsafeElement.of(element) + .setErrorMessage(e.getMessage()) + .setStacktrace(Throwables.getStackTraceAsString(e))); + failedCounter.inc(); + } } - } catch (Exception e) { - context.output( - failureTag(), - FailsafeElement.of(element) - .setErrorMessage(e.getMessage()) - .setStacktrace(Throwables.getStackTraceAsString(e))); - failedCounter.inc(); - } - } - }) + }) .withOutputTags(successTag(), TupleTagList.of(failureTag()))); } } @@ -245,9 +248,7 @@ private static Collection getScripts(String path) throws IOException { "Failed to match any files with the pattern: " + path); List scripts = - result - .metadata() - .stream() + result.metadata().stream() .filter(metadata -> metadata.resourceId().getFilename().endsWith(".js")) .map(Metadata::resourceId) .map( diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java index fe24a8aadcc1..6e8c8862b1c3 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java @@ -142,8 +142,7 @@ public String getJsonBeamSchema() { } /** - * Reads a file from GCS and returns it as a - * string. + * Reads a file from GCS and returns it as a string. * * @param filePath path to file in GCS * @return contents of the file as a string From 76accdc2a7e0f59b6856c0f79e383995e1d4ad1b Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Wed, 13 Jan 2021 10:56:03 +0300 Subject: [PATCH 05/29] build errors --- examples/java/build.gradle | 6 ++++++ .../ProtegrityDataTokenization.java | 2 ++ .../transforms/ProtegrityDataProtectors.java | 1 + .../datatokenization/transforms/io/BigTableIO.java | 13 ++++++++----- .../datatokenization/utils/CsvConverters.java | 1 + .../datatokenization/utils/ErrorConverters.java | 2 ++ .../datatokenization/utils/FailsafeElement.java | 12 ++++++------ .../utils/JavascriptTextTransformer.java | 6 +++++- .../datatokenization/utils/SchemasUtils.java | 2 ++ 9 files changed, 33 insertions(+), 12 deletions(-) diff --git a/examples/java/build.gradle b/examples/java/build.gradle index 86990d9bfa74..55c5ef30b579 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -138,3 +138,9 @@ task preCommit() { } } +task execute (type:JavaExec) { + main = System.getProperty("mainClass") + classpath = sourceSets.main.runtimeClasspath + systemProperties System.getProperties() + args System.getProperty("exec.args", "").split() +} \ No newline at end of file diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java index 9cfd13e34ad5..872c4a5774b7 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/ProtegrityDataTokenization.java @@ -59,6 +59,8 @@ /** * The {@link ProtegrityDataTokenization} pipeline. */ +@SuppressWarnings({"initialization.fields.uninitialized", "method.invocation.invalid", + "dereference.of.nullable", "argument.type.incompatible"}) public class ProtegrityDataTokenization { /** diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java index 05c699ccca3e..8ecc56215348 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/ProtegrityDataProtectors.java @@ -78,6 +78,7 @@ /** * TODO: Add javadoc. */ +@SuppressWarnings("ALL") public class ProtegrityDataProtectors { /** diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java index 99e1acd4241e..7cfe05144928 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java @@ -36,6 +36,9 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.Row; import org.apache.commons.lang3.tuple.Pair; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -102,11 +105,11 @@ public void processElement( .build()) .collect(Collectors.toSet()); // Converting key value to BigTable format - // TODO ramazan@akvelon.com check that please (NPE) - ByteString key = - ByteString.copyFrom( - Objects.requireNonNull(in.getString(options.getBigTableKeyColumnName())).getBytes()); - out.output(KV.of(key, mutations)); + String columnName = in.getString(options.getBigTableKeyColumnName()); + if (columnName != null) { + ByteString key = ByteString.copyFrom(columnName.getBytes()); + out.output(KV.of(key, mutations)); + } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java index b0f2a0595960..07e359d837c6 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java @@ -58,6 +58,7 @@ import org.slf4j.LoggerFactory; /** Common transforms for Csv files. */ +@SuppressWarnings({"argument.type.incompatible", "return.type.incompatible"}) public class CsvConverters { /* Logger for class. */ diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java index cf12f2909ee7..498fdc906146 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java @@ -123,6 +123,8 @@ public void processElement(ProcessContext context) { } } + /** Same as {@link WritePubsubMessageErrors} but for string encoded messages. */ + @AutoValue public abstract static class WriteStringMessageErrors extends PTransform>, WriteResult> { diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java index 69121ff41b22..420bb16193bb 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java @@ -15,10 +15,10 @@ */ package org.apache.beam.examples.complete.datatokenization.utils; +import com.google.common.base.MoreObjects; import java.util.Objects; import org.apache.avro.reflect.Nullable; import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; /** * The {@link FailsafeElement} class holds the current value and original value of a record within a @@ -28,14 +28,13 @@ * can be output to a dead-letter in the event of a failure during one of the pipelines transforms. */ @DefaultCoder(FailsafeElementCoder.class) +@SuppressWarnings("ALL") public class FailsafeElement { private final OriginalT originalPayload; private final CurrentT payload; - @Nullable - private String errorMessage; - @Nullable - private String stacktrace; + @Nullable private String errorMessage; + @Nullable private String stacktrace; private FailsafeElement(OriginalT originalPayload, CurrentT payload) { this.originalPayload = originalPayload; @@ -81,6 +80,7 @@ public FailsafeElement setStacktrace(String stacktrace) { } @Override + @SuppressWarnings("unchecked") public boolean equals(Object obj) { if (this == obj) { return true; @@ -89,7 +89,7 @@ public boolean equals(Object obj) { return false; } - final FailsafeElement other = (FailsafeElement) obj; + final FailsafeElement other = FailsafeElement.class.cast(obj); return Objects.deepEquals(this.originalPayload, other.getOriginalPayload()) && Objects.deepEquals(this.payload, other.getPayload()) && Objects.deepEquals(this.errorMessage, other.getErrorMessage()) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java index 893a4ccfc886..c13c879b180f 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java @@ -50,9 +50,13 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.CharStreams; /** A Text UDF Transform Function. Note that this class's implementation is not threadsafe */ +@SuppressWarnings({"initialization.fields.uninitialized", "method.invocation.invalid", + "dereference.of.nullable", "argument.type.incompatible", "assignment.type.incompatible", + "return.type.incompatible"}) @AutoValue public abstract class JavascriptTextTransformer { - /** + + /** * The {@link FailsafeJavascriptUdf} class processes user-defined functions is a fail-safe manner * by maintaining the original payload post-transformation and outputting to a dead-letter on * failure. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java index fe24a8aadcc1..a4d4a50d26d3 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java @@ -52,6 +52,8 @@ * The {@link SchemasUtils} Class to read JSON based schema. Is there available to read from file or * from string. Currently supported local File System and GCS. */ +@SuppressWarnings({"initialization.fields.uninitialized", "method.invocation.invalid", + "dereference.of.nullable", "argument.type.incompatible", "return.type.incompatible"}) public class SchemasUtils { /* Logger for class.*/ From 8782cba1b0652940b754296ab2e6896e478100ff Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Wed, 13 Jan 2021 17:32:16 +0300 Subject: [PATCH 06/29] minimize suppress --- .../datatokenization/DataTokenization.java | 33 +++++++++++------ .../transforms/DataProtectors.java | 35 ++++++++++--------- .../transforms/io/BigTableIO.java | 12 +++---- .../datatokenization/utils/CsvConverters.java | 13 +++---- .../utils/FailsafeElement.java | 24 +++---------- .../utils/JavascriptTextTransformer.java | 16 ++++++--- .../datatokenization/utils/SchemasUtils.java | 9 +++-- 7 files changed, 78 insertions(+), 64 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java index 2c561b716b27..f8e71b5a7e95 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java @@ -55,28 +55,40 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** The {@link DataTokenization} pipeline. */ -@SuppressWarnings({"initialization.fields.uninitialized", "method.invocation.invalid", - "dereference.of.nullable", "argument.type.incompatible"}) +/** + * The {@link DataTokenization} pipeline. + */ public class DataTokenization { - /** Logger for class. */ + /** + * Logger for class. + */ private static final Logger LOG = LoggerFactory.getLogger(DataTokenization.class); - /** String/String Coder for FailsafeElement. */ + /** + * String/String Coder for FailsafeElement. + */ private static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER = FailsafeElementCoder.of( NullableCoder.of(StringUtf8Coder.of()), NullableCoder.of(StringUtf8Coder.of())); - /** The default suffix for error tables if dead letter table is not specified. */ + /** + * The default suffix for error tables if dead letter table is not specified. + */ private static final String DEFAULT_DEADLETTER_TABLE_SUFFIX = "_error_records"; - /** The tag for the main output for the UDF. */ - private static final TupleTag TOKENIZATION_OUT = new TupleTag() {}; + /** + * The tag for the main output for the UDF. + */ + private static final TupleTag TOKENIZATION_OUT = new TupleTag() { + }; - /** The tag for the dead-letter output of the udf. */ + /** + * The tag for the dead-letter output of the udf. + */ static final TupleTag> TOKENIZATION_DEADLETTER_OUT = - new TupleTag>() {}; + new TupleTag>() { + }; /** * Main entry point for pipeline execution. @@ -99,6 +111,7 @@ public static void main(String[] args) { * @param options The execution options. * @return The pipeline result. */ + @SuppressWarnings({"dereference.of.nullable", "argument.type.incompatible"}) public static PipelineResult run(ProtegrityDataTokenizationOptions options) { SchemasUtils schema = null; try { diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java index d412d9f09336..5ed6868d07ca 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java @@ -26,6 +26,7 @@ import com.google.gson.reflect.TypeToken; import java.io.IOException; import java.lang.reflect.Type; +import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; @@ -71,7 +72,6 @@ import org.slf4j.LoggerFactory; /** TODO: Add javadoc. */ -@SuppressWarnings("ALL") public class DataProtectors { /** Logger for class. */ @@ -87,7 +87,7 @@ public abstract static class RowToTokenizedRow extends PTransform>, PCollectionTuple> { public static Builder newBuilder() { - return new AutoValue_ProtegrityDataProtectors_RowToTokenizedRow.Builder<>(); + return new AutoValue_DataProtectors_RowToTokenizedRow.Builder<>(); } public abstract TupleTag successTag(); @@ -107,9 +107,7 @@ public PCollectionTuple expand(PCollection> inputRows) { PCollectionTuple pCollectionTuple = inputRows.apply( "Tokenize", - ParDo.of( - new TokenizationFn( - schema(), batchSize(), rpcURI(), failureTag())) + ParDo.of(new TokenizationFn(schema(), batchSize(), rpcURI(), failureTag())) .withOutputTags(successTag(), TupleTagList.of(failureTag()))); return PCollectionTuple.of( successTag(), pCollectionTuple.get(successTag()).setRowSchema(schema())) @@ -135,10 +133,9 @@ public abstract static class Builder { } /** Class for data tokenization. */ + @SuppressWarnings("initialization.static.fields.uninitialized") public static class TokenizationFn extends DoFn, Row> { - public static final String ID_TOKEN_NAME = "ID"; - private static Schema schemaToTokenize; private static CloseableHttpClient httpclient; private static ObjectMapper objectMapperSerializerForTokenize; @@ -159,7 +156,7 @@ public static class TokenizationFn extends DoFn, Row> { private final TimerSpec expirySpec = TimerSpecs.timer(TimeDomain.EVENT_TIME); private boolean hasIdInInputs = true; - private String idFieldName; + private String idFieldName = ""; private Map inputRowsWithIds; public TokenizationFn( @@ -172,6 +169,7 @@ public TokenizationFn( this.rpcURI = rpcURI; bufferedEvents = StateSpecs.bag(RowCoder.of(schema)); this.failureTag = failureTag; + this.inputRowsWithIds = new HashMap<>(); } @Setup @@ -192,7 +190,10 @@ public void close() { try { httpclient.close(); } catch (IOException exception) { - LOG.warn("Can't close connection: {}", exception.getMessage()); + String exceptionMessage = exception.getMessage(); + if (exceptionMessage != null) { + LOG.warn("Can't close connection: {}", exceptionMessage); + } } } @@ -227,6 +228,7 @@ public void process( } } + @SuppressWarnings("argument.type.incompatible") private void processBufferedRows(Iterable rows, WindowedContext context) { try { @@ -262,7 +264,9 @@ private ArrayList rowsToJsons(Iterable inputRows) { } else { id = inputRow.getValue(idFieldName); } - inputRowsWithIds.put(id, inputRow); + if (id != null) { + inputRowsWithIds.put(id, inputRow); + } Row row = builder.build(); @@ -275,19 +279,18 @@ private ArrayList rowsToJsons(Iterable inputRows) { private String formatJsonsToRpcBatch(Iterable jsons) { StringBuilder stringBuilder = new StringBuilder(String.join(",", jsons)); Gson gson = new Gson(); - Type gsonType = new TypeToken>() {}.getType(); - stringBuilder - .append("]") - .insert(0, "{\"data\": [") - .append("}"); + Type gsonType = new TypeToken>() { + }.getType(); + stringBuilder.append("]").insert(0, "{\"data\": [").append("}"); return stringBuilder.toString(); } + @SuppressWarnings("argument.type.incompatible") private ArrayList getTokenizedRow(Iterable inputRows) throws IOException { ArrayList outputRows = new ArrayList<>(); CloseableHttpResponse response = - sendRpc(formatJsonsToRpcBatch(rowsToJsons(inputRows)).getBytes()); + sendRpc(formatJsonsToRpcBatch(rowsToJsons(inputRows)).getBytes(Charset.defaultCharset())); String tokenizedData = IOUtils.toString(response.getEntity().getContent(), StandardCharsets.UTF_8); diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java index e0392d9eb681..98fdc727391d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java @@ -19,6 +19,7 @@ import com.google.bigtable.v2.Mutation; import com.google.protobuf.ByteString; +import java.nio.charset.Charset; import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -36,9 +37,6 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.Row; import org.apache.commons.lang3.tuple.Pair; -import org.checkerframework.checker.initialization.qual.Initialized; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.checkerframework.checker.nullness.qual.UnknownKeyFor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,8 +92,10 @@ public void processElement( .setSetCell( Mutation.SetCell.newBuilder() .setFamilyName(options.getBigTableColumnFamilyName()) - .setColumnQualifier(ByteString.copyFrom(pair.getKey().getBytes())) - .setValue(ByteString.copyFrom(pair.getValue().getBytes())) + .setColumnQualifier( + ByteString.copyFrom(pair.getKey(), Charset.defaultCharset())) + .setValue(ByteString + .copyFrom(pair.getValue(), Charset.defaultCharset())) .setTimestampMicros(System.currentTimeMillis() * 1000) .build()) .build()) @@ -103,7 +103,7 @@ public void processElement( // Converting key value to BigTable format String columnName = in.getString(options.getBigTableKeyColumnName()); if (columnName != null) { - ByteString key = ByteString.copyFrom(columnName.getBytes()); + ByteString key = ByteString.copyFrom(columnName, Charset.defaultCharset()); out.output(KV.of(key, mutations)); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java index 2379a9232e8a..9aa10b3c7804 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java @@ -60,7 +60,7 @@ import org.slf4j.LoggerFactory; /** Common transforms for Csv files. */ -@SuppressWarnings({"argument.type.incompatible", "return.type.incompatible"}) +@SuppressWarnings({"argument.type.incompatible"}) public class CsvConverters { /* Logger for class. */ @@ -293,11 +293,12 @@ public PCollectionTuple expand(PCollectionTuple lines) { } // If no udf then use json schema - if (jsonSchemaPath() != null || jsonSchema() != null) { + String schemaPath = jsonSchemaPath(); + if (schemaPath != null || jsonSchema() != null) { String schema; - if (jsonSchemaPath() != null) { - schema = getGcsFileAsString(jsonSchemaPath()); + if (schemaPath != null) { + schema = getGcsFileAsString(schemaPath); } else { schema = jsonSchema(); } @@ -305,8 +306,8 @@ public PCollectionTuple expand(PCollectionTuple lines) { return lineFailsafeElements.apply( "LineToDocumentUsingSchema", ParDo.of( - new FailsafeElementToJsonFn( - headersView, schema, delimiter(), udfDeadletterTag())) + new FailsafeElementToJsonFn( + headersView, schema, delimiter(), udfDeadletterTag())) .withOutputTags(udfOutputTag(), TupleTagList.of(udfDeadletterTag()))); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java index 14c23764fd88..d2f732438bc9 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java @@ -30,13 +30,14 @@ * can be output to a dead-letter in the event of a failure during one of the pipelines transforms. */ @DefaultCoder(FailsafeElementCoder.class) -@SuppressWarnings("ALL") public class FailsafeElement { private final OriginalT originalPayload; private final CurrentT payload; - @Nullable private String errorMessage; - @Nullable private String stacktrace; + @Nullable + private String errorMessage = ""; + @Nullable + private String stacktrace = ""; private FailsafeElement(OriginalT originalPayload, CurrentT payload) { this.originalPayload = originalPayload; @@ -81,23 +82,6 @@ public FailsafeElement setStacktrace(String stacktrace) { return this; } - @Override - @SuppressWarnings("unchecked") - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - - final FailsafeElement other = FailsafeElement.class.cast(obj); - return Objects.deepEquals(this.originalPayload, other.getOriginalPayload()) - && Objects.deepEquals(this.payload, other.getPayload()) - && Objects.deepEquals(this.errorMessage, other.getErrorMessage()) - && Objects.deepEquals(this.stacktrace, other.getStacktrace()); - } - @Override public int hashCode() { return Objects.hash(originalPayload, payload, errorMessage, stacktrace); diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java index cdc4567734f1..b59aef537f90 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java @@ -51,10 +51,17 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.CharStreams; -/** A Text UDF Transform Function. Note that this class's implementation is not threadsafe */ -@SuppressWarnings({"initialization.fields.uninitialized", "method.invocation.invalid", - "dereference.of.nullable", "argument.type.incompatible", "assignment.type.incompatible", - "return.type.incompatible"}) +/** + * A Text UDF Transform Function. Note that this class's implementation is not threadsafe + */ +@SuppressWarnings({ + "initialization.fields.uninitialized", + "method.invocation.invalid", + "dereference.of.nullable", + "argument.type.incompatible", + "assignment.type.incompatible", + "return.type.incompatible" +}) @AutoValue public abstract class JavascriptTextTransformer { @@ -280,6 +287,7 @@ private static Collection getScripts(String path) throws IOException { * @param functionName The function name which will be invoked within the JavaScript script. * @return The {@link JavascriptRuntime} instance. */ + @Nullable private static JavascriptRuntime getJavascriptRuntime( String fileSystemPath, String functionName) { JavascriptRuntime javascriptRuntime = null; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java index 877c5a8ca843..23b910145238 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java @@ -52,8 +52,13 @@ * The {@link SchemasUtils} Class to read JSON based schema. Is there available to read from file or * from string. Currently supported local File System and GCS. */ -@SuppressWarnings({"initialization.fields.uninitialized", "method.invocation.invalid", - "dereference.of.nullable", "argument.type.incompatible", "return.type.incompatible"}) +@SuppressWarnings({ + "initialization.fields.uninitialized", + "method.invocation.invalid", + "dereference.of.nullable", + "argument.type.incompatible", + "return.type.incompatible" +}) public class SchemasUtils { /* Logger for class.*/ From 46805f3bb339c9de1f7bd91a91df734625abcea9 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 14 Jan 2021 16:32:35 +0300 Subject: [PATCH 07/29] remove UDF usages --- .../datatokenization/utils/CsvConverters.java | 37 +-- .../utils/JavascriptTextTransformer.java | 305 ------------------ 2 files changed, 2 insertions(+), 340 deletions(-) delete mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java index 9aa10b3c7804..0d59b829d449 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java @@ -32,7 +32,6 @@ import java.util.List; import java.util.stream.Collectors; import javax.annotation.Nullable; -import org.apache.beam.examples.complete.datatokenization.utils.JavascriptTextTransformer.FailsafeJavascriptUdf; import org.apache.beam.sdk.io.FileIO; import org.apache.beam.sdk.io.FileIO.ReadableFile; import org.apache.beam.sdk.io.TextIO; @@ -42,17 +41,8 @@ import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.Sample; -import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.transforms.*; +import org.apache.beam.sdk.values.*; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables; import org.apache.commons.csv.CSVFormat; import org.apache.commons.csv.CSVParser; @@ -248,12 +238,6 @@ public static Builder newBuilder() { public abstract String delimiter(); - @Nullable - public abstract String udfFileSystemPath(); - - @Nullable - public abstract String udfFunctionName(); - @Nullable public abstract String jsonSchemaPath(); @@ -279,19 +263,6 @@ public PCollectionTuple expand(PCollectionTuple lines) { .get(lineTag()) .apply("LineToFailsafeElement", ParDo.of(new LineToFailsafeElementFn())); - // If UDF is specified then use that to parse csv lines. - if (udfFileSystemPath() != null) { - - return lineFailsafeElements.apply( - "LineToDocumentUsingUdf", - FailsafeJavascriptUdf.newBuilder() - .setFileSystemPath(udfFileSystemPath()) - .setFunctionName(udfFunctionName()) - .setSuccessTag(udfOutputTag()) - .setFailureTag(udfDeadletterTag()) - .build()); - } - // If no udf then use json schema String schemaPath = jsonSchemaPath(); if (schemaPath != null || jsonSchema() != null) { @@ -347,10 +318,6 @@ headersView, jsonSchemaPath(), delimiter(), udfDeadletterTag())) public abstract static class Builder { public abstract Builder setDelimiter(String delimiter); - public abstract Builder setUdfFileSystemPath(String udfFileSystemPath); - - public abstract Builder setUdfFunctionName(String udfFunctionName); - public abstract Builder setJsonSchemaPath(String jsonSchemaPath); public abstract Builder setJsonSchema(String jsonSchema); diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java deleted file mode 100644 index b59aef537f90..000000000000 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/JavascriptTextTransformer.java +++ /dev/null @@ -1,305 +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.examples.complete.datatokenization.utils; - -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; - -import com.google.auto.value.AutoValue; -import java.io.IOException; -import java.io.Reader; -import java.io.UncheckedIOException; -import java.nio.channels.Channels; -import java.nio.charset.StandardCharsets; -import java.util.Collection; -import java.util.List; -import java.util.stream.Collectors; -import javax.annotation.Nullable; -import javax.script.Invocable; -import javax.script.ScriptEngine; -import javax.script.ScriptEngineManager; -import javax.script.ScriptException; -import jdk.nashorn.api.scripting.ScriptObjectMirror; -import org.apache.beam.sdk.io.FileSystems; -import org.apache.beam.sdk.io.fs.MatchResult; -import org.apache.beam.sdk.io.fs.MatchResult.Metadata; -import org.apache.beam.sdk.io.fs.MatchResult.Status; -import org.apache.beam.sdk.metrics.Counter; -import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.CharStreams; - -/** - * A Text UDF Transform Function. Note that this class's implementation is not threadsafe - */ -@SuppressWarnings({ - "initialization.fields.uninitialized", - "method.invocation.invalid", - "dereference.of.nullable", - "argument.type.incompatible", - "assignment.type.incompatible", - "return.type.incompatible" -}) -@AutoValue -public abstract class JavascriptTextTransformer { - - /** - * The {@link FailsafeJavascriptUdf} class processes user-defined functions is a fail-safe manner - * by maintaining the original payload post-transformation and outputting to a dead-letter on - * failure. - */ - @AutoValue - public abstract static class FailsafeJavascriptUdf - extends PTransform>, PCollectionTuple> { - public abstract @Nullable String fileSystemPath(); - - public abstract @Nullable String functionName(); - - public abstract TupleTag> successTag(); - - public abstract TupleTag> failureTag(); - - public static Builder newBuilder() { - return new AutoValue_JavascriptTextTransformer_FailsafeJavascriptUdf.Builder<>(); - } - - private Counter successCounter = - Metrics.counter(FailsafeJavascriptUdf.class, "udf-transform-success-count"); - - private Counter failedCounter = - Metrics.counter(FailsafeJavascriptUdf.class, "udf-transform-failed-count"); - - /** Builder for {@link FailsafeJavascriptUdf}. */ - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setFileSystemPath(@Nullable String fileSystemPath); - - public abstract Builder setFunctionName(@Nullable String functionName); - - public abstract Builder setSuccessTag(TupleTag> successTag); - - public abstract Builder setFailureTag(TupleTag> failureTag); - - public abstract FailsafeJavascriptUdf build(); - } - - @Override - public PCollectionTuple expand(PCollection> elements) { - return elements.apply( - "ProcessUdf", - ParDo.of( - new DoFn, FailsafeElement>() { - private JavascriptRuntime javascriptRuntime; - - @Setup - public void setup() { - if (fileSystemPath() != null && functionName() != null) { - javascriptRuntime = getJavascriptRuntime(fileSystemPath(), functionName()); - } - } - - @ProcessElement - public void processElement(ProcessContext context) { - FailsafeElement element = context.element(); - String payloadStr = element.getPayload(); - - try { - if (javascriptRuntime != null) { - payloadStr = javascriptRuntime.invoke(payloadStr); - } - - if (!Strings.isNullOrEmpty(payloadStr)) { - context.output( - FailsafeElement.of(element.getOriginalPayload(), payloadStr)); - successCounter.inc(); - } - } catch (Exception e) { - context.output( - failureTag(), - FailsafeElement.of(element) - .setErrorMessage(e.getMessage()) - .setStacktrace(Throwables.getStackTraceAsString(e))); - failedCounter.inc(); - } - } - }) - .withOutputTags(successTag(), TupleTagList.of(failureTag()))); - } - } - - /** - * Grabs code from a FileSystem, loads it into the Nashorn Javascript Engine, and executes - * Javascript Functions. - */ - @AutoValue - public abstract static class JavascriptRuntime { - @Nullable - public abstract String fileSystemPath(); - - @Nullable - public abstract String functionName(); - - private Invocable invocable; - - /** Builder for {@link JavascriptTextTransformer}. */ - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setFileSystemPath(@Nullable String fileSystemPath); - - public abstract Builder setFunctionName(@Nullable String functionName); - - public abstract JavascriptRuntime build(); - } - - /** - * Factory method for generating a JavascriptTextTransformer.Builder. - * - * @return a JavascriptTextTransformer builder - */ - public static Builder newBuilder() { - return new AutoValue_JavascriptTextTransformer_JavascriptRuntime.Builder(); - } - - /** - * Gets a cached Javascript Invocable, if fileSystemPath() not set, returns null. - * - * @return a Javascript Invocable or null - */ - @Nullable - public Invocable getInvocable() throws ScriptException, IOException { - - // return null if no UDF path specified. - if (Strings.isNullOrEmpty(fileSystemPath())) { - return null; - } - - if (invocable == null) { - Collection scripts = getScripts(fileSystemPath()); - invocable = newInvocable(scripts); - } - return invocable; - } - - /** - * Factory method for making a new Invocable. - * - * @param scripts a collection of javascript scripts encoded with UTF8 to load in - */ - @Nullable - private static Invocable newInvocable(Collection scripts) throws ScriptException { - ScriptEngineManager manager = new ScriptEngineManager(); - ScriptEngine engine = manager.getEngineByName("JavaScript"); - - for (String script : scripts) { - engine.eval(script); - } - - return (Invocable) engine; - } - - /** - * Invokes the UDF with specified data. - * - * @param data data to pass to the invocable function - * @return The data transformed by the UDF in String format - */ - @Nullable - public String invoke(String data) throws ScriptException, IOException, NoSuchMethodException { - Invocable invocable = getInvocable(); - if (invocable == null) { - throw new RuntimeException("No udf was loaded"); - } - - Object result = getInvocable().invokeFunction(functionName(), data); - if (result == null || ScriptObjectMirror.isUndefined(result)) { - return null; - - } else if (result instanceof String) { - return (String) result; - - } else { - String className = result.getClass().getName(); - throw new RuntimeException( - "UDF Function did not return a String. Instead got: " + className); - } - } - - /** - * Loads into memory scripts from a File System from a given path. Supports any file system that - * {@link FileSystems} supports. - * - * @return a collection of scripts loaded as UF8 Strings - */ - private static Collection getScripts(String path) throws IOException { - MatchResult result = FileSystems.match(path); - checkArgument( - result.status() == Status.OK && !result.metadata().isEmpty(), - "Failed to match any files with the pattern: " + path); - - List scripts = - result.metadata().stream() - .filter(metadata -> metadata.resourceId().getFilename().endsWith(".js")) - .map(Metadata::resourceId) - .map( - resourceId -> { - try (Reader reader = - Channels.newReader( - FileSystems.open(resourceId), StandardCharsets.UTF_8.name())) { - return CharStreams.toString(reader); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - }) - .collect(Collectors.toList()); - - return scripts; - } - } - - /** - * Retrieves a {@link JavascriptRuntime} configured to invoke the specified function within the - * script. If either the fileSystemPath or functionName is null or empty, this method will return - * null indicating that a runtime was unable to be created within the given parameters. - * - * @param fileSystemPath The file path to the JavaScript file to execute. - * @param functionName The function name which will be invoked within the JavaScript script. - * @return The {@link JavascriptRuntime} instance. - */ - @Nullable - private static JavascriptRuntime getJavascriptRuntime( - String fileSystemPath, String functionName) { - JavascriptRuntime javascriptRuntime = null; - - if (!Strings.isNullOrEmpty(fileSystemPath) && !Strings.isNullOrEmpty(functionName)) { - javascriptRuntime = - JavascriptRuntime.newBuilder() - .setFunctionName(functionName) - .setFileSystemPath(fileSystemPath) - .build(); - } - - return javascriptRuntime; - } -} From d9a8e3379f71106950c44d24234bf6c2973d5369 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Thu, 14 Jan 2021 16:50:06 +0300 Subject: [PATCH 08/29] Fixes for stylechecks --- examples/java/build.gradle | 2 +- .../datatokenization/DataTokenization.java | 30 +++++-------------- .../complete/datatokenization/README.md | 6 +--- .../transforms/DataProtectors.java | 5 ++-- .../transforms/io/BigTableIO.java | 10 +++---- .../datatokenization/utils/CsvConverters.java | 17 ++++++++--- .../utils/FailsafeElement.java | 8 ++--- .../datatokenization/utils/SchemasUtils.java | 4 +-- 8 files changed, 35 insertions(+), 47 deletions(-) diff --git a/examples/java/build.gradle b/examples/java/build.gradle index 1bb727e14a3f..71422133bf63 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -143,4 +143,4 @@ task execute (type:JavaExec) { classpath = sourceSets.main.runtimeClasspath systemProperties System.getProperties() args System.getProperty("exec.args", "").split() -} \ No newline at end of file +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java index f8e71b5a7e95..859567cc1dcb 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java @@ -55,40 +55,26 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * The {@link DataTokenization} pipeline. - */ +/** The {@link DataTokenization} pipeline. */ public class DataTokenization { - /** - * Logger for class. - */ + /** Logger for class. */ private static final Logger LOG = LoggerFactory.getLogger(DataTokenization.class); - /** - * String/String Coder for FailsafeElement. - */ + /** String/String Coder for FailsafeElement. */ private static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER = FailsafeElementCoder.of( NullableCoder.of(StringUtf8Coder.of()), NullableCoder.of(StringUtf8Coder.of())); - /** - * The default suffix for error tables if dead letter table is not specified. - */ + /** The default suffix for error tables if dead letter table is not specified. */ private static final String DEFAULT_DEADLETTER_TABLE_SUFFIX = "_error_records"; - /** - * The tag for the main output for the UDF. - */ - private static final TupleTag TOKENIZATION_OUT = new TupleTag() { - }; + /** The tag for the main output for the UDF. */ + private static final TupleTag TOKENIZATION_OUT = new TupleTag() {}; - /** - * The tag for the dead-letter output of the udf. - */ + /** The tag for the dead-letter output of the udf. */ static final TupleTag> TOKENIZATION_DEADLETTER_OUT = - new TupleTag>() { - }; + new TupleTag>() {}; /** * Main entry point for pipeline execution. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md index ee6b0233554c..8a664632aff6 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md @@ -149,7 +149,7 @@ in the following format: --dataSchemaGcsPath="path-to-data-schema-in-json-format" --inputGcsFilePattern="path-patternt-to-input-data" --outputGcsDirectory="path-to-output-directory" -#example to usage csv +# example to usage csv --inputGcsFileFormat="CSV" --outputGcsFileFormat="CSV" --csvContainsHeaders="true" @@ -167,10 +167,6 @@ By default, this will run the pipeline locally with the DirectRunner. To change See the [documentation](http://beam.apache.org/get-started/quickstart/) and the [Examples README](../../../../../../../../../README.md) for more information about how to run this example. - -## Running as a Dataflow Template - - ## Running as a Dataflow Template This example also exists as Google Dataflow Template, which you can build and run using Google Cloud Platform. See diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java index 5ed6868d07ca..5a633f683e5d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java @@ -18,7 +18,7 @@ package org.apache.beam.examples.complete.datatokenization.transforms; import static org.apache.beam.sdk.util.RowJsonUtils.rowToJson; -import static org.apache.beam.vendor.grpc.v1p26p0.com.google.common.base.MoreObjects.firstNonNull; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects.firstNonNull; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.services.bigquery.model.TableRow; @@ -279,8 +279,7 @@ private ArrayList rowsToJsons(Iterable inputRows) { private String formatJsonsToRpcBatch(Iterable jsons) { StringBuilder stringBuilder = new StringBuilder(String.join(",", jsons)); Gson gson = new Gson(); - Type gsonType = new TypeToken>() { - }.getType(); + Type gsonType = new TypeToken>() {}.getType(); stringBuilder.append("]").insert(0, "{\"data\": [").append("}"); return stringBuilder.toString(); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java index 98fdc727391d..c499936c37dd 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java @@ -19,7 +19,7 @@ import com.google.bigtable.v2.Mutation; import com.google.protobuf.ByteString; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -93,9 +93,9 @@ public void processElement( Mutation.SetCell.newBuilder() .setFamilyName(options.getBigTableColumnFamilyName()) .setColumnQualifier( - ByteString.copyFrom(pair.getKey(), Charset.defaultCharset())) - .setValue(ByteString - .copyFrom(pair.getValue(), Charset.defaultCharset())) + ByteString.copyFrom(pair.getKey(), StandardCharsets.UTF_8)) + .setValue( + ByteString.copyFrom(pair.getValue(), StandardCharsets.UTF_8)) .setTimestampMicros(System.currentTimeMillis() * 1000) .build()) .build()) @@ -103,7 +103,7 @@ public void processElement( // Converting key value to BigTable format String columnName = in.getString(options.getBigTableKeyColumnName()); if (columnName != null) { - ByteString key = ByteString.copyFrom(columnName, Charset.defaultCharset()); + ByteString key = ByteString.copyFrom(columnName, StandardCharsets.UTF_8); out.output(KV.of(key, mutations)); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java index 0d59b829d449..090abcb80eb5 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java @@ -41,8 +41,17 @@ import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.*; -import org.apache.beam.sdk.values.*; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Sample; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables; import org.apache.commons.csv.CSVFormat; import org.apache.commons.csv.CSVParser; @@ -277,8 +286,8 @@ public PCollectionTuple expand(PCollectionTuple lines) { return lineFailsafeElements.apply( "LineToDocumentUsingSchema", ParDo.of( - new FailsafeElementToJsonFn( - headersView, schema, delimiter(), udfDeadletterTag())) + new FailsafeElementToJsonFn( + headersView, schema, delimiter(), udfDeadletterTag())) .withOutputTags(udfOutputTag(), TupleTagList.of(udfDeadletterTag()))); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java index d2f732438bc9..66f1c3c176a8 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/FailsafeElement.java @@ -17,10 +17,10 @@ */ package org.apache.beam.examples.complete.datatokenization.utils; -import com.google.common.base.MoreObjects; import java.util.Objects; import org.apache.avro.reflect.Nullable; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; /** * The {@link FailsafeElement} class holds the current value and original value of a record within a @@ -34,10 +34,8 @@ public class FailsafeElement { private final OriginalT originalPayload; private final CurrentT payload; - @Nullable - private String errorMessage = ""; - @Nullable - private String stacktrace = ""; + @Nullable private String errorMessage = ""; + @Nullable private String stacktrace = ""; private FailsafeElement(OriginalT originalPayload, CurrentT payload) { this.originalPayload = originalPayload; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java index 23b910145238..86338cb8e3d2 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java @@ -41,9 +41,9 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; -import org.apache.beam.vendor.grpc.v1p26p0.com.google.common.io.ByteStreams; import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.Gson; import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.reflect.TypeToken; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.CharStreams; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -125,7 +125,7 @@ public Map getDataElementsToTokenize(String payloadConfigGcsPath Map dataElements; try { String rawJsonWithDataElements = - new String(readGcsFile(payloadConfigGcsPath), Charset.defaultCharset()); + new String(readGcsFile(payloadConfigGcsPath), StandardCharsets.UTF_8); Gson gson = new Gson(); Type type = new TypeToken>() {}.getType(); dataElements = gson.fromJson(rawJsonWithDataElements, type); From 76af970986455cac01f87cf66ff5999ea0cbaab1 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 14 Jan 2021 16:53:25 +0300 Subject: [PATCH 09/29] grooming for data protectors --- .../transforms/DataProtectors.java | 45 ++++++------------- 1 file changed, 13 insertions(+), 32 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java index 5ed6868d07ca..6508e90b0d3f 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java @@ -23,9 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.services.bigquery.model.TableRow; import com.google.auto.value.AutoValue; -import com.google.gson.reflect.TypeToken; import java.io.IOException; -import java.lang.reflect.Type; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -36,26 +34,14 @@ import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.state.BagState; -import org.apache.beam.sdk.state.StateSpec; -import org.apache.beam.sdk.state.StateSpecs; -import org.apache.beam.sdk.state.TimeDomain; -import org.apache.beam.sdk.state.Timer; -import org.apache.beam.sdk.state.TimerSpec; -import org.apache.beam.sdk.state.TimerSpecs; -import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.*; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.RowJson; import org.apache.beam.sdk.util.RowJsonUtils; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.*; import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.Gson; import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.JsonArray; import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.JsonObject; @@ -136,10 +122,9 @@ public abstract static class Builder { @SuppressWarnings("initialization.static.fields.uninitialized") public static class TokenizationFn extends DoFn, Row> { - private static Schema schemaToTokenize; private static CloseableHttpClient httpclient; - private static ObjectMapper objectMapperSerializerForTokenize; - private static ObjectMapper objectMapperDeserializerForTokenize; + private static ObjectMapper objectMapperSerializerForSchema; + private static ObjectMapper objectMapperDeserializerForSchema; private final Schema schema; private final int batchSize; @@ -175,12 +160,11 @@ public TokenizationFn( @Setup public void setup() { - schemaToTokenize = new Schema(schema.getFields()); - objectMapperSerializerForTokenize = - RowJsonUtils.newObjectMapperWith(RowJson.RowJsonSerializer.forSchema(schemaToTokenize)); + objectMapperSerializerForSchema = + RowJsonUtils.newObjectMapperWith(RowJson.RowJsonSerializer.forSchema(schema)); - objectMapperDeserializerForTokenize = - RowJsonUtils.newObjectMapperWith(RowJson.RowJsonDeserializer.forSchema(schemaToTokenize)); + objectMapperDeserializerForSchema = + RowJsonUtils.newObjectMapperWith(RowJson.RowJsonDeserializer.forSchema(schema)); httpclient = HttpClients.createDefault(); } @@ -251,8 +235,8 @@ private ArrayList rowsToJsons(Iterable inputRows) { Map inputRowsWithIds = new HashMap<>(); for (Row inputRow : inputRows) { - Row.Builder builder = Row.withSchema(schemaToTokenize); - for (Schema.Field field : schemaToTokenize.getFields()) { + Row.Builder builder = Row.withSchema(schema); + for (Schema.Field field : schema.getFields()) { if (inputRow.getSchema().hasField(field.getName())) { builder = builder.addValue(inputRow.getValue(field.getName())); } @@ -270,7 +254,7 @@ private ArrayList rowsToJsons(Iterable inputRows) { Row row = builder.build(); - jsons.add(rowToJson(objectMapperSerializerForTokenize, row)); + jsons.add(rowToJson(objectMapperSerializerForSchema, row)); } this.inputRowsWithIds = inputRowsWithIds; return jsons; @@ -278,9 +262,6 @@ private ArrayList rowsToJsons(Iterable inputRows) { private String formatJsonsToRpcBatch(Iterable jsons) { StringBuilder stringBuilder = new StringBuilder(String.join(",", jsons)); - Gson gson = new Gson(); - Type gsonType = new TypeToken>() { - }.getType(); stringBuilder.append("]").insert(0, "{\"data\": [").append("}"); return stringBuilder.toString(); } @@ -302,10 +283,10 @@ private ArrayList getTokenizedRow(Iterable inputRows) throws IOExcepti for (int i = 0; i < jsonTokenizedRows.size(); i++) { Row tokenizedRow = RowJsonUtils.jsonToRow( - objectMapperDeserializerForTokenize, jsonTokenizedRows.get(i).toString()); + objectMapperDeserializerForSchema, jsonTokenizedRows.get(i).toString()); Row.FieldValueBuilder rowBuilder = Row.fromRow(this.inputRowsWithIds.get(tokenizedRow.getString(idFieldName))); - for (Schema.Field field : schemaToTokenize.getFields()) { + for (Schema.Field field : schema.getFields()) { if (!hasIdInInputs && field.getName().equals(idFieldName)) { continue; } From bfcba7f0c252283dff3d8e9c1915b153ff995e89 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 14 Jan 2021 17:05:33 +0300 Subject: [PATCH 10/29] grooming for data protectors --- .../transforms/DataProtectors.java | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java index 13b1b34f5b50..32d85a66f6f9 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java @@ -34,14 +34,26 @@ import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.state.*; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.RowJson; import org.apache.beam.sdk.util.RowJsonUtils; -import org.apache.beam.sdk.values.*; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.Gson; import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.JsonArray; import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.JsonObject; @@ -283,7 +295,7 @@ private ArrayList getTokenizedRow(Iterable inputRows) throws IOExcepti for (int i = 0; i < jsonTokenizedRows.size(); i++) { Row tokenizedRow = RowJsonUtils.jsonToRow( - objectMapperDeserializerForSchema, jsonTokenizedRows.get(i).toString()); + objectMapperDeserializerForSchema, jsonTokenizedRows.get(i).toString()); Row.FieldValueBuilder rowBuilder = Row.fromRow(this.inputRowsWithIds.get(tokenizedRow.getString(idFieldName))); for (Schema.Field field : schema.getFields()) { From cfa41a77c68f037a30186c53e929e7ef87b7697d Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 14 Jan 2021 17:12:17 +0300 Subject: [PATCH 11/29] fix javadoc --- .../complete/datatokenization/utils/ErrorConverters.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java index 42a4b7086a07..77122bf5d291 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java @@ -116,7 +116,7 @@ public void processElement(ProcessContext context) { } } - /** Same as {@link WritePubsubMessageErrors} but for string encoded messages. */ + /** Write errors as string encoded messages. */ @AutoValue public abstract static class WriteStringMessageErrors extends PTransform>, WriteResult> { From c8200f4d01af2349e97c3d2e849bc888c84cc8d5 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Fri, 15 Jan 2021 16:36:05 +0300 Subject: [PATCH 12/29] Added support for window writing; Fixed ordering in tokenization process --- .../datatokenization/DataTokenization.java | 19 +++-- ...ions.java => DataTokenizationOptions.java} | 23 ++---- .../transforms/DataProtectors.java | 39 +++++----- .../transforms/io/BigTableIO.java | 9 +-- .../datatokenization/transforms/io/GcsIO.java | 56 +++++++++++--- .../datatokenization/utils/DurationUtils.java | 75 +++++++++++++++++++ .../utils/ErrorConverters.java | 29 ++++++- 7 files changed, 190 insertions(+), 60 deletions(-) rename examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/{ProtegrityDataTokenizationOptions.java => DataTokenizationOptions.java} (77%) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/DurationUtils.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java index 859567cc1dcb..f09bf30c66f7 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java @@ -17,12 +17,13 @@ */ package org.apache.beam.examples.complete.datatokenization; +import static org.apache.beam.examples.complete.datatokenization.utils.DurationUtils.parseDuration; import static org.apache.beam.examples.complete.datatokenization.utils.SchemasUtils.DEADLETTER_SCHEMA; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import java.io.IOException; import java.nio.charset.StandardCharsets; -import org.apache.beam.examples.complete.datatokenization.options.ProtegrityDataTokenizationOptions; +import org.apache.beam.examples.complete.datatokenization.options.DataTokenizationOptions; import org.apache.beam.examples.complete.datatokenization.transforms.DataProtectors.RowToTokenizedRow; import org.apache.beam.examples.complete.datatokenization.transforms.io.BigQueryIO; import org.apache.beam.examples.complete.datatokenization.transforms.io.BigTableIO; @@ -46,6 +47,8 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.JsonToRow; import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; @@ -82,10 +85,8 @@ public class DataTokenization { * @param args Command line arguments to the pipeline. */ public static void main(String[] args) { - ProtegrityDataTokenizationOptions options = - PipelineOptionsFactory.fromArgs(args) - .withValidation() - .as(ProtegrityDataTokenizationOptions.class); + DataTokenizationOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(DataTokenizationOptions.class); FileSystems.setDefaultPipelineOptions(options); run(options); @@ -98,7 +99,7 @@ public static void main(String[] args) { * @return The pipeline result. */ @SuppressWarnings({"dereference.of.nullable", "argument.type.incompatible"}) - public static PipelineResult run(ProtegrityDataTokenizationOptions options) { + public static PipelineResult run(DataTokenizationOptions options) { SchemasUtils schema = null; try { schema = new SchemasUtils(options.getDataSchemaGcsPath(), StandardCharsets.UTF_8); @@ -133,6 +134,10 @@ public static PipelineResult run(ProtegrityDataTokenizationOptions options) { jsons = pipeline.apply( "ReadMessagesFromPubsub", PubsubIO.readStrings().fromTopic(options.getPubsubTopic())); + if (options.getOutputGcsDirectory() != null) { + jsons = + jsons.apply(Window.into(FixedWindows.of(parseDuration(options.getWindowDuration())))); + } } else { throw new IllegalStateException("No source is provided, please configure GCS or Pub/Sub"); } @@ -176,7 +181,7 @@ public static PipelineResult run(ProtegrityDataTokenizationOptions options) { "DsgTokenization", RowToTokenizedRow.newBuilder() .setBatchSize(options.getBatchSize()) - .setRpcURI(options.getDsgUri()) + .setRpcURI(options.getRpcUri()) .setSchema(schema.getBeamSchema()) .setSuccessTag(TOKENIZATION_OUT) .setFailureTag(TOKENIZATION_DEADLETTER_OUT) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/ProtegrityDataTokenizationOptions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/DataTokenizationOptions.java similarity index 77% rename from examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/ProtegrityDataTokenizationOptions.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/DataTokenizationOptions.java index 1b4a258b7a76..6e383a1df822 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/ProtegrityDataTokenizationOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/DataTokenizationOptions.java @@ -24,10 +24,10 @@ import org.apache.beam.sdk.options.PipelineOptions; /** - * The {@link ProtegrityDataTokenizationOptions} interface provides the custom execution options - * passed by the executor at the command-line. + * The {@link DataTokenizationOptions} interface provides the custom execution options passed by the + * executor at the command-line. */ -public interface ProtegrityDataTokenizationOptions +public interface DataTokenizationOptions extends PipelineOptions, GcsIO.GcsPipelineOptions, BigTableIO.BigTableOptions { @Description("Path to data schema (JSON format) in GCS compatible with BigQuery.") @@ -48,25 +48,18 @@ public interface ProtegrityDataTokenizationOptions void setBigQueryTableName(String bigQueryTableName); - // Protegrity specific parameters - @Description("URI for the API calls to DSG.") - String getDsgUri(); + // Tokenization API specific parameters + @Description("URI for the API calls to RPC server.") + String getRpcUri(); - void setDsgUri(String dsgUri); + void setRpcUri(String dsgUri); - @Description("Size of the batch to send to DSG per request.") + @Description("Size of the batch to send to RPC server per request.") @Default.Integer(10) Integer getBatchSize(); void setBatchSize(Integer batchSize); - @Description( - "GCS path to the payload configuration file with an array of fields " - + "to extract for tokenization.") - String getPayloadConfigGcsPath(); - - void setPayloadConfigGcsPath(String payloadConfigGcsPath); - @Description("Dead-Letter GCS path to store not-tokenized data") String getNonTokenizedDeadLetterGcsPath(); diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java index 32d85a66f6f9..e14e8a9ab4ec 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java @@ -28,12 +28,15 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.UUID; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; +import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.state.StateSpecs; @@ -75,6 +78,8 @@ public class DataProtectors { /** Logger for class. */ private static final Logger LOG = LoggerFactory.getLogger(DataProtectors.class); + public static final String ID_FIELD_NAME = "ID"; + /** * The {@link RowToTokenizedRow} transform converts {@link Row} to {@link TableRow} objects. The * transform accepts a {@link FailsafeElement} object so the original payload of the incoming @@ -134,6 +139,7 @@ public abstract static class Builder { @SuppressWarnings("initialization.static.fields.uninitialized") public static class TokenizationFn extends DoFn, Row> { + private static Schema schemaToDsg; private static CloseableHttpClient httpclient; private static ObjectMapper objectMapperSerializerForSchema; private static ObjectMapper objectMapperDeserializerForSchema; @@ -152,8 +158,6 @@ public static class TokenizationFn extends DoFn, Row> { @TimerId("expiry") private final TimerSpec expirySpec = TimerSpecs.timer(TimeDomain.EVENT_TIME); - private boolean hasIdInInputs = true; - private String idFieldName = ""; private Map inputRowsWithIds; public TokenizationFn( @@ -172,11 +176,15 @@ public TokenizationFn( @Setup public void setup() { + List fields = schema.getFields(); + fields.add(Field.of(ID_FIELD_NAME, FieldType.STRING)); + schemaToDsg = new Schema(fields); + objectMapperSerializerForSchema = - RowJsonUtils.newObjectMapperWith(RowJson.RowJsonSerializer.forSchema(schema)); + RowJsonUtils.newObjectMapperWith(RowJson.RowJsonSerializer.forSchema(schemaToDsg)); objectMapperDeserializerForSchema = - RowJsonUtils.newObjectMapperWith(RowJson.RowJsonDeserializer.forSchema(schema)); + RowJsonUtils.newObjectMapperWith(RowJson.RowJsonDeserializer.forSchema(schemaToDsg)); httpclient = HttpClients.createDefault(); } @@ -247,22 +255,15 @@ private ArrayList rowsToJsons(Iterable inputRows) { Map inputRowsWithIds = new HashMap<>(); for (Row inputRow : inputRows) { - Row.Builder builder = Row.withSchema(schema); - for (Schema.Field field : schema.getFields()) { + Row.Builder builder = Row.withSchema(schemaToDsg); + for (Schema.Field field : schemaToDsg.getFields()) { if (inputRow.getSchema().hasField(field.getName())) { builder = builder.addValue(inputRow.getValue(field.getName())); } } - String id; - if (!hasIdInInputs) { - id = UUID.randomUUID().toString(); - builder = builder.addValue(id); - } else { - id = inputRow.getValue(idFieldName); - } - if (id != null) { - inputRowsWithIds.put(id, inputRow); - } + String id = UUID.randomUUID().toString(); + builder = builder.addValue(id); + inputRowsWithIds.put(id, inputRow); Row row = builder.build(); @@ -297,9 +298,9 @@ private ArrayList getTokenizedRow(Iterable inputRows) throws IOExcepti RowJsonUtils.jsonToRow( objectMapperDeserializerForSchema, jsonTokenizedRows.get(i).toString()); Row.FieldValueBuilder rowBuilder = - Row.fromRow(this.inputRowsWithIds.get(tokenizedRow.getString(idFieldName))); - for (Schema.Field field : schema.getFields()) { - if (!hasIdInInputs && field.getName().equals(idFieldName)) { + Row.fromRow(this.inputRowsWithIds.get(tokenizedRow.getString(ID_FIELD_NAME))); + for (Schema.Field field : schemaToDsg.getFields()) { + if (field.getName().equals(ID_FIELD_NAME)) { continue; } rowBuilder = diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java index c499936c37dd..695c25770e56 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java @@ -23,7 +23,7 @@ import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; -import org.apache.beam.examples.complete.datatokenization.options.ProtegrityDataTokenizationOptions; +import org.apache.beam.examples.complete.datatokenization.options.DataTokenizationOptions; import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO; import org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteResult; import org.apache.beam.sdk.options.Description; @@ -46,9 +46,9 @@ public class BigTableIO { /** Logger for class. */ private static final Logger LOG = LoggerFactory.getLogger(BigTableIO.class); - private final ProtegrityDataTokenizationOptions options; + private final DataTokenizationOptions options; - public BigTableIO(ProtegrityDataTokenizationOptions options) { + public BigTableIO(DataTokenizationOptions options) { this.options = options; } @@ -76,8 +76,7 @@ static class TransformToBigTableFormat extends DoFn>> out, ProcessContext c) { - ProtegrityDataTokenizationOptions options = - c.getPipelineOptions().as(ProtegrityDataTokenizationOptions.class); + DataTokenizationOptions options = c.getPipelineOptions().as(DataTokenizationOptions.class); // Mapping every field in provided Row to Mutation.SetCell, which will create/update // cell content with provided data Set mutations = diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java index 58bdc1510ae0..5116b17139ed 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java @@ -17,7 +17,7 @@ */ package org.apache.beam.examples.complete.datatokenization.transforms.io; -import org.apache.beam.examples.complete.datatokenization.options.ProtegrityDataTokenizationOptions; +import org.apache.beam.examples.complete.datatokenization.options.DataTokenizationOptions; import org.apache.beam.examples.complete.datatokenization.utils.CsvConverters; import org.apache.beam.examples.complete.datatokenization.utils.ErrorConverters; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement; @@ -34,6 +34,7 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ToJson; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.Row; @@ -102,6 +103,18 @@ public interface GcsPipelineOptions extends PipelineOptions { void setOutputGcsFileFormat(FORMAT outputGcsFileFormat); + @Description( + "The window duration in which data will be written. " + + "Should be specified only for 'Pub/Sub -> GCS' case. Defaults to 30s. " + + "Allowed formats are: " + + "Ns (for seconds, example: 5s), " + + "Nm (for minutes, example: 12m), " + + "Nh (for hours, example: 2h).") + @Default.String("30s") + String getWindowDuration(); + + void setWindowDuration(String windowDuration); + // CSV parameters @Description("If file(s) contain headers") Boolean getCsvContainsHeaders(); @@ -126,9 +139,9 @@ public interface GcsPipelineOptions extends PipelineOptions { void setCsvFormat(String csvFormat); } - private final ProtegrityDataTokenizationOptions options; + private final DataTokenizationOptions options; - public GcsIO(ProtegrityDataTokenizationOptions options) { + public GcsIO(DataTokenizationOptions options) { this.options = options; } @@ -196,19 +209,40 @@ public PCollection read(Pipeline pipeline, String schema) { public PDone write(PCollection input, Schema schema) { if (options.getOutputGcsFileFormat() == FORMAT.JSON) { - return input - .apply("RowsToJSON", ToJson.of()) - .apply("WriteToGCS", TextIO.write().to(options.getOutputGcsDirectory())); + PCollection jsons = input.apply("RowsToJSON", ToJson.of()); + + if (jsons.isBounded() == IsBounded.BOUNDED) { + return jsons.apply("WriteToGCS", TextIO.write().to(options.getOutputGcsDirectory())); + } else { + return jsons.apply( + "WriteToGCS", + TextIO.write() + .withWindowedWrites() + .withNumShards(1) + .to(options.getOutputGcsDirectory())); + } } else if (options.getOutputGcsFileFormat() == FORMAT.CSV) { String header = String.join(options.getCsvDelimiter(), schema.getFieldNames()); String csvDelimiter = options.getCsvDelimiter(); - return input - .apply( + + PCollection csvs = + input.apply( "ConvertToCSV", MapElements.into(TypeDescriptors.strings()) - .via((Row inputRow) -> new RowToCsv(csvDelimiter).getCsvFromRow(inputRow))) - .apply( - "WriteToGCS", TextIO.write().to(options.getOutputGcsDirectory()).withHeader(header)); + .via((Row inputRow) -> new RowToCsv(csvDelimiter).getCsvFromRow(inputRow))); + + if (csvs.isBounded() == IsBounded.BOUNDED) { + return csvs.apply( + "WriteToGCS", TextIO.write().to(options.getOutputGcsDirectory()).withHeader(header)); + } else { + return csvs.apply( + "WriteToGCS", + TextIO.write() + .withWindowedWrites() + .withNumShards(1) + .to(options.getOutputGcsDirectory()) + .withHeader(header)); + } } else { throw new IllegalStateException( diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/DurationUtils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/DurationUtils.java new file mode 100644 index 000000000000..83d3aea3acb5 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/DurationUtils.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.datatokenization.utils; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import java.util.Locale; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.MutablePeriod; +import org.joda.time.format.PeriodFormatterBuilder; +import org.joda.time.format.PeriodParser; + +/** + * The {@link DurationUtils} class provides common utilities for manipulating and formatting {@link + * Duration} objects. + */ +public class DurationUtils { + + /** + * Parses a duration from a period formatted string. Values are accepted in the following formats: + * + *

Formats Ns - Seconds. Example: 5s
+ * Nm - Minutes. Example: 13m
+ * Nh - Hours. Example: 2h + * + *

+   * parseDuration(null) = NullPointerException()
+   * parseDuration("")   = Duration.standardSeconds(0)
+   * parseDuration("2s") = Duration.standardSeconds(2)
+   * parseDuration("5m") = Duration.standardMinutes(5)
+   * parseDuration("3h") = Duration.standardHours(3)
+   * 
+ * + * @param value The period value to parse. + * @return The {@link Duration} parsed from the supplied period string. + */ + public static Duration parseDuration(String value) { + checkNotNull(value, "The specified duration must be a non-null value!"); + + PeriodParser parser = + new PeriodFormatterBuilder() + .appendSeconds() + .appendSuffix("s") + .appendMinutes() + .appendSuffix("m") + .appendHours() + .appendSuffix("h") + .toParser(); + + MutablePeriod period = new MutablePeriod(); + parser.parseInto(period, value, 0, Locale.getDefault()); + + Duration duration = period.toDurationFrom(new DateTime(0)); + checkArgument(duration.getMillis() > 0, "The window duration must be greater than 0!"); + + return duration; + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java index 77122bf5d291..1cba570239c1 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java @@ -21,6 +21,7 @@ import com.google.auto.value.AutoValue; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import javax.annotation.Nullable; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; @@ -29,9 +30,13 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PDone; import org.joda.time.DateTimeZone; +import org.joda.time.Duration; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; @@ -51,12 +56,28 @@ public static Builder newBuilder() { public abstract String csvDelimiter(); + @Nullable + public abstract Duration windowDuration(); + + @SuppressWarnings("argument.type.incompatible") @Override public PDone expand(PCollection> pCollection) { - return pCollection - .apply("GetFormattedErrorRow", ParDo.of(new FailedStringToCsvRowFn(csvDelimiter()))) - .apply(TextIO.write().to(errorWritePath()).withNumShards(1)); + PCollection formattedErrorRows = + pCollection.apply( + "GetFormattedErrorRow", ParDo.of(new FailedStringToCsvRowFn(csvDelimiter()))); + + if (pCollection.isBounded() == IsBounded.UNBOUNDED) { + if (windowDuration() != null) { + formattedErrorRows = + formattedErrorRows.apply(Window.into(FixedWindows.of(windowDuration()))); + } + return formattedErrorRows.apply( + TextIO.write().to(errorWritePath()).withNumShards(1).withWindowedWrites()); + + } else { + return formattedErrorRows.apply(TextIO.write().to(errorWritePath()).withNumShards(1)); + } } /** Builder for {@link WriteStringMessageErrorsAsCsv}. */ @@ -67,6 +88,8 @@ public abstract static class Builder { public abstract Builder setCsvDelimiter(String csvDelimiter); + public abstract Builder setWindowDuration(@Nullable Duration duration); + public abstract WriteStringMessageErrorsAsCsv build(); } } From 6e8f62f5e19b025e1b4d882cbbe3bb2593f8cb2a Mon Sep 17 00:00:00 2001 From: Ramazan Yapparov Date: Fri, 15 Jan 2021 16:52:30 +0300 Subject: [PATCH 13/29] supressed checkstyle errors for BigTableIO class --- sdks/java/build-tools/src/main/resources/beam/suppressions.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/java/build-tools/src/main/resources/beam/suppressions.xml b/sdks/java/build-tools/src/main/resources/beam/suppressions.xml index ee03dc303924..e51a71a60a82 100644 --- a/sdks/java/build-tools/src/main/resources/beam/suppressions.xml +++ b/sdks/java/build-tools/src/main/resources/beam/suppressions.xml @@ -106,6 +106,7 @@ + From b489cc76f9075a1631761481498f1bfa3cb4c96d Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Mon, 18 Jan 2021 11:13:36 +0300 Subject: [PATCH 14/29] add data tokenization tests --- .../DataTokenizationTest.java | 206 ++++++++++++++++++ examples/java/src/test/resources/schema.txt | 19 ++ .../java/src/test/resources/testInput.csv | 3 + .../java/src/test/resources/testInput.txt | 3 + 4 files changed, 231 insertions(+) create mode 100644 examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java create mode 100644 examples/java/src/test/resources/schema.txt create mode 100644 examples/java/src/test/resources/testInput.csv create mode 100644 examples/java/src/test/resources/testInput.txt diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java new file mode 100644 index 000000000000..513d9234fdfa --- /dev/null +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java @@ -0,0 +1,206 @@ +/* + * 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.examples.complete.datatokenization; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.startsWith; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import org.apache.beam.examples.complete.datatokenization.options.DataTokenizationOptions; +import org.apache.beam.examples.complete.datatokenization.transforms.io.GcsIO; +import org.apache.beam.examples.complete.datatokenization.transforms.io.GcsIO.FORMAT; +import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; +import org.apache.beam.examples.complete.datatokenization.utils.RowToCsv; +import org.apache.beam.examples.complete.datatokenization.utils.SchemasUtils; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.JsonToRow; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.Resources; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Test class for {@link DataTokenization}. + */ +@RunWith(JUnit4.class) +public class DataTokenizationTest { + + private static final String testSchema = + "{\"fields\":[{\"mode\":\"REQUIRED\",\"name\":\"FieldName1\",\"type\":\"STRING\"},{\"mode\":\"REQUIRED\",\"name\":\"FieldName2\",\"type\":\"STRING\"}]}"; + String[] fields = {"TestValue1", "TestValue2"}; + + @Rule + public final transient TestPipeline testPipeline = TestPipeline.create(); + + private static final String RESOURCES_DIR = "./"; + + private static final String CSV_FILE_PATH = + Resources.getResource(RESOURCES_DIR + "testInput.csv").getPath(); + + private static final String JSON_FILE_PATH = + Resources.getResource(RESOURCES_DIR + "testInput.txt").getPath(); + + private static final String SCHEMA_FILE_PATH = + Resources.getResource(RESOURCES_DIR + "schema.txt").getPath(); + + private static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER = + FailsafeElementCoder.of( + NullableCoder.of(StringUtf8Coder.of()), NullableCoder.of(StringUtf8Coder.of())); + + @Test + public void testGetBeamSchema() { + Schema expectedSchema = + Schema.builder() + .addField("FieldName1", FieldType.STRING) + .addField("FieldName2", FieldType.STRING) + .build(); + SchemasUtils schemasUtils = new SchemasUtils(testSchema); + Assert.assertEquals(expectedSchema, schemasUtils.getBeamSchema()); + } + + @Test + public void testGetBigQuerySchema() { + SchemasUtils schemasUtils = new SchemasUtils(testSchema); + Assert.assertEquals(testSchema, schemasUtils.getBigQuerySchema().toString()); + } + + // @Test + // public void testNullRowToCSV() { + // Schema beamSchema = Schema.builder().addNullableField("FieldString", + // FieldType.STRING).build(); + // Row.Builder rowBuilder = Row.withSchema(beamSchema); + // Row row = rowBuilder.addValue(null).build(); + // System.out.println(row); + // String csv = new RowToCsv(",").getCsvFromRow(row); + // Assert.assertEquals("null", csv); + // } + + @Test + public void testRowToCSV() { + Schema beamSchema = new SchemasUtils(testSchema).getBeamSchema(); + Row.Builder rowBuilder = Row.withSchema(beamSchema); + Row row = rowBuilder.addValues(new ArrayList<>(Arrays.asList(fields))).build(); + String csvResult = new RowToCsv(";").getCsvFromRow(row); + Assert.assertEquals(String.join(";", fields), csvResult); + } + + @Test + public void testGcsIOReadCSV() throws IOException { + PCollection jsons = gcsIORead(CSV_FILE_PATH, FORMAT.CSV); + assertField(jsons); + testPipeline.run(); + } + + @Test + public void testGcsIOReadJSON() throws IOException { + PCollection jsons = gcsIORead(JSON_FILE_PATH, FORMAT.JSON); + assertField(jsons); + testPipeline.run(); + } + + @Test + public void testJsonToRow() throws IOException { + PCollection jsons = gcsIORead(JSON_FILE_PATH, FORMAT.JSON); + SchemasUtils testSchemaUtils = new SchemasUtils(SCHEMA_FILE_PATH, StandardCharsets.UTF_8); + JsonToRow.ParseResult rows = + jsons.apply( + "JsonToRow", + JsonToRow.withExceptionReporting(testSchemaUtils.getBeamSchema()) + .withExtendedErrorInfo()); + PAssert.that(rows.getResults()) + .satisfies( + x -> { + LinkedList beamRows = Lists.newLinkedList(x); + assertThat(beamRows, hasSize(3)); + beamRows.forEach( + row -> { + List fieldValues = row.getValues(); + for (Object element : fieldValues) { + assertThat((String) element, startsWith("FieldValue")); + } + }); + return null; + }); + testPipeline.run(); + } + + private PCollection gcsIORead(String inputGcsFilePattern, FORMAT inputGcsFileFormat) + throws IOException { + DataTokenizationOptions options = + PipelineOptionsFactory.create().as(DataTokenizationOptions.class); + options.setDataSchemaGcsPath(SCHEMA_FILE_PATH); + options.setInputGcsFilePattern(inputGcsFilePattern); + options.setInputGcsFileFormat(inputGcsFileFormat); + if (inputGcsFileFormat == FORMAT.CSV) { + options.setCsvContainsHeaders(Boolean.FALSE); + } + + SchemasUtils testSchemaUtils = + new SchemasUtils(options.getDataSchemaGcsPath(), StandardCharsets.UTF_8); + + CoderRegistry coderRegistry = testPipeline.getCoderRegistry(); + coderRegistry.registerCoderForType( + FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor(), FAILSAFE_ELEMENT_CODER); + coderRegistry.registerCoderForType( + RowCoder.of(testSchemaUtils.getBeamSchema()).getEncodedTypeDescriptor(), + RowCoder.of(testSchemaUtils.getBeamSchema())); + /* + * Row/Row Coder for FailsafeElement. + */ + FailsafeElementCoder coder = + FailsafeElementCoder.of( + RowCoder.of(testSchemaUtils.getBeamSchema()), + RowCoder.of(testSchemaUtils.getBeamSchema())); + coderRegistry.registerCoderForType(coder.getEncodedTypeDescriptor(), coder); + + return new GcsIO(options).read(testPipeline, testSchemaUtils.getJsonBeamSchema()); + } + + private void assertField(PCollection jsons) { + PAssert.that(jsons) + .satisfies( + x -> { + LinkedList rows = Lists.newLinkedList(x); + assertThat(rows, hasSize(3)); + rows.forEach( + row -> { + assertThat(row, startsWith("{\"Field1\":")); + }); + return null; + }); + } +} diff --git a/examples/java/src/test/resources/schema.txt b/examples/java/src/test/resources/schema.txt new file mode 100644 index 000000000000..10f5ae2eded7 --- /dev/null +++ b/examples/java/src/test/resources/schema.txt @@ -0,0 +1,19 @@ +{ + "fields": [ + { + "mode": "REQUIRED", + "name": "Field1", + "type": "STRING" + }, + { + "mode": "REQUIRED", + "name": "Field2", + "type": "STRING" + }, + { + "mode": "REQUIRED", + "name": "Field3", + "type": "STRING" + } + ] +} \ No newline at end of file diff --git a/examples/java/src/test/resources/testInput.csv b/examples/java/src/test/resources/testInput.csv new file mode 100644 index 000000000000..9e88e1312b27 --- /dev/null +++ b/examples/java/src/test/resources/testInput.csv @@ -0,0 +1,3 @@ +FieldValue11,FieldValue12,FieldValue13 +FieldValue21,FieldValue22,FieldValue23 +FieldValue31,FieldValue32,FieldValue33 \ No newline at end of file diff --git a/examples/java/src/test/resources/testInput.txt b/examples/java/src/test/resources/testInput.txt new file mode 100644 index 000000000000..703469f53d18 --- /dev/null +++ b/examples/java/src/test/resources/testInput.txt @@ -0,0 +1,3 @@ +{"Field1": "FieldValue11", "Field2": "FieldValue12", "Field3": "FieldValue13"} +{"Field1": "FieldValue12", "Field2": "FieldValue22", "Field3": "FieldValue23"} +{"Field1": "FieldValue13", "Field2": "FieldValue32", "Field3": "FieldValue33"} \ No newline at end of file From ee93d1430235099301ba01b5d2d0f0f00c6bbba0 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Mon, 18 Jan 2021 18:21:21 +0300 Subject: [PATCH 15/29] Changed GCS to FileSystem and removed redundant function from SchemasUtils --- .../datatokenization/DataTokenization.java | 29 ++++---- .../complete/datatokenization/README.md | 30 ++++---- .../options/DataTokenizationOptions.java | 16 ++-- .../transforms/DataProtectors.java | 14 ++-- .../io/{GcsIO.java => FileSystemIO.java} | 74 +++++++++---------- .../datatokenization/utils/SchemasUtils.java | 25 ------- .../DataTokenizationTest.java | 35 ++++----- 7 files changed, 94 insertions(+), 129 deletions(-) rename examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/{GcsIO.java => FileSystemIO.java} (78%) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java index f09bf30c66f7..eb0e56e17ee7 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java @@ -27,7 +27,7 @@ import org.apache.beam.examples.complete.datatokenization.transforms.DataProtectors.RowToTokenizedRow; import org.apache.beam.examples.complete.datatokenization.transforms.io.BigQueryIO; import org.apache.beam.examples.complete.datatokenization.transforms.io.BigTableIO; -import org.apache.beam.examples.complete.datatokenization.transforms.io.GcsIO; +import org.apache.beam.examples.complete.datatokenization.transforms.io.FileSystemIO; import org.apache.beam.examples.complete.datatokenization.utils.ErrorConverters; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; @@ -102,7 +102,7 @@ public static void main(String[] args) { public static PipelineResult run(DataTokenizationOptions options) { SchemasUtils schema = null; try { - schema = new SchemasUtils(options.getDataSchemaGcsPath(), StandardCharsets.UTF_8); + schema = new SchemasUtils(options.getDataSchemaPath(), StandardCharsets.UTF_8); } catch (IOException e) { LOG.error("Failed to retrieve schema for data.", e); } @@ -128,18 +128,19 @@ public static PipelineResult run(DataTokenizationOptions options) { coderRegistry.registerCoderForType(coder.getEncodedTypeDescriptor(), coder); PCollection jsons; - if (options.getInputGcsFilePattern() != null) { - jsons = new GcsIO(options).read(pipeline, schema.getJsonBeamSchema()); + if (options.getInputFilePattern() != null) { + jsons = new FileSystemIO(options).read(pipeline, schema.getJsonBeamSchema()); } else if (options.getPubsubTopic() != null) { jsons = pipeline.apply( "ReadMessagesFromPubsub", PubsubIO.readStrings().fromTopic(options.getPubsubTopic())); - if (options.getOutputGcsDirectory() != null) { + if (options.getOutputDirectory() != null) { jsons = jsons.apply(Window.into(FixedWindows.of(parseDuration(options.getWindowDuration())))); } } else { - throw new IllegalStateException("No source is provided, please configure GCS or Pub/Sub"); + throw new IllegalStateException( + "No source is provided, please configure File System or Pub/Sub"); } JsonToRow.ParseResult rows = @@ -147,7 +148,7 @@ public static PipelineResult run(DataTokenizationOptions options) { "JsonToRow", JsonToRow.withExceptionReporting(schema.getBeamSchema()).withExtendedErrorInfo()); - if (options.getNonTokenizedDeadLetterGcsPath() != null) { + if (options.getNonTokenizedDeadLetterPath() != null) { /* * Write Row conversion errors to filesystem specified path */ @@ -160,10 +161,10 @@ public static PipelineResult run(DataTokenizationOptions options) { FailsafeElement.of(errRow.getString("line"), errRow.getString("line")) .setErrorMessage(errRow.getString("err")))) .apply( - "WriteCsvConversionErrorsToGcs", + "WriteCsvConversionErrorsToFS", ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder() .setCsvDelimiter(options.getCsvDelimiter()) - .setErrorWritePath(options.getNonTokenizedDeadLetterGcsPath()) + .setErrorWritePath(options.getNonTokenizedDeadLetterPath()) .build()); } /* @@ -188,7 +189,7 @@ public static PipelineResult run(DataTokenizationOptions options) { .build()); String csvDelimiter = options.getCsvDelimiter(); - if (options.getNonTokenizedDeadLetterGcsPath() != null) { + if (options.getNonTokenizedDeadLetterPath() != null) { /* Write tokenization errors to dead-letter sink */ @@ -203,15 +204,15 @@ public static PipelineResult run(DataTokenizationOptions options) { new RowToCsv(csvDelimiter).getCsvFromRow(fse.getOriginalPayload()), new RowToCsv(csvDelimiter).getCsvFromRow(fse.getPayload())))) .apply( - "WriteTokenizationErrorsToGcs", + "WriteTokenizationErrorsToFS", ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder() .setCsvDelimiter(options.getCsvDelimiter()) - .setErrorWritePath(options.getNonTokenizedDeadLetterGcsPath()) + .setErrorWritePath(options.getNonTokenizedDeadLetterPath()) .build()); } - if (options.getOutputGcsDirectory() != null) { - new GcsIO(options).write(tokenizedRows.get(TOKENIZATION_OUT), schema.getBeamSchema()); + if (options.getOutputDirectory() != null) { + new FileSystemIO(options).write(tokenizedRows.get(TOKENIZATION_OUT), schema.getBeamSchema()); } else if (options.getBigQueryTableName() != null) { WriteResult writeResult = BigQueryIO.write( diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md index 8a664632aff6..a454c80fbf49 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md @@ -29,14 +29,12 @@ Supported data formats: Supported input sources: -- Local file system -- [Google Cloud Storage](https://cloud.google.com/storage) +- File system - [Google Pub/Sub](https://cloud.google.com/pubsub) Supported destination sinks: -- Local file system -- Google Cloud Storage +- File system - [Google Cloud BigQuery](https://cloud.google.com/bigquery) - [Cloud BigTable](https://cloud.google.com/bigtable) @@ -92,14 +90,12 @@ gradle clean execute -DmainClass=org.apache.beam.examples.complete.datatokenizat To execute this pipeline, specify the parameters: -**NOTE:** Any parameters called as GSC also supports local file system. - - Data schema - - **dataSchemaGcsPath**: Path to data schema (JSON format) in GCS compatible with BigQuery. + - **dataSchemaPath**: Path to data schema (JSON format) compatible with BigQuery. - 1 specified input source out of these: - - Google Cloud Storage - - **inputGcsFilePattern**: GCS filepattern for files in bucket to read data from - - **inputGcsFileFormat**: File format of input files. Supported formats: JSON, CSV + - File System + - **inputFilePattern**: Filepattern for files to read data from + - **inputFileFormat**: File format of input files. Supported formats: JSON, CSV - In case if input data is in CSV format: - **csvContainsHeaders**: `true` if file(s) in bucket to read data from contain headers, and `false` otherwise @@ -113,9 +109,9 @@ To execute this pipeline, specify the parameters: - **pubsubTopic**: The Cloud Pub/Sub topic to read from, in the format of ' projects/yourproject/topics/yourtopic' - 1 specified output sink out of these: - - Google Cloud Storage - - **outputGcsDirectory**: GCS directory in bucket to write data to - - **outputGcsFileFormat**: File format of output files. Supported formats: JSON, CSV + - File System + - **outputDirectory**: Directory to write data to + - **outputFileFormat**: File format of output files. Supported formats: JSON, CSV - **windowDuration**: The window duration in which data will be written. Should be specified only for 'Pub/Sub -> GCS' case. Defaults to 30s. @@ -133,14 +129,14 @@ To execute this pipeline, specify the parameters: - **bigTableKeyColumnName**: Column name to use as a key in Cloud BigTable - **bigTableColumnFamilyName**: Column family name to use in Cloud BigTable - DSG parameters - - **dsgUri**: URI for the API calls to DSG + - **rpcUri**: URI for the API calls to DSG - **batchSize**: Size of the batch to send to DSG per request - - **payloadConfigGcsPath**: GCS path to the payload configuration file with an array of fields - to extract for tokenization + - **payloadConfigPath**: Path to the payload configuration file with an array of fields to + extract for tokenization The template allows for the user to supply the following optional parameter: -- **nonTokenizedDeadLetterGcsPath**: GCS folder where failed to tokenize data will be stored +- **nonTokenizedDeadLetterPath**: Folder where failed to tokenize data will be stored in the following format: diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/DataTokenizationOptions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/DataTokenizationOptions.java index 6e383a1df822..767ca8644d3d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/DataTokenizationOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/DataTokenizationOptions.java @@ -18,7 +18,7 @@ package org.apache.beam.examples.complete.datatokenization.options; import org.apache.beam.examples.complete.datatokenization.transforms.io.BigTableIO; -import org.apache.beam.examples.complete.datatokenization.transforms.io.GcsIO; +import org.apache.beam.examples.complete.datatokenization.transforms.io.FileSystemIO.FileSystemPipelineOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; @@ -28,12 +28,12 @@ * executor at the command-line. */ public interface DataTokenizationOptions - extends PipelineOptions, GcsIO.GcsPipelineOptions, BigTableIO.BigTableOptions { + extends PipelineOptions, FileSystemPipelineOptions, BigTableIO.BigTableOptions { - @Description("Path to data schema (JSON format) in GCS compatible with BigQuery.") - String getDataSchemaGcsPath(); + @Description("Path to data schema (JSON format) compatible with BigQuery.") + String getDataSchemaPath(); - void setDataSchemaGcsPath(String dataSchemaGcsPath); + void setDataSchemaPath(String dataSchemaPath); @Description( "The Cloud Pub/Sub topic to read from." @@ -60,8 +60,8 @@ public interface DataTokenizationOptions void setBatchSize(Integer batchSize); - @Description("Dead-Letter GCS path to store not-tokenized data") - String getNonTokenizedDeadLetterGcsPath(); + @Description("Dead-Letter path to store not-tokenized data") + String getNonTokenizedDeadLetterPath(); - void setNonTokenizedDeadLetterGcsPath(String nonTokenizedDeadLetterGcsPath); + void setNonTokenizedDeadLetterPath(String nonTokenizedDeadLetterPath); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java index e14e8a9ab4ec..245e99c84459 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java @@ -139,7 +139,7 @@ public abstract static class Builder { @SuppressWarnings("initialization.static.fields.uninitialized") public static class TokenizationFn extends DoFn, Row> { - private static Schema schemaToDsg; + private static Schema schemaToRpc; private static CloseableHttpClient httpclient; private static ObjectMapper objectMapperSerializerForSchema; private static ObjectMapper objectMapperDeserializerForSchema; @@ -178,13 +178,13 @@ public void setup() { List fields = schema.getFields(); fields.add(Field.of(ID_FIELD_NAME, FieldType.STRING)); - schemaToDsg = new Schema(fields); + schemaToRpc = new Schema(fields); objectMapperSerializerForSchema = - RowJsonUtils.newObjectMapperWith(RowJson.RowJsonSerializer.forSchema(schemaToDsg)); + RowJsonUtils.newObjectMapperWith(RowJson.RowJsonSerializer.forSchema(schemaToRpc)); objectMapperDeserializerForSchema = - RowJsonUtils.newObjectMapperWith(RowJson.RowJsonDeserializer.forSchema(schemaToDsg)); + RowJsonUtils.newObjectMapperWith(RowJson.RowJsonDeserializer.forSchema(schemaToRpc)); httpclient = HttpClients.createDefault(); } @@ -255,8 +255,8 @@ private ArrayList rowsToJsons(Iterable inputRows) { Map inputRowsWithIds = new HashMap<>(); for (Row inputRow : inputRows) { - Row.Builder builder = Row.withSchema(schemaToDsg); - for (Schema.Field field : schemaToDsg.getFields()) { + Row.Builder builder = Row.withSchema(schemaToRpc); + for (Schema.Field field : schemaToRpc.getFields()) { if (inputRow.getSchema().hasField(field.getName())) { builder = builder.addValue(inputRow.getValue(field.getName())); } @@ -299,7 +299,7 @@ private ArrayList getTokenizedRow(Iterable inputRows) throws IOExcepti objectMapperDeserializerForSchema, jsonTokenizedRows.get(i).toString()); Row.FieldValueBuilder rowBuilder = Row.fromRow(this.inputRowsWithIds.get(tokenizedRow.getString(ID_FIELD_NAME))); - for (Schema.Field field : schemaToDsg.getFields()) { + for (Schema.Field field : schemaToRpc.getFields()) { if (field.getName().equals(ID_FIELD_NAME)) { continue; } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/FileSystemIO.java similarity index 78% rename from examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/FileSystemIO.java index 5116b17139ed..a5dfaa86674b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/GcsIO.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/FileSystemIO.java @@ -43,8 +43,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** The {@link GcsIO} class to read/write data from/into Google Cloud Storage. */ -public class GcsIO { +/** The {@link FileSystemIO} class to read/write data from/into File Systems. */ +public class FileSystemIO { /** The tag for the headers of the CSV if required. */ static final TupleTag CSV_HEADERS = new TupleTag() {}; @@ -61,7 +61,7 @@ public class GcsIO { new TupleTag>() {}; /* Logger for class. */ - private static final Logger LOG = LoggerFactory.getLogger(GcsIO.class); + private static final Logger LOG = LoggerFactory.getLogger(FileSystemIO.class); public static final String DEAD_LETTER_PREFIX = "CSV_CONVERTOR"; @@ -77,35 +77,35 @@ public enum FORMAT { } /** - * Necessary {@link PipelineOptions} options for Pipelines that operate with JSON/CSV data in GCS. + * Necessary {@link PipelineOptions} options for Pipelines that operate with JSON/CSV data in FS. */ - public interface GcsPipelineOptions extends PipelineOptions { + public interface FileSystemPipelineOptions extends PipelineOptions { - @Description("GCS filepattern for files in bucket to read data from") - String getInputGcsFilePattern(); + @Description("Filepattern for files to read data from") + String getInputFilePattern(); - void setOutputGcsDirectory(String outputGcsDirectory); + void setInputFilePattern(String inputFilePattern); @Description("File format of input files. Supported formats: JSON, CSV") @Default.Enum("JSON") - GcsIO.FORMAT getInputGcsFileFormat(); + FileSystemIO.FORMAT getInputFileFormat(); - void setInputGcsFilePattern(String inputGcsFilePattern); + void setInputFileFormat(FORMAT inputFileFormat); - @Description("GCS directory in bucket to write data to") - String getOutputGcsDirectory(); + @Description("Directory to write data to") + String getOutputDirectory(); - void setInputGcsFileFormat(FORMAT inputGcsFileFormat); + void setOutputDirectory(String outputDirectory); @Description("File format of output files. Supported formats: JSON, CSV") @Default.Enum("JSON") - GcsIO.FORMAT getOutputGcsFileFormat(); + FileSystemIO.FORMAT getOutputFileFormat(); - void setOutputGcsFileFormat(FORMAT outputGcsFileFormat); + void setOutputFileFormat(FORMAT outputFileFormat); @Description( "The window duration in which data will be written. " - + "Should be specified only for 'Pub/Sub -> GCS' case. Defaults to 30s. " + + "Should be specified only for 'Pub/Sub -> FS' case. Defaults to 30s. " + "Allowed formats are: " + "Ns (for seconds, example: 5s), " + "Nm (for minutes, example: 12m), " @@ -141,27 +141,26 @@ public interface GcsPipelineOptions extends PipelineOptions { private final DataTokenizationOptions options; - public GcsIO(DataTokenizationOptions options) { + public FileSystemIO(DataTokenizationOptions options) { this.options = options; } public PCollection read(Pipeline pipeline, String schema) { - if (options.getInputGcsFileFormat() == FORMAT.JSON) { - return pipeline.apply( - "ReadJsonFromGCSFiles", TextIO.read().from(options.getInputGcsFilePattern())); - } else if (options.getInputGcsFileFormat() == FORMAT.CSV) { + if (options.getInputFileFormat() == FORMAT.JSON) { + return pipeline.apply("ReadJsonFromFiles", TextIO.read().from(options.getInputFilePattern())); + } else if (options.getInputFileFormat() == FORMAT.CSV) { PCollectionTuple jsons = pipeline /* - * Step 1: Read CSV file(s) from Cloud Storage using {@link CsvConverters.ReadCsv}. + * Step 1: Read CSV file(s) from File System using {@link CsvConverters.ReadCsv}. */ .apply( - "ReadCsvFromGcsFiles", + "ReadCsvFromFiles", CsvConverters.ReadCsv.newBuilder() .setCsvFormat(options.getCsvFormat()) .setDelimiter(options.getCsvDelimiter()) .setHasHeaders(options.getCsvContainsHeaders()) - .setInputFileSpec(options.getInputGcsFilePattern()) + .setInputFileSpec(options.getInputFilePattern()) .setHeaderTag(CSV_HEADERS) .setLineTag(CSV_LINES) .build()) @@ -179,17 +178,17 @@ public PCollection read(Pipeline pipeline, String schema) { .setUdfDeadletterTag(PROCESSING_DEADLETTER_OUT) .build()); - if (options.getNonTokenizedDeadLetterGcsPath() != null) { + if (options.getNonTokenizedDeadLetterPath() != null) { /* - * Step 3: Write jsons to dead-letter gcs that were successfully processed. + * Step 3: Write jsons to dead-letter that weren't successfully processed. */ jsons .get(PROCESSING_DEADLETTER_OUT) .apply( - "WriteCsvConversionErrorsToGcs", + "WriteCsvConversionErrorsToFS", ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder() .setCsvDelimiter(options.getCsvDelimiter()) - .setErrorWritePath(options.getNonTokenizedDeadLetterGcsPath()) + .setErrorWritePath(options.getNonTokenizedDeadLetterPath()) .build()); } @@ -208,20 +207,17 @@ public PCollection read(Pipeline pipeline, String schema) { } public PDone write(PCollection input, Schema schema) { - if (options.getOutputGcsFileFormat() == FORMAT.JSON) { + if (options.getOutputFileFormat() == FORMAT.JSON) { PCollection jsons = input.apply("RowsToJSON", ToJson.of()); if (jsons.isBounded() == IsBounded.BOUNDED) { - return jsons.apply("WriteToGCS", TextIO.write().to(options.getOutputGcsDirectory())); + return jsons.apply("WriteToFS", TextIO.write().to(options.getOutputDirectory())); } else { return jsons.apply( - "WriteToGCS", - TextIO.write() - .withWindowedWrites() - .withNumShards(1) - .to(options.getOutputGcsDirectory())); + "WriteToFS", + TextIO.write().withWindowedWrites().withNumShards(1).to(options.getOutputDirectory())); } - } else if (options.getOutputGcsFileFormat() == FORMAT.CSV) { + } else if (options.getOutputFileFormat() == FORMAT.CSV) { String header = String.join(options.getCsvDelimiter(), schema.getFieldNames()); String csvDelimiter = options.getCsvDelimiter(); @@ -233,14 +229,14 @@ public PDone write(PCollection input, Schema schema) { if (csvs.isBounded() == IsBounded.BOUNDED) { return csvs.apply( - "WriteToGCS", TextIO.write().to(options.getOutputGcsDirectory()).withHeader(header)); + "WriteToFS", TextIO.write().to(options.getOutputDirectory()).withHeader(header)); } else { return csvs.apply( - "WriteToGCS", + "WriteToFS", TextIO.write() .withWindowedWrites() .withNumShards(1) - .to(options.getOutputGcsDirectory()) + .to(options.getOutputDirectory()) .withHeader(header)); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java index 86338cb8e3d2..e8dd24eabeee 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java @@ -24,25 +24,19 @@ import java.io.IOException; import java.io.InputStream; import java.io.Reader; -import java.lang.reflect.Type; import java.nio.channels.Channels; import java.nio.channels.ReadableByteChannel; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; 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.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.Schema.Field; -import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.Gson; -import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.reflect.TypeToken; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.CharStreams; import org.slf4j.Logger; @@ -121,25 +115,6 @@ public static byte[] readGcsFile(String gcsFilePath) throws IOException { } } - public Map getDataElementsToTokenize(String payloadConfigGcsPath) { - Map dataElements; - try { - String rawJsonWithDataElements = - new String(readGcsFile(payloadConfigGcsPath), StandardCharsets.UTF_8); - Gson gson = new Gson(); - Type type = new TypeToken>() {}.getType(); - dataElements = gson.fromJson(rawJsonWithDataElements, type); - } catch (IOException | NullPointerException exception) { - LOG.error( - "Cant parse fields to tokenize, or input parameter payloadConfigGcsPath was not specified." - + " All fields will be sent to the protectors"); - dataElements = - this.getBeamSchema().getFields().stream() - .collect(Collectors.toMap(Field::getName, e -> "")); - } - return dataElements; - } - public Schema getBeamSchema() { return beamSchema; } diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java index 513d9234fdfa..276ece3d5ca5 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java @@ -28,8 +28,8 @@ import java.util.LinkedList; import java.util.List; import org.apache.beam.examples.complete.datatokenization.options.DataTokenizationOptions; -import org.apache.beam.examples.complete.datatokenization.transforms.io.GcsIO; -import org.apache.beam.examples.complete.datatokenization.transforms.io.GcsIO.FORMAT; +import org.apache.beam.examples.complete.datatokenization.transforms.io.FileSystemIO; +import org.apache.beam.examples.complete.datatokenization.transforms.io.FileSystemIO.FORMAT; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; import org.apache.beam.examples.complete.datatokenization.utils.RowToCsv; import org.apache.beam.examples.complete.datatokenization.utils.SchemasUtils; @@ -53,9 +53,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** - * Test class for {@link DataTokenization}. - */ +/** Test class for {@link DataTokenization}. */ @RunWith(JUnit4.class) public class DataTokenizationTest { @@ -63,8 +61,7 @@ public class DataTokenizationTest { "{\"fields\":[{\"mode\":\"REQUIRED\",\"name\":\"FieldName1\",\"type\":\"STRING\"},{\"mode\":\"REQUIRED\",\"name\":\"FieldName2\",\"type\":\"STRING\"}]}"; String[] fields = {"TestValue1", "TestValue2"}; - @Rule - public final transient TestPipeline testPipeline = TestPipeline.create(); + @Rule public final transient TestPipeline testPipeline = TestPipeline.create(); private static final String RESOURCES_DIR = "./"; @@ -119,22 +116,22 @@ public void testRowToCSV() { } @Test - public void testGcsIOReadCSV() throws IOException { - PCollection jsons = gcsIORead(CSV_FILE_PATH, FORMAT.CSV); + public void testFileSystemIOReadCSV() throws IOException { + PCollection jsons = fileSystemIORead(CSV_FILE_PATH, FORMAT.CSV); assertField(jsons); testPipeline.run(); } @Test - public void testGcsIOReadJSON() throws IOException { - PCollection jsons = gcsIORead(JSON_FILE_PATH, FORMAT.JSON); + public void testFileSystemIOReadJSON() throws IOException { + PCollection jsons = fileSystemIORead(JSON_FILE_PATH, FORMAT.JSON); assertField(jsons); testPipeline.run(); } @Test public void testJsonToRow() throws IOException { - PCollection jsons = gcsIORead(JSON_FILE_PATH, FORMAT.JSON); + PCollection jsons = fileSystemIORead(JSON_FILE_PATH, FORMAT.JSON); SchemasUtils testSchemaUtils = new SchemasUtils(SCHEMA_FILE_PATH, StandardCharsets.UTF_8); JsonToRow.ParseResult rows = jsons.apply( @@ -158,19 +155,19 @@ public void testJsonToRow() throws IOException { testPipeline.run(); } - private PCollection gcsIORead(String inputGcsFilePattern, FORMAT inputGcsFileFormat) - throws IOException { + private PCollection fileSystemIORead( + String inputGcsFilePattern, FORMAT inputGcsFileFormat) throws IOException { DataTokenizationOptions options = PipelineOptionsFactory.create().as(DataTokenizationOptions.class); - options.setDataSchemaGcsPath(SCHEMA_FILE_PATH); - options.setInputGcsFilePattern(inputGcsFilePattern); - options.setInputGcsFileFormat(inputGcsFileFormat); + options.setDataSchemaPath(SCHEMA_FILE_PATH); + options.setInputFilePattern(inputGcsFilePattern); + options.setInputFileFormat(inputGcsFileFormat); if (inputGcsFileFormat == FORMAT.CSV) { options.setCsvContainsHeaders(Boolean.FALSE); } SchemasUtils testSchemaUtils = - new SchemasUtils(options.getDataSchemaGcsPath(), StandardCharsets.UTF_8); + new SchemasUtils(options.getDataSchemaPath(), StandardCharsets.UTF_8); CoderRegistry coderRegistry = testPipeline.getCoderRegistry(); coderRegistry.registerCoderForType( @@ -187,7 +184,7 @@ private PCollection gcsIORead(String inputGcsFilePattern, FORMAT inputGc RowCoder.of(testSchemaUtils.getBeamSchema())); coderRegistry.registerCoderForType(coder.getEncodedTypeDescriptor(), coder); - return new GcsIO(options).read(testPipeline, testSchemaUtils.getJsonBeamSchema()); + return new FileSystemIO(options).read(testPipeline, testSchemaUtils.getJsonBeamSchema()); } private void assertField(PCollection jsons) { From 4b59ed9873ad6777e8ef983e74c362cd95e1ea5c Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Tue, 19 Jan 2021 15:38:40 +0300 Subject: [PATCH 16/29] Updated README.md for local run with BigQuery sink --- .../beam/examples/complete/datatokenization/README.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md index a454c80fbf49..72378afef6f1 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md @@ -119,8 +119,10 @@ To execute this pipeline, specify the parameters: - Ns (for seconds, example: 5s), - Nm (for minutes, example: 12m), - Nh (for hours, example: 2h). - - Google Cloud BigQuery - - **bigQueryTableName**: Cloud BigQuery table name to write into + - Google Cloud BigQuery + - **bigQueryTableName**: Cloud BigQuery table name to write into + - **tempLocation**: Folder in a Google Cloud Storage bucket, which is needed for + BigQuery to handle data writing - Cloud BigTable - **bigTableProjectId**: Id of the project where the Cloud BigTable instance to write into is located From 2e0efe0982b4c78bdf4f3915db0ca26d9d702dd8 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Wed, 20 Jan 2021 19:12:32 +0300 Subject: [PATCH 17/29] add docstring --- .../datatokenization/transforms/DataProtectors.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java index 245e99c84459..9589345cebb5 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java @@ -72,7 +72,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** TODO: Add javadoc. */ +/** + * The {@link DataProtectors} Using passing parameters transform will buffer input rows in batch and + * will send it when the count of buffered rows will equal specified batch size. When it takes the + * last one batch, it will send it when the last row will come to doFn even count of buffered rows + * will less than the batch size. + */ public class DataProtectors { /** Logger for class. */ @@ -135,7 +140,7 @@ public abstract static class Builder { } } - /** Class for data tokenization. */ + /** Class implements stateful doFn for data tokenization using remote RPC. */ @SuppressWarnings("initialization.static.fields.uninitialized") public static class TokenizationFn extends DoFn, Row> { From 31362496bfd0c69978a1ab38dbc9154cbbc8cb43 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Fri, 22 Jan 2021 14:15:07 +0300 Subject: [PATCH 18/29] Updated README.md --- .../examples/complete/datatokenization/README.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md index 72378afef6f1..119c0a2a1bc1 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md @@ -144,14 +144,14 @@ The template allows for the user to supply the following optional parameter: in the following format: ```bash ---dataSchemaGcsPath="path-to-data-schema-in-json-format" ---inputGcsFilePattern="path-patternt-to-input-data" ---outputGcsDirectory="path-to-output-directory" +--dataSchemaPath="path-to-data-schema-in-json-format" +--inputFilePattern="path-pattern-to-input-data" +--outputDirectory="path-to-output-directory" # example to usage csv ---inputGcsFileFormat="CSV" ---outputGcsFileFormat="CSV" +--inputFileFormat="CSV" +--outputFileFormat="CSV" --csvContainsHeaders="true" ---nonTokenizedDeadLetterGcsPath="path-to-errors-rows-writing" +--nonTokenizedDeadLetterPath="path-to-errors-rows-writing" --batchSize=batch-size-number --rpcUri=http://host:port/tokenize ``` From 606ecb272564223d634ac43f9e7e9c14134b43ad Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Mon, 25 Jan 2021 18:16:57 +0300 Subject: [PATCH 19/29] Updated README.md and added javadoc for the main pipeline class --- .../datatokenization/DataTokenization.java | 116 +++++++++++++++++- .../complete/datatokenization/README.md | 10 +- 2 files changed, 119 insertions(+), 7 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java index eb0e56e17ee7..393be07c3245 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java @@ -58,7 +58,121 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** The {@link DataTokenization} pipeline. */ +/** + * The {@link DataTokenization} pipeline reads data from one of the supported sources, + * tokenizes data with external API calls to some tokenization server, and writes data + * into one of the supported sinks. + *
+ *

Pipeline Requirements + *

    + *
  • Java 8
  • + *
  • Data schema (JSON with an array of fields described in BigQuery format)
  • + *
  • 1 of supported sources to read data from
  • + * + *
  • 1 of supported destination sinks to write data into
  • + * + *
  • A configured tokenization server
  • + *
+ * + *

Example Usage + *

+ * Gradle Preparation
+ * To run this example your  build.gradle file should contain the following task
+ * to execute the pipeline:
+ *   {@code
+ *   task execute (type:JavaExec) {
+ *      main = System.getProperty("mainClass")
+ *      classpath = sourceSets.main.runtimeClasspath
+ *      systemProperties System.getProperties()
+ *      args System.getProperty("exec.args", "").split()
+ *   }
+ *   }
+ * This task allows to run the pipeline via the following command:
+ *   {@code
+ *   gradle clean execute -DmainClass=org.apache.beam.examples.complete.datatokenization.DataTokenization \
+ *        -Dexec.args="--= --="
+ *   }
+ * Running the pipeline
+ * To execute this pipeline, specify the parameters:
+ *
+ * - Data schema
+ *     - dataSchemaPath: Path to data schema (JSON format) compatible with BigQuery.
+ * - 1 specified input source out of these:
+ *     - File System
+ *         - inputFilePattern: Filepattern for files to read data from
+ *         - inputFileFormat: File format of input files. Supported formats: JSON, CSV
+ *         - In case if input data is in CSV format:
+ *             - csvContainsHeaders: `true` if file(s) in bucket to read data from contain headers,
+ *               and `false` otherwise
+ *             - csvDelimiter: Delimiting character in CSV. Default: use delimiter provided in
+ *               csvFormat
+ *             - csvFormat: Csv format according to Apache Commons CSV format. Default is:
+ *               [Apache Commons CSV default](https://static.javadoc.io/org.apache.commons/commons-csv/1.7/org/apache/commons/csv/CSVFormat.html#DEFAULT)
+ *               . Must match format names exactly found
+ *               at: https://static.javadoc.io/org.apache.commons/commons-csv/1.7/org/apache/commons/csv/CSVFormat.Predefined.html
+ *     - Google Pub/Sub
+ *         - pubsubTopic: The Cloud Pub/Sub topic to read from, in the format of '
+ *           projects/yourproject/topics/yourtopic'
+ * - 1 specified output sink out of these:
+ *     - File System
+ *         - outputDirectory: Directory to write data to
+ *         - outputFileFormat: File format of output files. Supported formats: JSON, CSV
+ *         - windowDuration: The window duration in which data will be written. Should be specified
+ *           only for 'Pub/Sub -> FileSystem' case. Defaults to 30s.
+ *
+ *           Allowed formats are:
+ *             - Ns (for seconds, example: 5s),
+ *             - Nm (for minutes, example: 12m),
+ *             - Nh (for hours, example: 2h).
+ *     - Google Cloud BigQuery
+ *         - bigQueryTableName: Cloud BigQuery table name to write into
+ *         - tempLocation: Folder in a Google Cloud Storage bucket, which is needed for
+ *           BigQuery to handle data writing
+ *     - Cloud BigTable
+ *         - bigTableProjectId: Id of the project where the Cloud BigTable instance to write into
+ *           is located
+ *         - bigTableInstanceId: Id of the Cloud BigTable instance to write into
+ *         - bigTableTableId: Id of the Cloud BigTable table to write into
+ *         - bigTableKeyColumnName: Column name to use as a key in Cloud BigTable
+ *         - bigTableColumnFamilyName: Column family name to use in Cloud BigTable
+ * - RPC server parameters
+ *     - rpcUri: URI for the API calls to RPC server
+ *     - batchSize: Size of the batch to send to RPC server per request
+ *
+ * The template allows for the user to supply the following optional parameter:
+ *
+ * - nonTokenizedDeadLetterPath: Folder where failed to tokenize data will be stored
+ *
+ *
+ * Specify the parameters in the following format:
+ *
+ * {@code
+ * --dataSchemaPath="path-to-data-schema-in-json-format"
+ * --inputFilePattern="path-pattern-to-input-data"
+ * --outputDirectory="path-to-output-directory"
+ * # example for CSV case
+ * --inputFileFormat="CSV"
+ * --outputFileFormat="CSV"
+ * --csvContainsHeaders="true"
+ * --nonTokenizedDeadLetterPath="path-to-errors-rows-writing"
+ * --batchSize=batch-size-number
+ * --rpcUri=http://host:port/tokenize
+ * }
+ *
+ * By default, this will run the pipeline locally with the DirectRunner. To change the runner, specify:
+ *
+ * {@code
+ * --runner=YOUR_SELECTED_RUNNER
+ * }
+ * 
+ */ public class DataTokenization { /** Logger for class. */ diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md index 119c0a2a1bc1..d34721a4598a 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md @@ -130,11 +130,9 @@ To execute this pipeline, specify the parameters: - **bigTableTableId**: Id of the Cloud BigTable table to write into - **bigTableKeyColumnName**: Column name to use as a key in Cloud BigTable - **bigTableColumnFamilyName**: Column family name to use in Cloud BigTable -- DSG parameters - - **rpcUri**: URI for the API calls to DSG - - **batchSize**: Size of the batch to send to DSG per request - - **payloadConfigPath**: Path to the payload configuration file with an array of fields to - extract for tokenization +- RPC server parameters + - **rpcUri**: URI for the API calls to RPC server + - **batchSize**: Size of the batch to send to RPC server per request The template allows for the user to supply the following optional parameter: @@ -147,7 +145,7 @@ in the following format: --dataSchemaPath="path-to-data-schema-in-json-format" --inputFilePattern="path-pattern-to-input-data" --outputDirectory="path-to-output-directory" -# example to usage csv +# example for CSV case --inputFileFormat="CSV" --outputFileFormat="CSV" --csvContainsHeaders="true" From 2b302404dccec67a0fd4c9651aa1b7ae968f7473 Mon Sep 17 00:00:00 2001 From: daria-malkova Date: Wed, 27 Jan 2021 13:27:41 +0300 Subject: [PATCH 20/29] remove unused test case --- .../datatokenization/DataTokenizationTest.java | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java index 276ece3d5ca5..883417b31b74 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java @@ -95,17 +95,6 @@ public void testGetBigQuerySchema() { Assert.assertEquals(testSchema, schemasUtils.getBigQuerySchema().toString()); } - // @Test - // public void testNullRowToCSV() { - // Schema beamSchema = Schema.builder().addNullableField("FieldString", - // FieldType.STRING).build(); - // Row.Builder rowBuilder = Row.withSchema(beamSchema); - // Row row = rowBuilder.addValue(null).build(); - // System.out.println(row); - // String csv = new RowToCsv(",").getCsvFromRow(row); - // Assert.assertEquals("null", csv); - // } - @Test public void testRowToCSV() { Schema beamSchema = new SchemasUtils(testSchema).getBeamSchema(); From 3002f7c6b6513fc4780478b8abbeea74ecd4eb0d Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Tue, 16 Feb 2021 19:01:23 +0300 Subject: [PATCH 21/29] Style fix --- .../datatokenization/DataTokenization.java | 38 ++++++++++--------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java index 393be07c3245..6c6690f1eb35 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java @@ -59,29 +59,31 @@ import org.slf4j.LoggerFactory; /** - * The {@link DataTokenization} pipeline reads data from one of the supported sources, - * tokenizes data with external API calls to some tokenization server, and writes data - * into one of the supported sinks. - *
+ * The {@link DataTokenization} pipeline reads data from one of the supported sources, tokenizes + * data with external API calls to some tokenization server, and writes data into one of the + * supported sinks.
+ * *

Pipeline Requirements + * *

    - *
  • Java 8
  • - *
  • Data schema (JSON with an array of fields described in BigQuery format)
  • - *
  • 1 of supported sources to read data from
  • - * - *
  • 1 of supported destination sinks to write data into
  • - * - *
  • A configured tokenization server
  • + *
  • Java 8 + *
  • Data schema (JSON with an array of fields described in BigQuery format) + *
  • 1 of supported sources to read data from + * + *
  • 1 of supported destination sinks to write data into + * + *
  • A configured tokenization server *
* *

Example Usage + * *

  * Gradle Preparation
  * To run this example your  build.gradle file should contain the following task

From 7b7664e0f134e3386fd3c8283e44073b32efd670 Mon Sep 17 00:00:00 2001
From: Artur Khanin 
Date: Tue, 16 Feb 2021 20:12:56 +0300
Subject: [PATCH 22/29] Whitespaces fix

---
 .../org/apache/beam/examples/complete/datatokenization/README.md | 1 -
 1 file changed, 1 deletion(-)

diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md
index d34721a4598a..9e38b53b6a55 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md
@@ -64,7 +64,6 @@ This section describes what is needed to get the template up and running.
     - Build Data Tokenization Dataflow Flex Template
     - Creating the Dataflow Flex Template
     - Executing Template
-    
 
 ## Gradle preparation
 

From aeb471e455fbdf20bb100db71ad0a98d4deb0b75 Mon Sep 17 00:00:00 2001
From: Artur Khanin 
Date: Wed, 17 Feb 2021 12:57:28 +0300
Subject: [PATCH 23/29] Fixed undeclared dependencies and excluded .csv
 resource files from license analysis

---
 build.gradle.kts           | 1 +
 examples/java/build.gradle | 5 ++++-
 2 files changed, 5 insertions(+), 1 deletion(-)

diff --git a/build.gradle.kts b/build.gradle.kts
index c2af480ea39a..afbe9d4743e9 100644
--- a/build.gradle.kts
+++ b/build.gradle.kts
@@ -50,6 +50,7 @@ tasks.rat {
     "**/test.avsc",
     "**/user.avsc",
     "**/test/resources/**/*.txt",
+    "**/test/resources/**/*.csv",
     "**/test/**/.placeholder",
 
     // Default eclipse excludes neglect subprojects
diff --git a/examples/java/build.gradle b/examples/java/build.gradle
index e95f038f3c64..c14573ad1633 100644
--- a/examples/java/build.gradle
+++ b/examples/java/build.gradle
@@ -72,10 +72,12 @@ dependencies {
   compile library.java.google_code_gson
   compile library.java.google_http_client
   compile library.java.google_oauth_client
+  compile library.java.jackson_databind
   compile library.java.joda_time
+  compile library.java.protobuf_java
+  compile library.java.proto_google_cloud_bigtable_v2
   compile library.java.proto_google_cloud_datastore_v1
   compile library.java.slf4j_api
-  compile library.java.slf4j_jdk14
   provided library.java.commons_io
   provided library.java.commons_csv
   runtime project(path: ":runners:direct-java", configuration: "shadow")
@@ -87,6 +89,7 @@ dependencies {
     // "spotbugs-annotations:3.1.12" used in Beam. Not required.
     exclude group: "org.apache.zookeeper", module: "zookeeper"
   }
+  compile "org.apache.commons:commons-lang3:3.9"
   compile "org.apache.httpcomponents:httpclient:4.5.13"
   compile "org.apache.httpcomponents:httpcore:4.4.13"
   testCompile project(path: ":runners:direct-java", configuration: "shadow")

From a9238463e5d6ba9d15afdc4db1df7b90f01dd220 Mon Sep 17 00:00:00 2001
From: Mikhail Medvedev 
Date: Thu, 18 Mar 2021 18:21:50 +0400
Subject: [PATCH 24/29] Fix for incorrect rpc url

---
 .../examples/complete/datatokenization/DataTokenization.java | 3 ++-
 .../complete/datatokenization/transforms/DataProtectors.java | 5 +++++
 2 files changed, 7 insertions(+), 1 deletion(-)

diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java
index 6c6690f1eb35..355140c97fc7 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java
@@ -55,6 +55,7 @@
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -275,7 +276,7 @@ public static PipelineResult run(DataTokenizationOptions options) {
                   .via(
                       (Row errRow) ->
                           FailsafeElement.of(errRow.getString("line"), errRow.getString("line"))
-                              .setErrorMessage(errRow.getString("err"))))
+                              .setErrorMessage(Strings.nullToEmpty(errRow.getString("err")))))
           .apply(
               "WriteCsvConversionErrorsToFS",
               ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder()
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java
index 9589345cebb5..f946876a06b2 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java
@@ -63,6 +63,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables;
 import org.apache.commons.io.IOUtils;
 import org.apache.http.HttpEntity;
+import org.apache.http.HttpStatus;
 import org.apache.http.client.methods.CloseableHttpResponse;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.entity.ByteArrayEntity;
@@ -291,6 +292,10 @@ private ArrayList getTokenizedRow(Iterable inputRows) throws IOExcepti
       CloseableHttpResponse response =
           sendRpc(formatJsonsToRpcBatch(rowsToJsons(inputRows)).getBytes(Charset.defaultCharset()));
 
+      if (response.getStatusLine().getStatusCode() != HttpStatus.SC_OK) {
+        LOG.error("Send to RPC '{}' failed with '{}'", this.rpcURI, response.getStatusLine());
+      }
+
       String tokenizedData =
           IOUtils.toString(response.getEntity().getContent(), StandardCharsets.UTF_8);
 

From 5a5d0c4c4b19aa61f46ae0e4bf209b0cce08339d Mon Sep 17 00:00:00 2001
From: Mikhail Medvedev 
Date: Thu, 18 Mar 2021 19:37:30 +0400
Subject: [PATCH 25/29] Fix for nullable types

---
 .../examples/complete/datatokenization/DataTokenization.java  | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java
index 355140c97fc7..5fcb0718c617 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java
@@ -275,7 +275,9 @@ public static PipelineResult run(DataTokenizationOptions options) {
               MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor())
                   .via(
                       (Row errRow) ->
-                          FailsafeElement.of(errRow.getString("line"), errRow.getString("line"))
+                          FailsafeElement.of(
+                                  Strings.nullToEmpty(errRow.getString("line")),
+                                  Strings.nullToEmpty(errRow.getString("line")))
                               .setErrorMessage(Strings.nullToEmpty(errRow.getString("err")))))
           .apply(
               "WriteCsvConversionErrorsToFS",

From 6ac4193e224735bd6cfd2e5288a092bc61dcb161 Mon Sep 17 00:00:00 2001
From: Nuzhdina-Elena <79855159+Nuzhdina-Elena@users.noreply.github.com>
Date: Mon, 22 Mar 2021 11:18:41 +0300
Subject: [PATCH 26/29] Data tokenization example group into batches (#11)

* GroupIntoBatches was used in the data tokenization pipeline

* io files were renamed for the data tokenization template
---
 .../datatokenization/DataTokenization.java    |  73 ++----
 .../options/DataTokenizationOptions.java      |   6 +-
 .../transforms/DataProtectors.java            |  80 ++----
 .../transforms/JsonToBeamRow.java             |  78 ++++++
 .../transforms/SerializableFunctions.java     |  49 ++++
 ...eryIO.java => TokenizationBigQueryIO.java} |   8 +-
 ...bleIO.java => TokenizationBigTableIO.java} |   8 +-
 ...mIO.java => TokenizationFileSystemIO.java} | 229 +++++++++---------
 .../utils/ErrorConverters.java                |  90 +++++++
 .../DataTokenizationTest.java                 |  41 ++--
 10 files changed, 414 insertions(+), 248 deletions(-)
 create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/JsonToBeamRow.java
 create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/SerializableFunctions.java
 rename examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/{BigQueryIO.java => TokenizationBigQueryIO.java} (92%)
 rename examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/{BigTableIO.java => TokenizationBigTableIO.java} (95%)
 rename examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/{FileSystemIO.java => TokenizationFileSystemIO.java} (51%)

diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java
index 5fcb0718c617..bbe3759a8ff6 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/DataTokenization.java
@@ -25,9 +25,11 @@
 import java.nio.charset.StandardCharsets;
 import org.apache.beam.examples.complete.datatokenization.options.DataTokenizationOptions;
 import org.apache.beam.examples.complete.datatokenization.transforms.DataProtectors.RowToTokenizedRow;
-import org.apache.beam.examples.complete.datatokenization.transforms.io.BigQueryIO;
-import org.apache.beam.examples.complete.datatokenization.transforms.io.BigTableIO;
-import org.apache.beam.examples.complete.datatokenization.transforms.io.FileSystemIO;
+import org.apache.beam.examples.complete.datatokenization.transforms.JsonToBeamRow;
+import org.apache.beam.examples.complete.datatokenization.transforms.SerializableFunctions;
+import org.apache.beam.examples.complete.datatokenization.transforms.io.TokenizationBigQueryIO;
+import org.apache.beam.examples.complete.datatokenization.transforms.io.TokenizationBigTableIO;
+import org.apache.beam.examples.complete.datatokenization.transforms.io.TokenizationFileSystemIO;
 import org.apache.beam.examples.complete.datatokenization.utils.ErrorConverters;
 import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement;
 import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder;
@@ -45,7 +47,6 @@
 import org.apache.beam.sdk.io.gcp.bigquery.WriteResult;
 import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.JsonToRow;
 import org.apache.beam.sdk.transforms.MapElements;
 import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
@@ -55,7 +56,6 @@
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TypeDescriptors;
-import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -182,7 +182,7 @@ public class DataTokenization {
   private static final Logger LOG = LoggerFactory.getLogger(DataTokenization.class);
 
   /** String/String Coder for FailsafeElement. */
-  private static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER =
+  public static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER =
       FailsafeElementCoder.of(
           NullableCoder.of(StringUtf8Coder.of()), NullableCoder.of(StringUtf8Coder.of()));
 
@@ -244,54 +244,31 @@ public static PipelineResult run(DataTokenizationOptions options) {
 
     coderRegistry.registerCoderForType(coder.getEncodedTypeDescriptor(), coder);
 
-    PCollection jsons;
+    PCollection rows;
     if (options.getInputFilePattern() != null) {
-      jsons = new FileSystemIO(options).read(pipeline, schema.getJsonBeamSchema());
+      rows = new TokenizationFileSystemIO(options).read(pipeline, schema);
     } else if (options.getPubsubTopic() != null) {
-      jsons =
-          pipeline.apply(
-              "ReadMessagesFromPubsub", PubsubIO.readStrings().fromTopic(options.getPubsubTopic()));
+      rows =
+          pipeline
+              .apply(
+                  "ReadMessagesFromPubsub",
+                  PubsubIO.readStrings().fromTopic(options.getPubsubTopic()))
+              .apply(
+                  "TransformToBeamRow",
+                  new JsonToBeamRow(options.getNonTokenizedDeadLetterPath(), schema));
       if (options.getOutputDirectory() != null) {
-        jsons =
-            jsons.apply(Window.into(FixedWindows.of(parseDuration(options.getWindowDuration()))));
+        rows = rows.apply(Window.into(FixedWindows.of(parseDuration(options.getWindowDuration()))));
       }
     } else {
       throw new IllegalStateException(
           "No source is provided, please configure File System or Pub/Sub");
     }
 
-    JsonToRow.ParseResult rows =
-        jsons.apply(
-            "JsonToRow",
-            JsonToRow.withExceptionReporting(schema.getBeamSchema()).withExtendedErrorInfo());
-
-    if (options.getNonTokenizedDeadLetterPath() != null) {
-      /*
-       * Write Row conversion errors to filesystem specified path
-       */
-      rows.getFailedToParseLines()
-          .apply(
-              "ToFailsafeElement",
-              MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor())
-                  .via(
-                      (Row errRow) ->
-                          FailsafeElement.of(
-                                  Strings.nullToEmpty(errRow.getString("line")),
-                                  Strings.nullToEmpty(errRow.getString("line")))
-                              .setErrorMessage(Strings.nullToEmpty(errRow.getString("err")))))
-          .apply(
-              "WriteCsvConversionErrorsToFS",
-              ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder()
-                  .setCsvDelimiter(options.getCsvDelimiter())
-                  .setErrorWritePath(options.getNonTokenizedDeadLetterPath())
-                  .build());
-    }
     /*
     Tokenize data using remote API call
      */
     PCollectionTuple tokenizedRows =
-        rows.getResults()
-            .setRowSchema(schema.getBeamSchema())
+        rows.setRowSchema(schema.getBeamSchema())
             .apply(
                 MapElements.into(
                         TypeDescriptors.kvs(TypeDescriptors.integers(), TypeDescriptors.rows()))
@@ -324,17 +301,18 @@ public static PipelineResult run(DataTokenizationOptions options) {
                               new RowToCsv(csvDelimiter).getCsvFromRow(fse.getPayload()))))
           .apply(
               "WriteTokenizationErrorsToFS",
-              ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder()
-                  .setCsvDelimiter(options.getCsvDelimiter())
+              ErrorConverters.WriteErrorsToTextIO.newBuilder()
                   .setErrorWritePath(options.getNonTokenizedDeadLetterPath())
+                  .setTranslateFunction(SerializableFunctions.getCsvErrorConverter())
                   .build());
     }
 
     if (options.getOutputDirectory() != null) {
-      new FileSystemIO(options).write(tokenizedRows.get(TOKENIZATION_OUT), schema.getBeamSchema());
+      new TokenizationFileSystemIO(options)
+          .write(tokenizedRows.get(TOKENIZATION_OUT), schema.getBeamSchema());
     } else if (options.getBigQueryTableName() != null) {
       WriteResult writeResult =
-          BigQueryIO.write(
+          TokenizationBigQueryIO.write(
               tokenizedRows.get(TOKENIZATION_OUT),
               options.getBigQueryTableName(),
               schema.getBigQuerySchema());
@@ -343,7 +321,7 @@ public static PipelineResult run(DataTokenizationOptions options) {
           .apply(
               "WrapInsertionErrors",
               MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor())
-                  .via(BigQueryIO::wrapBigQueryInsertError))
+                  .via(TokenizationBigQueryIO::wrapBigQueryInsertError))
           .setCoder(FAILSAFE_ELEMENT_CODER)
           .apply(
               "WriteInsertionFailedRecords",
@@ -353,7 +331,8 @@ public static PipelineResult run(DataTokenizationOptions options) {
                   .setErrorRecordsTableSchema(DEADLETTER_SCHEMA)
                   .build());
     } else if (options.getBigTableInstanceId() != null) {
-      new BigTableIO(options).write(tokenizedRows.get(TOKENIZATION_OUT), schema.getBeamSchema());
+      new TokenizationBigTableIO(options)
+          .write(tokenizedRows.get(TOKENIZATION_OUT), schema.getBeamSchema());
     } else {
       throw new IllegalStateException(
           "No sink is provided, please configure BigQuery or BigTable.");
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/DataTokenizationOptions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/DataTokenizationOptions.java
index 767ca8644d3d..c68a5378e487 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/DataTokenizationOptions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/options/DataTokenizationOptions.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.examples.complete.datatokenization.options;
 
-import org.apache.beam.examples.complete.datatokenization.transforms.io.BigTableIO;
-import org.apache.beam.examples.complete.datatokenization.transforms.io.FileSystemIO.FileSystemPipelineOptions;
+import org.apache.beam.examples.complete.datatokenization.transforms.io.TokenizationBigTableIO;
+import org.apache.beam.examples.complete.datatokenization.transforms.io.TokenizationFileSystemIO.FileSystemPipelineOptions;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -28,7 +28,7 @@
  * executor at the command-line.
  */
 public interface DataTokenizationOptions
-    extends PipelineOptions, FileSystemPipelineOptions, BigTableIO.BigTableOptions {
+    extends PipelineOptions, FileSystemPipelineOptions, TokenizationBigTableIO.BigTableOptions {
 
   @Description("Path to data schema (JSON format) compatible with BigQuery.")
   String getDataSchemaPath();
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java
index f946876a06b2..6de4144c7ded 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java
@@ -18,7 +18,6 @@
 package org.apache.beam.examples.complete.datatokenization.transforms;
 
 import static org.apache.beam.sdk.util.RowJsonUtils.rowToJson;
-import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects.firstNonNull;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.api.services.bigquery.model.TableRow;
@@ -37,18 +36,10 @@
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.Field;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
-import org.apache.beam.sdk.state.BagState;
-import org.apache.beam.sdk.state.StateSpec;
-import org.apache.beam.sdk.state.StateSpecs;
-import org.apache.beam.sdk.state.TimeDomain;
-import org.apache.beam.sdk.state.Timer;
-import org.apache.beam.sdk.state.TimerSpec;
-import org.apache.beam.sdk.state.TimerSpecs;
-import org.apache.beam.sdk.state.ValueState;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupIntoBatches;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.RowJson;
 import org.apache.beam.sdk.util.RowJsonUtils;
 import org.apache.beam.sdk.values.KV;
@@ -70,6 +61,7 @@
 import org.apache.http.entity.ContentType;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.HttpClients;
+import org.joda.time.Duration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -85,6 +77,7 @@ public class DataProtectors {
   private static final Logger LOG = LoggerFactory.getLogger(DataProtectors.class);
 
   public static final String ID_FIELD_NAME = "ID";
+  private static final Integer MAX_BUFFERING = 100;
 
   /**
    * The {@link RowToTokenizedRow} transform converts {@link Row} to {@link TableRow} objects. The
@@ -113,11 +106,18 @@ public static  Builder newBuilder() {
     public PCollectionTuple expand(PCollection> inputRows) {
       FailsafeElementCoder coder =
           FailsafeElementCoder.of(RowCoder.of(schema()), RowCoder.of(schema()));
+
+      Duration maxBuffering = Duration.millis(MAX_BUFFERING);
       PCollectionTuple pCollectionTuple =
-          inputRows.apply(
-              "Tokenize",
-              ParDo.of(new TokenizationFn(schema(), batchSize(), rpcURI(), failureTag()))
-                  .withOutputTags(successTag(), TupleTagList.of(failureTag())));
+          inputRows
+              .apply("GroupRowsIntoBatches",
+                  GroupIntoBatches.ofSize(batchSize())
+                      .withMaxBufferingDuration(maxBuffering))
+              .apply(
+                  "Tokenize",
+                  ParDo.of(new TokenizationFn(schema(), rpcURI(), failureTag()))
+                      .withOutputTags(successTag(), TupleTagList.of(failureTag())));
+
       return PCollectionTuple.of(
               successTag(), pCollectionTuple.get(successTag()).setRowSchema(schema()))
           .and(failureTag(), pCollectionTuple.get(failureTag()).setCoder(coder));
@@ -143,7 +143,7 @@ public abstract static class Builder {
 
   /** Class implements stateful doFn for data tokenization using remote RPC. */
   @SuppressWarnings("initialization.static.fields.uninitialized")
-  public static class TokenizationFn extends DoFn, Row> {
+  public static class TokenizationFn extends DoFn>, Row> {
 
     private static Schema schemaToRpc;
     private static CloseableHttpClient httpclient;
@@ -151,30 +151,15 @@ public static class TokenizationFn extends DoFn, Row> {
     private static ObjectMapper objectMapperDeserializerForSchema;
 
     private final Schema schema;
-    private final int batchSize;
     private final String rpcURI;
     private final TupleTag> failureTag;
 
-    @StateId("buffer")
-    private final StateSpec> bufferedEvents;
-
-    @StateId("count")
-    private final StateSpec> countState = StateSpecs.value();
-
-    @TimerId("expiry")
-    private final TimerSpec expirySpec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
-
     private Map inputRowsWithIds;
 
     public TokenizationFn(
-        Schema schema,
-        int batchSize,
-        String rpcURI,
-        TupleTag> failureTag) {
+        Schema schema, String rpcURI, TupleTag> failureTag) {
       this.schema = schema;
-      this.batchSize = batchSize;
       this.rpcURI = rpcURI;
-      bufferedEvents = StateSpecs.bag(RowCoder.of(schema));
       this.failureTag = failureTag;
       this.inputRowsWithIds = new HashMap<>();
     }
@@ -207,39 +192,10 @@ public void close() {
       }
     }
 
-    @OnTimer("expiry")
-    public void onExpiry(OnTimerContext context, @StateId("buffer") BagState bufferState) {
-      boolean isEmpty = firstNonNull(bufferState.isEmpty().read(), true);
-      if (!isEmpty) {
-        processBufferedRows(bufferState.read(), context);
-        bufferState.clear();
-      }
-    }
-
     @ProcessElement
-    public void process(
-        ProcessContext context,
-        BoundedWindow window,
-        @StateId("buffer") BagState bufferState,
-        @StateId("count") ValueState countState,
-        @TimerId("expiry") Timer expiryTimer) {
-
-      expiryTimer.set(window.maxTimestamp());
-
-      int count = firstNonNull(countState.read(), 0);
-      count++;
-      countState.write(count);
-      bufferState.add(context.element().getValue());
-
-      if (count >= batchSize) {
-        processBufferedRows(bufferState.read(), context);
-        bufferState.clear();
-        countState.clear();
-      }
-    }
-
     @SuppressWarnings("argument.type.incompatible")
-    private void processBufferedRows(Iterable rows, WindowedContext context) {
+    public void process(@Element KV> element, ProcessContext context) {
+      Iterable rows = element.getValue();
 
       try {
         for (Row outputRow : getTokenizedRow(rows)) {
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/JsonToBeamRow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/JsonToBeamRow.java
new file mode 100644
index 000000000000..27af8f0294e5
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/JsonToBeamRow.java
@@ -0,0 +1,78 @@
+/*
+ * 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.examples.complete.datatokenization.transforms;
+
+import static org.apache.beam.examples.complete.datatokenization.DataTokenization.FAILSAFE_ELEMENT_CODER;
+
+import org.apache.beam.examples.complete.datatokenization.utils.ErrorConverters;
+import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement;
+import org.apache.beam.examples.complete.datatokenization.utils.SchemasUtils;
+import org.apache.beam.sdk.transforms.JsonToRow;
+import org.apache.beam.sdk.transforms.JsonToRow.ParseResult;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
+
+/**
+ * The {@link JsonToBeamRow} converts jsons string to beam rows.
+ */
+public class JsonToBeamRow extends PTransform, PCollection> {
+
+  private final String failedToParseDeadLetterPath;
+  private final transient SchemasUtils schema;
+
+  public JsonToBeamRow(String failedToParseDeadLetterPath, SchemasUtils schema) {
+    this.failedToParseDeadLetterPath = failedToParseDeadLetterPath;
+    this.schema = schema;
+  }
+
+  @Override
+  @SuppressWarnings("argument.type.incompatible")
+  public PCollection expand(PCollection jsons) {
+    ParseResult rows =
+        jsons.apply(
+            "JsonToRow",
+            JsonToRow.withExceptionReporting(schema.getBeamSchema()).withExtendedErrorInfo());
+
+    if (failedToParseDeadLetterPath != null) {
+      /*
+       * Write Row conversion errors to filesystem specified path
+       */
+      rows.getFailedToParseLines()
+          .apply(
+              "ToFailsafeElement",
+              MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor())
+                  .via(
+                      (Row errRow) ->
+                          FailsafeElement.of(
+                                  Strings.nullToEmpty(errRow.getString("line")),
+                                  Strings.nullToEmpty(errRow.getString("line")))
+                              .setErrorMessage(Strings.nullToEmpty(errRow.getString("err")))))
+          .apply(
+              "WriteCsvConversionErrorsToFS",
+              ErrorConverters.WriteErrorsToTextIO.newBuilder()
+                  .setErrorWritePath(failedToParseDeadLetterPath)
+                  .setTranslateFunction(SerializableFunctions.getCsvErrorConverter())
+                  .build());
+    }
+
+    return rows.getResults().setRowSchema(schema.getBeamSchema());
+  }
+}
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/SerializableFunctions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/SerializableFunctions.java
new file mode 100644
index 000000000000..1e5df0776e16
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/SerializableFunctions.java
@@ -0,0 +1,49 @@
+/*
+ * 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.examples.complete.datatokenization.transforms;
+
+import java.time.Instant;
+import java.util.ArrayList;
+import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+
+/** The {@link SerializableFunctions} class to store static Serializable functions. */
+public class SerializableFunctions {
+
+  private static final SerializableFunction, String>
+      csvErrorConverter =
+          (FailsafeElement failsafeElement) -> {
+            ArrayList outputRow = new ArrayList<>();
+            final String message = failsafeElement.getOriginalPayload();
+            String timestamp = Instant.now().toString();
+            outputRow.add(timestamp);
+            outputRow.add(failsafeElement.getErrorMessage());
+            outputRow.add(failsafeElement.getStacktrace());
+            // Only set the payload if it's populated on the message.
+            if (failsafeElement.getOriginalPayload() != null) {
+              outputRow.add(message);
+            }
+
+            return String.join(",", outputRow);
+          };
+
+  public static SerializableFunction, String>
+      getCsvErrorConverter() {
+    return csvErrorConverter;
+  }
+}
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/TokenizationBigQueryIO.java
similarity index 92%
rename from examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java
rename to examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/TokenizationBigQueryIO.java
index 17bcabae025e..fe8f4c1afad8 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigQueryIO.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/TokenizationBigQueryIO.java
@@ -32,11 +32,11 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/** The {@link BigQueryIO} class for writing data from template to BigTable. */
-public class BigQueryIO {
+/** The {@link TokenizationBigQueryIO} class for writing data from template to BigTable. */
+public class TokenizationBigQueryIO {
 
   /** Logger for class. */
-  private static final Logger LOG = LoggerFactory.getLogger(BigQueryIO.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TokenizationBigQueryIO.class);
 
   public static WriteResult write(
       PCollection input, String bigQueryTableName, TableSchema schema) {
@@ -77,7 +77,7 @@ public static FailsafeElement wrapBigQueryInsertError(
       failsafeElement.setErrorMessage(insertError.getError().toPrettyString());
 
     } catch (IOException e) {
-      BigQueryIO.LOG.error("Failed to wrap BigQuery insert error.");
+      TokenizationBigQueryIO.LOG.error("Failed to wrap BigQuery insert error.");
       throw new RuntimeException(e);
     }
     return failsafeElement;
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/TokenizationBigTableIO.java
similarity index 95%
rename from examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java
rename to examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/TokenizationBigTableIO.java
index 695c25770e56..d7d1c3e97232 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/BigTableIO.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/TokenizationBigTableIO.java
@@ -40,15 +40,15 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/** The {@link BigTableIO} class for writing data from template to BigTable. */
-public class BigTableIO {
+/** The {@link TokenizationBigTableIO} class for writing data from template to BigTable. */
+public class TokenizationBigTableIO {
 
   /** Logger for class. */
-  private static final Logger LOG = LoggerFactory.getLogger(BigTableIO.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TokenizationBigTableIO.class);
 
   private final DataTokenizationOptions options;
 
-  public BigTableIO(DataTokenizationOptions options) {
+  public TokenizationBigTableIO(DataTokenizationOptions options) {
     this.options = options;
   }
 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/FileSystemIO.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/TokenizationFileSystemIO.java
similarity index 51%
rename from examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/FileSystemIO.java
rename to examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/TokenizationFileSystemIO.java
index a5dfaa86674b..54805e11e2d5 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/FileSystemIO.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/io/TokenizationFileSystemIO.java
@@ -18,14 +18,14 @@
 package org.apache.beam.examples.complete.datatokenization.transforms.io;
 
 import org.apache.beam.examples.complete.datatokenization.options.DataTokenizationOptions;
+import org.apache.beam.examples.complete.datatokenization.transforms.JsonToBeamRow;
+import org.apache.beam.examples.complete.datatokenization.transforms.SerializableFunctions;
 import org.apache.beam.examples.complete.datatokenization.utils.CsvConverters;
 import org.apache.beam.examples.complete.datatokenization.utils.ErrorConverters;
 import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElement;
-import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder;
 import org.apache.beam.examples.complete.datatokenization.utils.RowToCsv;
+import org.apache.beam.examples.complete.datatokenization.utils.SchemasUtils;
 import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.NullableCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
@@ -40,11 +40,9 @@
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TypeDescriptors;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-/** The {@link FileSystemIO} class to read/write data from/into File Systems. */
-public class FileSystemIO {
+/** The {@link TokenizationFileSystemIO} class to read/write data from/into File Systems. */
+public class TokenizationFileSystemIO {
 
   /** The tag for the headers of the CSV if required. */
   static final TupleTag CSV_HEADERS = new TupleTag() {};
@@ -60,16 +58,6 @@ public class FileSystemIO {
   static final TupleTag> PROCESSING_OUT =
       new TupleTag>() {};
 
-  /* Logger for class. */
-  private static final Logger LOG = LoggerFactory.getLogger(FileSystemIO.class);
-
-  public static final String DEAD_LETTER_PREFIX = "CSV_CONVERTOR";
-
-  /** String/String Coder for FailsafeElement. */
-  private static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER =
-      FailsafeElementCoder.of(
-          NullableCoder.of(StringUtf8Coder.of()), NullableCoder.of(StringUtf8Coder.of()));
-
   /** Supported format to read from GCS. */
   public enum FORMAT {
     JSON,
@@ -88,7 +76,7 @@ public interface FileSystemPipelineOptions extends PipelineOptions {
 
     @Description("File format of input files. Supported formats: JSON, CSV")
     @Default.Enum("JSON")
-    FileSystemIO.FORMAT getInputFileFormat();
+    TokenizationFileSystemIO.FORMAT getInputFileFormat();
 
     void setInputFileFormat(FORMAT inputFileFormat);
 
@@ -99,7 +87,7 @@ public interface FileSystemPipelineOptions extends PipelineOptions {
 
     @Description("File format of output files. Supported formats: JSON, CSV")
     @Default.Enum("JSON")
-    FileSystemIO.FORMAT getOutputFileFormat();
+    TokenizationFileSystemIO.FORMAT getOutputFileFormat();
 
     void setOutputFileFormat(FORMAT outputFileFormat);
 
@@ -141,108 +129,133 @@ public interface FileSystemPipelineOptions extends PipelineOptions {
 
   private final DataTokenizationOptions options;
 
-  public FileSystemIO(DataTokenizationOptions options) {
+  public TokenizationFileSystemIO(DataTokenizationOptions options) {
     this.options = options;
   }
 
-  public PCollection read(Pipeline pipeline, String schema) {
-    if (options.getInputFileFormat() == FORMAT.JSON) {
-      return pipeline.apply("ReadJsonFromFiles", TextIO.read().from(options.getInputFilePattern()));
-    } else if (options.getInputFileFormat() == FORMAT.CSV) {
-      PCollectionTuple jsons =
-          pipeline
-              /*
-               * Step 1: Read CSV file(s) from File System using {@link CsvConverters.ReadCsv}.
-               */
-              .apply(
-                  "ReadCsvFromFiles",
-                  CsvConverters.ReadCsv.newBuilder()
-                      .setCsvFormat(options.getCsvFormat())
-                      .setDelimiter(options.getCsvDelimiter())
-                      .setHasHeaders(options.getCsvContainsHeaders())
-                      .setInputFileSpec(options.getInputFilePattern())
-                      .setHeaderTag(CSV_HEADERS)
-                      .setLineTag(CSV_LINES)
-                      .build())
-              /*
-               * Step 2: Convert lines to Json.
-               */
-              .apply(
-                  "LineToJson",
-                  CsvConverters.LineToFailsafeJson.newBuilder()
-                      .setDelimiter(options.getCsvDelimiter())
-                      .setJsonSchema(schema)
-                      .setHeaderTag(CSV_HEADERS)
-                      .setLineTag(CSV_LINES)
-                      .setUdfOutputTag(PROCESSING_OUT)
-                      .setUdfDeadletterTag(PROCESSING_DEADLETTER_OUT)
-                      .build());
-
-      if (options.getNonTokenizedDeadLetterPath() != null) {
-        /*
-         * Step 3: Write jsons to dead-letter that weren't successfully processed.
-         */
-        jsons
-            .get(PROCESSING_DEADLETTER_OUT)
-            .apply(
-                "WriteCsvConversionErrorsToFS",
-                ErrorConverters.WriteStringMessageErrorsAsCsv.newBuilder()
-                    .setCsvDelimiter(options.getCsvDelimiter())
-                    .setErrorWritePath(options.getNonTokenizedDeadLetterPath())
-                    .build());
-      }
+  public PCollection read(Pipeline pipeline, SchemasUtils schema) {
+    switch (options.getInputFileFormat()) {
+      case JSON:
+        return readJson(pipeline)
+            .apply(new JsonToBeamRow(options.getNonTokenizedDeadLetterPath(), schema));
+      case CSV:
+        return readCsv(pipeline, schema)
+            .apply(new JsonToBeamRow(options.getNonTokenizedDeadLetterPath(), schema));
+      default:
+        throw new IllegalStateException(
+            "No valid format for input data is provided. Please, choose JSON or CSV.");
+    }
+  }
 
+  private PCollection readJson(Pipeline pipeline) {
+    return pipeline.apply("ReadJsonFromFiles", TextIO.read().from(options.getInputFilePattern()));
+  }
+
+  private PCollection readCsv(Pipeline pipeline, SchemasUtils schema) {
+    /*
+     * Step 1: Read CSV file(s) from File System using {@link CsvConverters.ReadCsv}.
+     */
+    PCollectionTuple csvLines = readCsv(pipeline);
+    /*
+     * Step 2: Convert lines to Json.
+     */
+    PCollectionTuple jsons = csvLineToJson(csvLines, schema.getJsonBeamSchema());
+
+    if (options.getNonTokenizedDeadLetterPath() != null) {
       /*
-       * Step 4: Get jsons that were successfully processed.
+       * Step 3: Write jsons to dead-letter that weren't successfully processed.
        */
-      return jsons
-          .get(PROCESSING_OUT)
+      jsons
+          .get(PROCESSING_DEADLETTER_OUT)
           .apply(
-              "GetJson",
-              MapElements.into(TypeDescriptors.strings()).via(FailsafeElement::getPayload));
-    } else {
-      throw new IllegalStateException(
-          "No valid format for input data is provided. Please, choose JSON or CSV.");
+              "WriteCsvConversionErrorsToFS",
+              ErrorConverters.WriteErrorsToTextIO.newBuilder()
+                  .setErrorWritePath(options.getNonTokenizedDeadLetterPath())
+                  .setTranslateFunction(SerializableFunctions.getCsvErrorConverter())
+                  .build());
     }
+
+    /*
+     * Step 4: Get jsons that were successfully processed.
+     */
+    return jsons
+        .get(PROCESSING_OUT)
+        .apply(
+            "GetJson",
+            MapElements.into(TypeDescriptors.strings()).via(FailsafeElement::getPayload));
+  }
+
+  private PCollectionTuple readCsv(Pipeline pipeline) {
+    return pipeline.apply(
+        "ReadCsvFromFiles",
+        CsvConverters.ReadCsv.newBuilder()
+            .setCsvFormat(options.getCsvFormat())
+            .setDelimiter(options.getCsvDelimiter())
+            .setHasHeaders(options.getCsvContainsHeaders())
+            .setInputFileSpec(options.getInputFilePattern())
+            .setHeaderTag(CSV_HEADERS)
+            .setLineTag(CSV_LINES)
+            .build());
+  }
+
+  private PCollectionTuple csvLineToJson(PCollectionTuple csvLines, String jsonSchema) {
+    return csvLines.apply(
+        "LineToJson",
+        CsvConverters.LineToFailsafeJson.newBuilder()
+            .setDelimiter(options.getCsvDelimiter())
+            .setJsonSchema(jsonSchema)
+            .setHeaderTag(CSV_HEADERS)
+            .setLineTag(CSV_LINES)
+            .setUdfOutputTag(PROCESSING_OUT)
+            .setUdfDeadletterTag(PROCESSING_DEADLETTER_OUT)
+            .build());
   }
 
   public PDone write(PCollection input, Schema schema) {
-    if (options.getOutputFileFormat() == FORMAT.JSON) {
-      PCollection jsons = input.apply("RowsToJSON", ToJson.of());
-
-      if (jsons.isBounded() == IsBounded.BOUNDED) {
-        return jsons.apply("WriteToFS", TextIO.write().to(options.getOutputDirectory()));
-      } else {
-        return jsons.apply(
-            "WriteToFS",
-            TextIO.write().withWindowedWrites().withNumShards(1).to(options.getOutputDirectory()));
-      }
-    } else if (options.getOutputFileFormat() == FORMAT.CSV) {
-      String header = String.join(options.getCsvDelimiter(), schema.getFieldNames());
-      String csvDelimiter = options.getCsvDelimiter();
-
-      PCollection csvs =
-          input.apply(
-              "ConvertToCSV",
-              MapElements.into(TypeDescriptors.strings())
-                  .via((Row inputRow) -> new RowToCsv(csvDelimiter).getCsvFromRow(inputRow)));
-
-      if (csvs.isBounded() == IsBounded.BOUNDED) {
-        return csvs.apply(
-            "WriteToFS", TextIO.write().to(options.getOutputDirectory()).withHeader(header));
-      } else {
-        return csvs.apply(
-            "WriteToFS",
-            TextIO.write()
-                .withWindowedWrites()
-                .withNumShards(1)
-                .to(options.getOutputDirectory())
-                .withHeader(header));
-      }
+    switch (options.getOutputFileFormat()) {
+      case JSON:
+        return writeJson(input);
+      case CSV:
+        return writeCsv(input, schema);
+      default:
+        throw new IllegalStateException(
+            "No valid format for output data is provided. Please, choose JSON or CSV.");
+    }
+  }
+
+  private PDone writeJson(PCollection input) {
+    PCollection jsons = input.apply("RowsToJSON", ToJson.of());
+
+    if (jsons.isBounded() == IsBounded.BOUNDED) {
+      return jsons.apply("WriteToFS", TextIO.write().to(options.getOutputDirectory()));
+    } else {
+      return jsons.apply(
+          "WriteToFS",
+          TextIO.write().withWindowedWrites().withNumShards(1).to(options.getOutputDirectory()));
+    }
+  }
+
+  private PDone writeCsv(PCollection input, Schema schema) {
+    String header = String.join(options.getCsvDelimiter(), schema.getFieldNames());
+    String csvDelimiter = options.getCsvDelimiter();
+
+    PCollection csvs =
+        input.apply(
+            "ConvertToCSV",
+            MapElements.into(TypeDescriptors.strings())
+                .via((Row inputRow) -> new RowToCsv(csvDelimiter).getCsvFromRow(inputRow)));
 
+    if (csvs.isBounded() == IsBounded.BOUNDED) {
+      return csvs.apply(
+          "WriteToFS", TextIO.write().to(options.getOutputDirectory()).withHeader(header));
     } else {
-      throw new IllegalStateException(
-          "No valid format for output data is provided. Please, choose JSON or CSV.");
+      return csvs.apply(
+          "WriteToFS",
+          TextIO.write()
+              .withWindowedWrites()
+              .withNumShards(1)
+              .to(options.getOutputDirectory())
+              .withHeader(header));
     }
   }
 }
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java
index 1cba570239c1..04beac038926 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/ErrorConverters.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.examples.complete.datatokenization.utils;
 
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.auto.value.AutoValue;
 import java.nio.charset.StandardCharsets;
@@ -28,13 +30,16 @@
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
 import org.apache.beam.sdk.io.gcp.bigquery.WriteResult;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.MapElements;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.TypeDescriptors;
 import org.joda.time.DateTimeZone;
 import org.joda.time.Duration;
 import org.joda.time.format.DateTimeFormat;
@@ -218,4 +223,89 @@ public void processElement(ProcessContext context) {
       context.output(failedRow);
     }
   }
+
+  /**
+   * {@link WriteErrorsToTextIO} is a {@link PTransform} that writes strings error messages to file
+   * system using TextIO and custom line format {@link SerializableFunction} to convert errors in
+   * necessary format. 
+ * Example of usage in pipeline: + * + *
{@code
+   * pCollection.apply("Write to TextIO",
+   *   WriteErrorsToTextIO.newBuilder()
+   *     .setErrorWritePath("errors.txt")
+   *     .setTranslateFunction((FailsafeElement failsafeElement) -> {
+   *       ArrayList outputRow  = new ArrayList<>();
+   *       final String message = failsafeElement.getOriginalPayload();
+   *       String timestamp = Instant.now().toString();
+   *       outputRow.add(timestamp);
+   *       outputRow.add(failsafeElement.getErrorMessage());
+   *       outputRow.add(failsafeElement.getStacktrace());
+   *       // Only set the payload if it's populated on the message.
+   *       if (failsafeElement.getOriginalPayload() != null) {
+   *         outputRow.add(message);
+   *       }
+   *
+   *       return String.join(",",outputRow);
+   *     })
+   * }
+ */ + @AutoValue + public abstract static class WriteErrorsToTextIO + extends PTransform>, PDone> { + + public static WriteErrorsToTextIO.Builder newBuilder() { + return new AutoValue_ErrorConverters_WriteErrorsToTextIO.Builder<>(); + } + + public abstract String errorWritePath(); + + public abstract SerializableFunction, String> translateFunction(); + + @Nullable + public abstract Duration windowDuration(); + + @Override + @SuppressWarnings("argument.type.incompatible") + public PDone expand(PCollection> pCollection) { + + PCollection formattedErrorRows = + pCollection.apply( + "GetFormattedErrorRow", + MapElements.into(TypeDescriptors.strings()).via(translateFunction())); + + if (pCollection.isBounded() == PCollection.IsBounded.UNBOUNDED) { + if (windowDuration() == null) { + throw new RuntimeException("Unbounded input requires window interval to be set"); + } + return formattedErrorRows + .apply(Window.into(FixedWindows.of(windowDuration()))) + .apply(TextIO.write().to(errorWritePath()).withNumShards(1).withWindowedWrites()); + } + + return formattedErrorRows.apply(TextIO.write().to(errorWritePath()).withNumShards(1)); + } + + /** Builder for {@link WriteErrorsToTextIO}. */ + @AutoValue.Builder + public abstract static class Builder { + + public abstract WriteErrorsToTextIO.Builder setErrorWritePath(String errorWritePath); + + public abstract WriteErrorsToTextIO.Builder setTranslateFunction( + SerializableFunction, String> translateFunction); + + public abstract WriteErrorsToTextIO.Builder setWindowDuration( + @Nullable Duration duration); + + abstract SerializableFunction, String> translateFunction(); + + abstract WriteErrorsToTextIO autoBuild(); + + public WriteErrorsToTextIO build() { + checkNotNull(translateFunction(), "translateFunction is required."); + return autoBuild(); + } + } + } } diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java index 883417b31b74..293135f2b7cb 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java @@ -18,8 +18,10 @@ package org.apache.beam.examples.complete.datatokenization; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.startsWith; +import static org.junit.Assert.assertNotNull; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -28,8 +30,8 @@ import java.util.LinkedList; import java.util.List; import org.apache.beam.examples.complete.datatokenization.options.DataTokenizationOptions; -import org.apache.beam.examples.complete.datatokenization.transforms.io.FileSystemIO; -import org.apache.beam.examples.complete.datatokenization.transforms.io.FileSystemIO.FORMAT; +import org.apache.beam.examples.complete.datatokenization.transforms.io.TokenizationFileSystemIO; +import org.apache.beam.examples.complete.datatokenization.transforms.io.TokenizationFileSystemIO.FORMAT; import org.apache.beam.examples.complete.datatokenization.utils.FailsafeElementCoder; import org.apache.beam.examples.complete.datatokenization.utils.RowToCsv; import org.apache.beam.examples.complete.datatokenization.utils.SchemasUtils; @@ -42,7 +44,6 @@ import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.JsonToRow; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; @@ -106,28 +107,24 @@ public void testRowToCSV() { @Test public void testFileSystemIOReadCSV() throws IOException { - PCollection jsons = fileSystemIORead(CSV_FILE_PATH, FORMAT.CSV); - assertField(jsons); + PCollection jsons = fileSystemIORead(CSV_FILE_PATH, FORMAT.CSV); + assertRows(jsons); testPipeline.run(); } @Test public void testFileSystemIOReadJSON() throws IOException { - PCollection jsons = fileSystemIORead(JSON_FILE_PATH, FORMAT.JSON); - assertField(jsons); + PCollection jsons = fileSystemIORead(JSON_FILE_PATH, FORMAT.JSON); + assertRows(jsons); testPipeline.run(); } @Test public void testJsonToRow() throws IOException { - PCollection jsons = fileSystemIORead(JSON_FILE_PATH, FORMAT.JSON); + PCollection rows = fileSystemIORead(JSON_FILE_PATH, FORMAT.JSON); SchemasUtils testSchemaUtils = new SchemasUtils(SCHEMA_FILE_PATH, StandardCharsets.UTF_8); - JsonToRow.ParseResult rows = - jsons.apply( - "JsonToRow", - JsonToRow.withExceptionReporting(testSchemaUtils.getBeamSchema()) - .withExtendedErrorInfo()); - PAssert.that(rows.getResults()) + + PAssert.that(rows) .satisfies( x -> { LinkedList beamRows = Lists.newLinkedList(x); @@ -144,8 +141,8 @@ public void testJsonToRow() throws IOException { testPipeline.run(); } - private PCollection fileSystemIORead( - String inputGcsFilePattern, FORMAT inputGcsFileFormat) throws IOException { + private PCollection fileSystemIORead(String inputGcsFilePattern, FORMAT inputGcsFileFormat) + throws IOException { DataTokenizationOptions options = PipelineOptionsFactory.create().as(DataTokenizationOptions.class); options.setDataSchemaPath(SCHEMA_FILE_PATH); @@ -173,18 +170,22 @@ private PCollection fileSystemIORead( RowCoder.of(testSchemaUtils.getBeamSchema())); coderRegistry.registerCoderForType(coder.getEncodedTypeDescriptor(), coder); - return new FileSystemIO(options).read(testPipeline, testSchemaUtils.getJsonBeamSchema()); + return new TokenizationFileSystemIO(options).read(testPipeline, testSchemaUtils); } - private void assertField(PCollection jsons) { + private void assertRows(PCollection jsons) { PAssert.that(jsons) .satisfies( x -> { - LinkedList rows = Lists.newLinkedList(x); + LinkedList rows = Lists.newLinkedList(x); assertThat(rows, hasSize(3)); rows.forEach( row -> { - assertThat(row, startsWith("{\"Field1\":")); + assertNotNull(row.getSchema()); + assertThat(row.getSchema().getFields(), hasSize(3)); + assertThat(row.getSchema().getField(0).getName(), equalTo("Field1")); + + assertThat(row.getValues(), hasSize(3)); }); return null; }); From 906e6e75c6c582d42d6168c8f5499268978da107 Mon Sep 17 00:00:00 2001 From: Nuzhdina-Elena Date: Mon, 22 Mar 2021 13:01:18 +0300 Subject: [PATCH 27/29] code format fixed --- .../complete/datatokenization/transforms/DataProtectors.java | 3 ++- .../complete/datatokenization/transforms/JsonToBeamRow.java | 4 +--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java index 6de4144c7ded..1cf69e656bec 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java @@ -110,7 +110,8 @@ public PCollectionTuple expand(PCollection> inputRows) { Duration maxBuffering = Duration.millis(MAX_BUFFERING); PCollectionTuple pCollectionTuple = inputRows - .apply("GroupRowsIntoBatches", + .apply( + "GroupRowsIntoBatches", GroupIntoBatches.ofSize(batchSize()) .withMaxBufferingDuration(maxBuffering)) .apply( diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/JsonToBeamRow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/JsonToBeamRow.java index 27af8f0294e5..a6c87a368cf9 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/JsonToBeamRow.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/JsonToBeamRow.java @@ -30,9 +30,7 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; -/** - * The {@link JsonToBeamRow} converts jsons string to beam rows. - */ +/** The {@link JsonToBeamRow} converts jsons string to beam rows. */ public class JsonToBeamRow extends PTransform, PCollection> { private final String failedToParseDeadLetterPath; From 9ad7e12db17835e3e169614f18243dca11c927e1 Mon Sep 17 00:00:00 2001 From: Nuzhdina-Elena <79855159+Nuzhdina-Elena@users.noreply.github.com> Date: Tue, 23 Mar 2021 10:50:54 +0300 Subject: [PATCH 28/29] Getting value from environment variables for maxBufferingDurationMs. Information about it added to README (#14) Getting value from environment variables for maxBufferingDurationMs --- .../apache/beam/examples/complete/datatokenization/README.md | 3 +++ .../complete/datatokenization/transforms/DataProtectors.java | 5 +++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md index 9e38b53b6a55..7d92f604ee4f 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md @@ -137,6 +137,9 @@ The template allows for the user to supply the following optional parameter: - **nonTokenizedDeadLetterPath**: Folder where failed to tokenize data will be stored +The template also allows user to override the environment variable: + +- **MAX_BUFFERING_DURATION_MS**: Max duration of buffering rows in milliseconds. Default value: 100ms. in the following format: diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java index 1cf69e656bec..5ddf706d6f2d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/transforms/DataProtectors.java @@ -77,7 +77,8 @@ public class DataProtectors { private static final Logger LOG = LoggerFactory.getLogger(DataProtectors.class); public static final String ID_FIELD_NAME = "ID"; - private static final Integer MAX_BUFFERING = 100; + private static final Long MAX_BUFFERING_DURATION_MS = + Long.valueOf(System.getenv().getOrDefault("MAX_BUFFERING_DURATION_MS", "100")); /** * The {@link RowToTokenizedRow} transform converts {@link Row} to {@link TableRow} objects. The @@ -107,7 +108,7 @@ public PCollectionTuple expand(PCollection> inputRows) { FailsafeElementCoder coder = FailsafeElementCoder.of(RowCoder.of(schema()), RowCoder.of(schema())); - Duration maxBuffering = Duration.millis(MAX_BUFFERING); + Duration maxBuffering = Duration.millis(MAX_BUFFERING_DURATION_MS); PCollectionTuple pCollectionTuple = inputRows .apply( From 3ab9289ae217b0d47cfa12c21e94290df4a4add5 Mon Sep 17 00:00:00 2001 From: MikhailMedvedevAkvelon <78736905+MikhailMedvedevAkvelon@users.noreply.github.com> Date: Tue, 23 Mar 2021 11:54:18 +0400 Subject: [PATCH 29/29] [DATAFLOW-139] Incorrect DSG url lead to NPE (#13) * Fix bug incorrect DSG url lead to NPE DATAFLOW-139 --- .../complete/datatokenization/utils/RowToCsv.java | 5 ++++- .../datatokenization/DataTokenizationTest.java | 15 +++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/RowToCsv.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/RowToCsv.java index 2b4f3b32469b..f8ab7d4ff39f 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/RowToCsv.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/RowToCsv.java @@ -30,6 +30,9 @@ public RowToCsv(String csvDelimiter) { } public String getCsvFromRow(Row row) { - return row.getValues().stream().map(Object::toString).collect(Collectors.joining(csvDelimiter)); + return row.getValues().stream() + .map(item -> item == null ? "null" : item) + .map(Object::toString) + .collect(Collectors.joining(csvDelimiter)); } } diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java index 293135f2b7cb..a2b1ec05a2ec 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/datatokenization/DataTokenizationTest.java @@ -105,6 +105,21 @@ public void testRowToCSV() { Assert.assertEquals(String.join(";", fields), csvResult); } + @Test + public void testRowToCSVWithNull() { + final String nullableTestSchema = + "{\"fields\":[{\"mode\":\"REQUIRED\",\"name\":\"FieldName1\",\"type\":\"STRING\"},{\"mode\":\"NULLABLE\",\"name\":\"FieldName2\",\"type\":\"STRING\"}]}"; + final String expectedCsv = "TestValueOne;null"; + + List values = Lists.newArrayList("TestValueOne", null); + + Schema beamSchema = new SchemasUtils(nullableTestSchema).getBeamSchema(); + Row.Builder rowBuilder = Row.withSchema(beamSchema); + Row row = rowBuilder.addValues(values).build(); + String csvResult = new RowToCsv(";").getCsvFromRow(row); + Assert.assertEquals(expectedCsv, csvResult); + } + @Test public void testFileSystemIOReadCSV() throws IOException { PCollection jsons = fileSystemIORead(CSV_FILE_PATH, FORMAT.CSV);