[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-07-15 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r454831991



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
##
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.DeidentifyContentRequest;
+import com.google.privacy.dlp.v2.DeidentifyContentResponse;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP 
(https://cloud.google.com/dlp/docs/libraries) and
+ * deidentifying text according to provided settings. The transform supports 
both columnar delimited
+ * input data (eg. CSV) and unstructured input.
+ *
+ * If the headerColumns property is set and a sideinput with table headers 
is added to the
+ * PTransform, delimiter also should be set, else the results will be 
incorrect. If headerColumns is
+ * neither set nor passed as side input, input is assumed to be unstructured.
+ *
+ * Either deidentifyTemplateName (String) or deidentifyConfig {@link 
DeidentifyConfig} need to be
+ * set. inspectTemplateName and inspectConfig ({@link InspectConfig} are 
optional.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ *
+ * The transform consumes {@link KV} of {@link String}s (assumed to be 
filename as key and
+ * contents as value) and outputs {@link KV} of {@link String} (eg. filename) 
and {@link
+ * DeidentifyContentResponse}, which will contain {@link Table} of results for 
the user to consume.
+ */
+@Experimental
+@AutoValue
+public abstract class DLPDeidentifyText
+extends PTransform<
+PCollection>, PCollection>> {
+
+  public static final Integer DLP_PAYLOAD_LIMIT_BYTES = 524000;
+
+  /** @return Template name for data inspection. */
+  @Nullable
+  public abstract String getInspectTemplateName();
+
+  /** @return Template name for data deidentification. */
+  @Nullable
+  public abstract String getDeidentifyTemplateName();
+
+  /**
+   * @return Configuration object for data inspection. If present, supersedes 
the template settings.
+   */
+  @Nullable
+  public abstract InspectConfig getInspectConfig();
+
+  /** @return Configuration object for deidentification. If present, 
supersedes the template. */
+  @Nullable
+  public abstract DeidentifyConfig getDeidentifyConfig();
+
+  /** @return List of column names if the input KV value is a delimited row. */
+  @Nullable
+  public abstract PCollectionView> getHeaderColumns();
+
+  /** @return Delimiter to be used when splitting values from input strings 
into columns. */
+  @Nullable
+  public abstract String getColumnDelimiter();
+
+  /** @return Size of input elements batch to be sent to Cloud DLP service in 
one request. */
+  public abstract Integer getBatchSizeBytes();
+
+  /** @return ID of Google Cloud project to be used when deidentifying data. */
+  public abstract String getProjectId();
+
+  @AutoValue.Builder
+  public abstract static class Builder {
+/** @param inspectTemplateName Template name for data inspection. */
+public abstract Builder setInspectTemplateName(String inspectTemplateName);
+
+/** @param headerColumns List of column names if the input KV value is a 
delimited row. */
+public abstract Builder setHeaderColumns(PCollectionView> 

[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-06-01 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r433069651



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java
##
@@ -0,0 +1,281 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.ReidentifyContentRequest;
+import com.google.privacy.dlp.v2.ReidentifyContentResponse;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP 
(https://cloud.google.com/dlp/docs/libraries) and
+ * inspecting text for identifying data according to provided settings.
+ *
+ * The transform supports both delimited columnar input data and 
unstructured input.
+ *
+ * If the headerColumns property is set and a sideinput with headers is 
added to the PTransform,
+ * delimiter also should be set, else the results will be incorrect. If 
headerColumns is neither set
+ * nor passed as sideinput, input is assumed to be unstructured.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ *
+ * The transform consumes {@link KV} of {@link String}s (assumed to be 
filename as key and
+ * contents as value) and outputs {@link KV} of {@link String} (eg. filename) 
and {@link
+ * ReidentifyContentResponse}, which will contain {@link Table} of results for 
the user to consume.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ *
+ * Either reidentifyTemplateName {@link String} or reidentifyConfig {@link 
DeidentifyConfig} need
+ * to be set. inspectConfig {@link InspectConfig} and inspectTemplateName 
{@link String} are
+ * optional.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ */
+@Experimental
+@AutoValue
+public abstract class DLPReidentifyText
+extends PTransform<
+PCollection>, PCollection>> {
+
+  public static final Integer DLP_PAYLOAD_LIMIT_BYTES = 524000;
+
+  /** @return Template name for data inspection. */
+  @Nullable
+  public abstract String getInspectTemplateName();
+
+  /** @return Template name for data reidentification. */
+  @Nullable
+  public abstract String getReidentifyTemplateName();
+
+  /**
+   * @return Configuration object for data inspection. If present, supersedes 
the template settings.
+   */
+  @Nullable
+  public abstract InspectConfig getInspectConfig();
+
+  /** @return Configuration object for reidentification. If present, 
supersedes the template. */
+  @Nullable
+  public abstract DeidentifyConfig getReidentifyConfig();
+
+  /** @return Delimiter to be used when splitting values from input strings 
into columns. */
+  @Nullable
+  public abstract String getColumnDelimiter();
+
+  /** @return List of column names if the input KV value is a delimited row. */
+  @Nullable
+  public abstract PCollectionView> getHeaderColumns();
+
+  /** @return Size of input elements batch to be sent to Cloud DLP service in 
one request. */
+  public abstract Integer getBatchSizeBytes();
+
+  /** @return ID of Google Cloud project to be used when deidentifying data. */
+  public abstract String getProjectId();
+
+  @AutoValue.Builder
+  public abstract static class Builder {
+/** @param inspectTemplateName Template name for data inspection. */
+public abstract Builder setInspectTemplateName(String 

[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-06-01 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r433068917



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java
##
@@ -0,0 +1,281 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.ReidentifyContentRequest;
+import com.google.privacy.dlp.v2.ReidentifyContentResponse;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP 
(https://cloud.google.com/dlp/docs/libraries) and
+ * inspecting text for identifying data according to provided settings.
+ *
+ * The transform supports both delimited columnar input data and 
unstructured input.
+ *
+ * If the headerColumns property is set and a sideinput with headers is 
added to the PTransform,
+ * delimiter also should be set, else the results will be incorrect. If 
headerColumns is neither set
+ * nor passed as sideinput, input is assumed to be unstructured.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ *
+ * The transform consumes {@link KV} of {@link String}s (assumed to be 
filename as key and
+ * contents as value) and outputs {@link KV} of {@link String} (eg. filename) 
and {@link
+ * ReidentifyContentResponse}, which will contain {@link Table} of results for 
the user to consume.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ *
+ * Either reidentifyTemplateName {@link String} or reidentifyConfig {@link 
DeidentifyConfig} need
+ * to be set. inspectConfig {@link InspectConfig} and inspectTemplateName 
{@link String} are
+ * optional.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ */
+@Experimental
+@AutoValue
+public abstract class DLPReidentifyText
+extends PTransform<
+PCollection>, PCollection>> {
+
+  public static final Integer DLP_PAYLOAD_LIMIT_BYTES = 524000;
+
+  /** @return Template name for data inspection. */
+  @Nullable
+  public abstract String getInspectTemplateName();
+
+  /** @return Template name for data reidentification. */
+  @Nullable
+  public abstract String getReidentifyTemplateName();
+
+  /**
+   * @return Configuration object for data inspection. If present, supersedes 
the template settings.
+   */
+  @Nullable
+  public abstract InspectConfig getInspectConfig();
+
+  /** @return Configuration object for reidentification. If present, 
supersedes the template. */
+  @Nullable
+  public abstract DeidentifyConfig getReidentifyConfig();
+
+  /** @return Delimiter to be used when splitting values from input strings 
into columns. */
+  @Nullable
+  public abstract String getColumnDelimiter();
+
+  /** @return List of column names if the input KV value is a delimited row. */
+  @Nullable
+  public abstract PCollectionView> getHeaderColumns();
+
+  /** @return Size of input elements batch to be sent to Cloud DLP service in 
one request. */
+  public abstract Integer getBatchSizeBytes();
+
+  /** @return ID of Google Cloud project to be used when deidentifying data. */
+  public abstract String getProjectId();
+
+  @AutoValue.Builder
+  public abstract static class Builder {
+/** @param inspectTemplateName Template name for data inspection. */
+public abstract Builder setInspectTemplateName(String 

[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-28 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r431640905



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
##
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.DeidentifyContentRequest;
+import com.google.privacy.dlp.v2.DeidentifyContentResponse;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP 
(https://cloud.google.com/dlp/docs/libraries) and
+ * deidentifying text according to provided settings. The transform supports 
both CSV formatted
+ * input data and unstructured input.
+ *
+ * If the csvHeader property is set and a sideinput with CSV headers is 
added to the PTransform,
+ * csvDelimiter also should be set, else the results will be incorrect. If 
csvHeader is neither set
+ * nor passed as sideinput, input is assumed to be unstructured.

Review comment:
   Sure, that's a good idea.

##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java
##
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.InspectContentRequest;
+import com.google.privacy.dlp.v2.InspectContentResponse;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP 
(https://cloud.google.com/dlp/docs/libraries) and
+ * inspecting text for identifying data according to provided settings. The 
transform supports both
+ * CSV formatted input data and unstructured input.
+ *
+ * If the csvHeader property is set and a sideinput with CSV headers is 
added to the PTransform,
+ * 

[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-28 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r431627973



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
##
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.DeidentifyContentRequest;
+import com.google.privacy.dlp.v2.DeidentifyContentResponse;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP 
(https://cloud.google.com/dlp/docs/libraries) and
+ * deidentifying text according to provided settings. The transform supports 
both CSV formatted
+ * input data and unstructured input.
+ *
+ * If the csvHeader property is set and a sideinput with CSV headers is 
added to the PTransform,
+ * csvDelimiter also should be set, else the results will be incorrect. If 
csvHeader is neither set
+ * nor passed as sideinput, input is assumed to be unstructured.
+ *
+ * Either deidentifyTemplateName (String) or deidentifyConfig {@link 
DeidentifyConfig} need to be
+ * set. inspectTemplateName and inspectConfig ({@link InspectConfig} are 
optional.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ *
+ * The transform consumes {@link KV} of {@link String}s (assumed to be 
filename as key and
+ * contents as value) and outputs {@link KV} of {@link String} (eg. filename) 
and {@link
+ * DeidentifyContentResponse}, which will contain {@link Table} of results for 
the user to consume.
+ */
+@Experimental
+@AutoValue
+public abstract class DLPDeidentifyText
+extends PTransform<
+PCollection>, PCollection>> {
+
+  public static final Integer DLP_PAYLOAD_LIMIT_BYTES = 524000;
+
+  /** @return Template name for data inspection. */
+  @Nullable
+  public abstract String inspectTemplateName();
+
+  /** @return Template name for data deidentification. */
+  @Nullable
+  public abstract String deidentifyTemplateName();
+
+  /**
+   * @return Configuration object for data inspection. If present, supersedes 
the template settings.
+   */
+  @Nullable
+  public abstract InspectConfig inspectConfig();
+
+  /** @return Configuration object for deidentification. If present, 
supersedes the template. */
+  @Nullable
+  public abstract DeidentifyConfig deidentifyConfig();
+
+  /** @return List of column names if the input KV value is a CSV formatted 
row. */
+  @Nullable
+  public abstract PCollectionView> csvHeader();

Review comment:
   Right, I must have missed that. Thanks!





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-25 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r429816114



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
##
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.DeidentifyContentRequest;
+import com.google.privacy.dlp.v2.DeidentifyContentResponse;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP and deidentifying text 
according to provided
+ * settings. The transform supports both CSV formatted input data and 
unstructured input.
+ *
+ * If the csvHeader property is set, csvDelimiter also should be, else the 
results will be
+ * incorrect. If csvHeader is not set, input is assumed to be unstructured.
+ *
+ * Either inspectTemplateName (String) or inspectConfig {@link 
InspectConfig} need to be set. The
+ * situation is the same with deidentifyTemplateName and deidentifyConfig 
({@link DeidentifyConfig}.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ *
+ * The transform outputs {@link KV} of {@link String} (eg. filename) and 
{@link
+ * DeidentifyContentResponse}, which will contain {@link Table} of results for 
the user to consume.
+ */
+@Experimental
+@AutoValue
+public abstract class DLPDeidentifyText
+extends PTransform<
+PCollection>, PCollection>> {
+
+  @Nullable
+  public abstract String inspectTemplateName();
+
+  @Nullable
+  public abstract String deidentifyTemplateName();
+
+  @Nullable
+  public abstract InspectConfig inspectConfig();
+
+  @Nullable
+  public abstract DeidentifyConfig deidentifyConfig();
+
+  @Nullable
+  public abstract PCollectionView> csvHeader();
+
+  @Nullable
+  public abstract String csvDelimiter();
+
+  public abstract Integer batchSize();
+
+  public abstract String projectId();
+
+  @AutoValue.Builder
+  public abstract static class Builder {
+public abstract Builder setInspectTemplateName(String inspectTemplateName);
+
+public abstract Builder setCsvHeader(PCollectionView> 
csvHeader);
+
+public abstract Builder setCsvDelimiter(String delimiter);
+
+public abstract Builder setBatchSize(Integer batchSize);
+
+public abstract Builder setProjectId(String projectId);
+
+public abstract Builder setDeidentifyTemplateName(String 
deidentifyTemplateName);
+
+public abstract Builder setInspectConfig(InspectConfig inspectConfig);
+
+public abstract Builder setDeidentifyConfig(DeidentifyConfig 
deidentifyConfig);
+
+public abstract DLPDeidentifyText build();
+  }
+
+  public static DLPDeidentifyText.Builder newBuilder() {
+return new AutoValue_DLPDeidentifyText.Builder();
+  }
+
+  /**
+   * The transform batches the contents of input PCollection and then calls 
Cloud DLP service to
+   * perform the deidentification.
+   *
+   * @param input input PCollection
+   * @return PCollection after transformations
+   */
+  @Override
+  public PCollection> expand(
+  PCollection> input) {
+return input
+.apply(ParDo.of(new MapStringToDlpRow(csvDelimiter(
+.apply("Batch Contents", ParDo.of(new BatchRequestForDLP(batchSize(
+.apply(
+"DLPDeidentify",
+ParDo.of(
+new 

[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-22 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r429213975



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
##
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.DeidentifyContentRequest;
+import com.google.privacy.dlp.v2.DeidentifyContentResponse;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP and deidentifying text 
according to provided
+ * settings. The transform supports both CSV formatted input data and 
unstructured input.
+ *
+ * If the csvHeader property is set, csvDelimiter also should be, else the 
results will be
+ * incorrect. If csvHeader is not set, input is assumed to be unstructured.

Review comment:
   Thanks for this remark, I added early validation to the code.

##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
##
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.DeidentifyContentRequest;
+import com.google.privacy.dlp.v2.DeidentifyContentResponse;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP and deidentifying text 
according to provided
+ * settings. The transform supports both CSV formatted input data and 
unstructured input.
+ *
+ * If the csvHeader property is set, csvDelimiter also should be, else the 
results will be
+ * incorrect. If csvHeader is not set, input is assumed to be unstructured.
+ *
+ * Either inspectTemplateName (String) or 

[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-22 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r429213709



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDLP.java
##
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.privacy.dlp.v2.Table;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+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.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DoFn batching the input PCollection into bigger requests in order to better 
utilize the Cloud DLP
+ * service.
+ */
+@Experimental
+class BatchRequestForDLP extends DoFn, KV>> {
+  public static final Logger LOG = 
LoggerFactory.getLogger(BatchRequestForDLP.class);
+
+  private final Counter numberOfRowsBagged =
+  Metrics.counter(BatchRequestForDLP.class, "numberOfRowsBagged");
+  private final Integer batchSize;
+
+  @StateId("elementsBag")
+  private final StateSpec>> elementsBag = 
StateSpecs.bag();
+
+  @TimerId("eventTimer")
+  private final TimerSpec eventTimer = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+  public BatchRequestForDLP(Integer batchSize) {
+this.batchSize = batchSize;
+  }
+
+  @ProcessElement
+  public void process(
+  @Element KV element,
+  @StateId("elementsBag") BagState> elementsBag,
+  @TimerId("eventTimer") Timer eventTimer,
+  BoundedWindow w) {
+elementsBag.add(element);
+eventTimer.set(w.maxTimestamp());
+  }
+
+  @OnTimer("eventTimer")
+  public void onTimer(
+  @StateId("elementsBag") BagState> elementsBag,
+  OutputReceiver>> output) {
+String key = elementsBag.read().iterator().next().getKey();
+AtomicInteger bufferSize = new AtomicInteger();
+List rows = new ArrayList<>();
+elementsBag
+.read()
+.forEach(
+element -> {
+  int elementSize = element.getValue().getSerializedSize();
+  boolean clearBuffer = bufferSize.intValue() + elementSize > 
batchSize;
+  if (clearBuffer) {
+numberOfRowsBagged.inc(rows.size());
+LOG.debug("Clear Buffer {} , Key {}", bufferSize.intValue(), 
element.getKey());
+output.output(KV.of(element.getKey(), rows));
+rows.clear();
+bufferSize.set(0);
+  }
+  rows.add(element.getValue());
+  bufferSize.getAndAdd(element.getValue().getSerializedSize());
+});
+if (!rows.isEmpty()) {
+  LOG.debug("Remaining rows {}", rows.size());

Review comment:
   Done, thanks

##
File path: 
sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPTextOperationsIT.java
##
@@ -0,0 +1,154 @@
+/*
+ * 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 

[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-22 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r429213646



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDLP.java
##
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.privacy.dlp.v2.Table;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+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.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DoFn batching the input PCollection into bigger requests in order to better 
utilize the Cloud DLP
+ * service.
+ */
+@Experimental
+class BatchRequestForDLP extends DoFn, KV>> {
+  public static final Logger LOG = 
LoggerFactory.getLogger(BatchRequestForDLP.class);
+
+  private final Counter numberOfRowsBagged =
+  Metrics.counter(BatchRequestForDLP.class, "numberOfRowsBagged");
+  private final Integer batchSize;
+
+  @StateId("elementsBag")
+  private final StateSpec>> elementsBag = 
StateSpecs.bag();
+
+  @TimerId("eventTimer")
+  private final TimerSpec eventTimer = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+  public BatchRequestForDLP(Integer batchSize) {
+this.batchSize = batchSize;
+  }
+
+  @ProcessElement
+  public void process(
+  @Element KV element,
+  @StateId("elementsBag") BagState> elementsBag,
+  @TimerId("eventTimer") Timer eventTimer,
+  BoundedWindow w) {
+elementsBag.add(element);
+eventTimer.set(w.maxTimestamp());
+  }
+
+  @OnTimer("eventTimer")
+  public void onTimer(
+  @StateId("elementsBag") BagState> elementsBag,
+  OutputReceiver>> output) {
+String key = elementsBag.read().iterator().next().getKey();

Review comment:
   Ok, done!





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-22 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r429202254



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
##
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.DeidentifyContentRequest;
+import com.google.privacy.dlp.v2.DeidentifyContentResponse;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP and deidentifying text 
according to provided
+ * settings. The transform supports both CSV formatted input data and 
unstructured input.
+ *
+ * If the csvHeader property is set, csvDelimiter also should be, else the 
results will be
+ * incorrect. If csvHeader is not set, input is assumed to be unstructured.
+ *
+ * Either inspectTemplateName (String) or inspectConfig {@link 
InspectConfig} need to be set. The
+ * situation is the same with deidentifyTemplateName and deidentifyConfig 
({@link DeidentifyConfig}.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ *
+ * The transform outputs {@link KV} of {@link String} (eg. filename) and 
{@link
+ * DeidentifyContentResponse}, which will contain {@link Table} of results for 
the user to consume.
+ */
+@Experimental
+@AutoValue
+public abstract class DLPDeidentifyText
+extends PTransform<
+PCollection>, PCollection>> {
+
+  @Nullable
+  public abstract String inspectTemplateName();

Review comment:
   Sure, I'll add comments





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-21 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r428679955



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
##
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.DeidentifyContentRequest;
+import com.google.privacy.dlp.v2.DeidentifyContentResponse;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP and deidentifying text 
according to provided
+ * settings. The transform supports both CSV formatted input data and 
unstructured input.
+ *
+ * If the csvHeader property is set, csvDelimiter also should be, else the 
results will be
+ * incorrect. If csvHeader is not set, input is assumed to be unstructured.
+ *
+ * Either inspectTemplateName (String) or inspectConfig {@link 
InspectConfig} need to be set. The
+ * situation is the same with deidentifyTemplateName and deidentifyConfig 
({@link DeidentifyConfig}.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ *
+ * The transform outputs {@link KV} of {@link String} (eg. filename) and 
{@link
+ * DeidentifyContentResponse}, which will contain {@link Table} of results for 
the user to consume.
+ */
+@Experimental
+@AutoValue
+public abstract class DLPDeidentifyText
+extends PTransform<
+PCollection>, PCollection>> {
+
+  @Nullable
+  public abstract String inspectTemplateName();
+
+  @Nullable
+  public abstract String deidentifyTemplateName();
+
+  @Nullable
+  public abstract InspectConfig inspectConfig();
+
+  @Nullable
+  public abstract DeidentifyConfig deidentifyConfig();
+
+  @Nullable
+  public abstract PCollectionView> csvHeader();
+
+  @Nullable
+  public abstract String csvDelimiter();
+
+  public abstract Integer batchSize();
+
+  public abstract String projectId();
+
+  @AutoValue.Builder
+  public abstract static class Builder {
+public abstract Builder setInspectTemplateName(String inspectTemplateName);
+
+public abstract Builder setCsvHeader(PCollectionView> 
csvHeader);
+
+public abstract Builder setCsvDelimiter(String delimiter);
+
+public abstract Builder setBatchSize(Integer batchSize);
+
+public abstract Builder setProjectId(String projectId);
+
+public abstract Builder setDeidentifyTemplateName(String 
deidentifyTemplateName);
+
+public abstract Builder setInspectConfig(InspectConfig inspectConfig);
+
+public abstract Builder setDeidentifyConfig(DeidentifyConfig 
deidentifyConfig);
+
+public abstract DLPDeidentifyText build();
+  }
+

Review comment:
   Thanks!





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-21 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r428676595



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.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.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.ReidentifyContentRequest;
+import com.google.privacy.dlp.v2.ReidentifyContentResponse;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP and inspecting text for 
identifying data according
+ * to provided settings.
+ *
+ * Either inspectTemplateName (String) or inspectConfig {@link 
InspectConfig} need to be set, the
+ * same goes for reidentifyTemplateName or reidentifyConfig.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ */
+@Experimental
+@AutoValue
+public abstract class DLPReidentifyText
+extends PTransform<
+PCollection>, PCollection>> {
+
+  public static final Logger LOG = 
LoggerFactory.getLogger(DLPInspectText.class);
+
+  public static final Integer DLP_PAYLOAD_LIMIT = 52400;
+
+  @Nullable
+  public abstract String inspectTemplateName();
+
+  @Nullable
+  public abstract String reidentifyTemplateName();
+
+  @Nullable
+  public abstract InspectConfig inspectConfig();
+
+  @Nullable
+  public abstract DeidentifyConfig reidentifyConfig();
+
+  @Nullable
+  public abstract String csvDelimiter();
+
+  @Nullable
+  public abstract PCollectionView> csvHeaders();
+
+  public abstract Integer batchSize();
+
+  public abstract String projectId();
+
+  @AutoValue.Builder
+  public abstract static class Builder {
+public abstract Builder setInspectTemplateName(String inspectTemplateName);
+
+public abstract Builder setInspectConfig(InspectConfig inspectConfig);
+
+public abstract Builder setReidentifyConfig(DeidentifyConfig 
deidentifyConfig);
+
+public abstract Builder setReidentifyTemplateName(String 
deidentifyTemplateName);
+
+public abstract Builder setBatchSize(Integer batchSize);
+
+public abstract Builder setCsvHeaders(PCollectionView> 
csvHeaders);
+
+public abstract Builder setCsvDelimiter(String delimiter);
+
+public abstract Builder setProjectId(String projectId);
+
+public abstract DLPReidentifyText build();
+  }
+
+  public static DLPReidentifyText.Builder newBuilder() {
+return new AutoValue_DLPReidentifyText.Builder();
+  }
+
+  @Override
+  public PCollection> expand(
+  PCollection> input) {
+return input
+.apply(ParDo.of(new MapStringToDlpRow(csvDelimiter(
+.apply("Batch Contents", ParDo.of(new BatchRequestForDLP(batchSize(
+.apply(
+"DLPDeidentify",
+ParDo.of(
+new ReidentifyText(
+projectId(),
+inspectTemplateName(),
+reidentifyTemplateName(),
+inspectConfig(),
+reidentifyConfig(),
+csvHeaders(;
+  }
+
+  public static class ReidentifyText
+  extends DoFn>, KV> {
+private final String projectId;
+private final String inspectTemplateName;
+private final String reidentifyTemplateName;
+private final InspectConfig 

[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-21 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r428666183



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRow.java
##
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.privacy.dlp.v2.Table;
+import com.google.privacy.dlp.v2.Value;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.values.KV;
+
+class MapStringToDlpRow extends DoFn, KV> {
+  private final String delimiter;
+
+  public MapStringToDlpRow(String delimiter) {
+this.delimiter = delimiter;
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext context) {

Review comment:
   Ok, will do!





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-21 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r428661780



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
##
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.DeidentifyContentRequest;
+import com.google.privacy.dlp.v2.DeidentifyContentResponse;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP and deidentifying text 
according to provided
+ * settings. The transform supports both CSV formatted input data and 
unstructured input.
+ *
+ * If the csvHeader property is set, csvDelimiter also should be, else the 
results will be
+ * incorrect. If csvHeader is not set, input is assumed to be unstructured.
+ *
+ * Either inspectTemplateName (String) or inspectConfig {@link 
InspectConfig} need to be set. The
+ * situation is the same with deidentifyTemplateName and deidentifyConfig 
({@link DeidentifyConfig}.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ *
+ * The transform outputs {@link KV} of {@link String} (eg. filename) and 
{@link
+ * DeidentifyContentResponse}, which will contain {@link Table} of results for 
the user to consume.
+ */
+@Experimental
+@AutoValue
+public abstract class DLPDeidentifyText
+extends PTransform<
+PCollection>, PCollection>> {
+
+  @Nullable
+  public abstract String inspectTemplateName();
+
+  @Nullable
+  public abstract String deidentifyTemplateName();
+
+  @Nullable
+  public abstract InspectConfig inspectConfig();
+
+  @Nullable
+  public abstract DeidentifyConfig deidentifyConfig();
+
+  @Nullable
+  public abstract PCollectionView> csvHeader();
+
+  @Nullable
+  public abstract String csvDelimiter();
+
+  public abstract Integer batchSize();
+
+  public abstract String projectId();
+
+  @AutoValue.Builder
+  public abstract static class Builder {
+public abstract Builder setInspectTemplateName(String inspectTemplateName);
+
+public abstract Builder setCsvHeader(PCollectionView> 
csvHeader);
+
+public abstract Builder setCsvDelimiter(String delimiter);
+
+public abstract Builder setBatchSize(Integer batchSize);
+
+public abstract Builder setProjectId(String projectId);
+
+public abstract Builder setDeidentifyTemplateName(String 
deidentifyTemplateName);
+
+public abstract Builder setInspectConfig(InspectConfig inspectConfig);
+
+public abstract Builder setDeidentifyConfig(DeidentifyConfig 
deidentifyConfig);
+
+public abstract DLPDeidentifyText build();
+  }
+
+  public static DLPDeidentifyText.Builder newBuilder() {
+return new AutoValue_DLPDeidentifyText.Builder();
+  }
+
+  /**
+   * The transform batches the contents of input PCollection and then calls 
Cloud DLP service to
+   * perform the deidentification.
+   *
+   * @param input input PCollection
+   * @return PCollection after transformations
+   */
+  @Override
+  public PCollection> expand(
+  PCollection> input) {
+return input
+.apply(ParDo.of(new MapStringToDlpRow(csvDelimiter(
+.apply("Batch Contents", ParDo.of(new BatchRequestForDLP(batchSize(
+.apply(
+"DLPDeidentify",
+ParDo.of(
+new 

[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-21 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r428606587



##
File path: 
sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/DLPTextOperationsIT.java
##
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import static org.junit.Assert.assertTrue;
+
+import com.google.privacy.dlp.v2.CharacterMaskConfig;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.DeidentifyContentResponse;
+import com.google.privacy.dlp.v2.Finding;
+import com.google.privacy.dlp.v2.InfoType;
+import com.google.privacy.dlp.v2.InfoTypeTransformations;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.InspectContentResponse;
+import com.google.privacy.dlp.v2.Likelihood;
+import com.google.privacy.dlp.v2.PrimitiveTransformation;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class DLPTextOperationsIT {
+  @Rule public TestPipeline testPipeline = TestPipeline.create();
+
+  private static final String IDENTIFYING_TEXT = "mary@example.com";
+  private static InfoType emailAddress = 
InfoType.newBuilder().setName("EMAIL_ADDRESS").build();
+  private static final InspectConfig inspectConfig =
+  InspectConfig.newBuilder()
+  .addInfoTypes(emailAddress)
+  .setMinLikelihood(Likelihood.LIKELY)
+  .build();
+
+  @Test
+  public void inspectsText() {
+String projectId = 
testPipeline.getOptions().as(GcpOptions.class).getProject();
+PCollection> inspectionResult =
+testPipeline
+.apply(Create.of(KV.of("", IDENTIFYING_TEXT)))
+.apply(
+DLPInspectText.newBuilder()
+.setBatchSize(52400)

Review comment:
   @santhh Is it 52400 or 524000? It should be the latter, since we're 
setting the limit to 524 kb, I think.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-21 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r428605560



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDLP.java
##
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.ml;
+
+import com.google.privacy.dlp.v2.Table;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+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.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DoFn batching the input PCollection into bigger requests in order to better 
utilize the Cloud DLP
+ * service.
+ */
+@Experimental
+class BatchRequestForDLP extends DoFn, KV>> {
+  public static final Logger LOG = 
LoggerFactory.getLogger(BatchRequestForDLP.class);
+
+  private final Counter numberOfRowsBagged =
+  Metrics.counter(BatchRequestForDLP.class, "numberOfRowsBagged");
+  private final Integer batchSize;
+
+  @StateId("elementsBag")
+  private final StateSpec>> elementsBag = 
StateSpecs.bag();
+
+  @TimerId("eventTimer")
+  private final TimerSpec eventTimer = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+  public BatchRequestForDLP(Integer batchSize) {
+this.batchSize = batchSize;
+  }
+
+  @ProcessElement
+  public void process(
+  @Element KV element,
+  @StateId("elementsBag") BagState> elementsBag,
+  @TimerId("eventTimer") Timer eventTimer,
+  BoundedWindow w) {
+elementsBag.add(element);
+eventTimer.set(w.maxTimestamp());
+  }
+
+  @OnTimer("eventTimer")
+  public void onTimer(
+  @StateId("elementsBag") BagState> elementsBag,
+  OutputReceiver>> output) {
+String key = elementsBag.read().iterator().next().getKey();
+AtomicInteger bufferSize = new AtomicInteger();
+List rows = new ArrayList<>();
+elementsBag
+.read()
+.forEach(
+element -> {
+  int elementSize = element.getValue().getSerializedSize();
+  boolean clearBuffer = bufferSize.intValue() + elementSize > 
batchSize;
+  if (clearBuffer) {
+numberOfRowsBagged.inc(rows.size());
+LOG.debug("Clear Buffer {} , Key {}", bufferSize.intValue(), 
element.getKey());

Review comment:
   Sure, I'll move it and add units. Thanks!





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-20 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r428033761



##
File path: sdks/java/extensions/ml/build.gradle
##
@@ -26,15 +28,23 @@ description = 'Apache Beam :: SDKs :: Java :: Extensions :: 
ML'
 dependencies {
 compile project(path: ":sdks:java:core", configuration: "shadow")
 compile project(":sdks:java:expansion-service")
-testCompile project(path: ':sdks:java:core', configuration: 'shadowTest')
 compile 'com.google.cloud:google-cloud-video-intelligence:1.2.0'
+compile 'com.google.cloud:google-cloud-dlp:1.1.1'

Review comment:
   Sure, thanks for this insight.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-20 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r427880303



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.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.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.ReidentifyContentRequest;
+import com.google.privacy.dlp.v2.ReidentifyContentResponse;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP and inspecting text for 
identifying data according
+ * to provided settings.
+ *
+ * Either inspectTemplateName (String) or inspectConfig {@link 
InspectConfig} need to be set, the
+ * same goes for reidentifyTemplateName or reidentifyConfig.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ */
+@Experimental
+@AutoValue
+public abstract class DLPReidentifyText
+extends PTransform<
+PCollection>, PCollection>> {
+
+  public static final Logger LOG = 
LoggerFactory.getLogger(DLPInspectText.class);
+
+  public static final Integer DLP_PAYLOAD_LIMIT = 52400;

Review comment:
   I agree with Tyson's comment about early validation, I think I'll leave 
the limits in here.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] mwalenia commented on a change in pull request #11566: [BEAM-9723] Add DLP integration transforms

2020-05-20 Thread GitBox


mwalenia commented on a change in pull request #11566:
URL: https://github.com/apache/beam/pull/11566#discussion_r427872119



##
File path: 
sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.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.sdk.extensions.ml;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.dlp.v2.DlpServiceClient;
+import com.google.privacy.dlp.v2.ContentItem;
+import com.google.privacy.dlp.v2.DeidentifyConfig;
+import com.google.privacy.dlp.v2.FieldId;
+import com.google.privacy.dlp.v2.InspectConfig;
+import com.google.privacy.dlp.v2.ProjectName;
+import com.google.privacy.dlp.v2.ReidentifyContentRequest;
+import com.google.privacy.dlp.v2.ReidentifyContentResponse;
+import com.google.privacy.dlp.v2.Table;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+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.PCollectionView;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link PTransform} connecting to Cloud DLP and inspecting text for 
identifying data according
+ * to provided settings.
+ *
+ * Either inspectTemplateName (String) or inspectConfig {@link 
InspectConfig} need to be set, the
+ * same goes for reidentifyTemplateName or reidentifyConfig.
+ *
+ * Batch size defines how big are batches sent to DLP at once in bytes.
+ */
+@Experimental
+@AutoValue
+public abstract class DLPReidentifyText
+extends PTransform<
+PCollection>, PCollection>> {
+
+  public static final Logger LOG = 
LoggerFactory.getLogger(DLPInspectText.class);
+
+  public static final Integer DLP_PAYLOAD_LIMIT = 52400;
+
+  @Nullable
+  public abstract String inspectTemplateName();
+
+  @Nullable
+  public abstract String reidentifyTemplateName();
+
+  @Nullable
+  public abstract InspectConfig inspectConfig();
+
+  @Nullable
+  public abstract DeidentifyConfig reidentifyConfig();
+
+  @Nullable
+  public abstract String csvDelimiter();
+
+  @Nullable
+  public abstract PCollectionView> csvHeaders();
+
+  public abstract Integer batchSize();
+
+  public abstract String projectId();
+
+  @AutoValue.Builder
+  public abstract static class Builder {
+public abstract Builder setInspectTemplateName(String inspectTemplateName);
+
+public abstract Builder setInspectConfig(InspectConfig inspectConfig);
+
+public abstract Builder setReidentifyConfig(DeidentifyConfig 
deidentifyConfig);
+
+public abstract Builder setReidentifyTemplateName(String 
deidentifyTemplateName);
+
+public abstract Builder setBatchSize(Integer batchSize);
+
+public abstract Builder setCsvHeaders(PCollectionView> 
csvHeaders);
+
+public abstract Builder setCsvDelimiter(String delimiter);
+
+public abstract Builder setProjectId(String projectId);
+
+public abstract DLPReidentifyText build();
+  }
+
+  public static DLPReidentifyText.Builder newBuilder() {
+return new AutoValue_DLPReidentifyText.Builder();
+  }
+
+  @Override
+  public PCollection> expand(
+  PCollection> input) {
+return input
+.apply(ParDo.of(new MapStringToDlpRow(csvDelimiter(
+.apply("Batch Contents", ParDo.of(new BatchRequestForDLP(batchSize(
+.apply(
+"DLPDeidentify",

Review comment:
   Right, I'll rename this





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org