pabloem commented on a change in pull request #11950: URL: https://github.com/apache/beam/pull/11950#discussion_r436992485
########## File path: sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEvent.java ########## @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.splunk; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import com.google.auto.value.AutoValue; +import com.google.gson.annotations.SerializedName; +import javax.annotation.Nullable; + +/** + * A {@link SplunkEvent} describes a single payload sent to Splunk's Http Event Collector (HEC) + * endpoint. + * + * <p>Each object represents a single event and related metadata elements such as: + * + * <ul> + * <li>time + * <li>host + * <li>source + * <li>sourceType + * <li>index + * </ul> + */ +@AutoValue +public abstract class SplunkEvent { Review comment: I recommend you use `@DefaultSchema(AutoValueSchema.class)` for this class instead of writing a custom coder. ########## File path: sdks/java/io/splunk/build.gradle ########## @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +plugins { + id 'org.apache.beam.module' +} +applyJavaNature(automaticModuleName: 'org.apache.beam.sdk.io.splunk') + +description = "Apache Beam :: SDKs :: Java :: IO :: Splunk" +ext.summary = "IO to write events to Splunk Http Event Collector (HEC)" + +dependencies { + compile library.java.slf4j_api + compile project(path: ":sdks:java:core", configuration: "shadow") + compile group: "com.google.code.gson", name: "gson", version: "2.8.6" + compile group: "com.google.api-client", name: "google-api-client", version: "1.30.9" + compile group: "com.google.http-client", name: "google-http-client-apache-v2", version: "1.31.0" Review comment: TODO(pablo/sameer) - perhaps add these to the project dependencies. Ensure they don't give extra trouble. ########## File path: sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkIO.java ########## @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.splunk; + +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 com.google.auto.value.AutoValue; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.options.ValueProvider; +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.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An unbounded sink for Splunk's Http Event Collector (HEC). + * + * <p>For more information, see the online documentation at <a + * href="https://dev.splunk.com/enterprise/docs/dataapps/httpeventcollector/">Splunk HEC</a>. + * + * <h3>Writing to Splunk's HEC</h3> + * + * <p>The {@link SplunkIO} class provides a {@link PTransform} that allows writing {@link + * SplunkEvent} messages into a Splunk HEC end point. + * + * <p>It takes as an input a {@link PCollection PCollection<SplunkEvent>}, where each {@link + * SplunkEvent} represents an event to be published to HEC. + * + * <p>To configure a {@link SplunkIO}, you must provide at a minimum: + * + * <ul> + * <li>url - HEC endpoint URL. + * <li>token - HEC endpoint token. + * </ul> + * + * <p>The {@link SplunkIO} transform can be customized further by optionally specifying: + * + * <ul> + * <li>parallelism - Number of parallel requests to the HEC. + * <li>batchCount - Number of events in a single batch. + * <li>disableCertificateValidation - Whether to disable ssl validation (useful for self-signed + * certificates) + * </ul> + * + * <p>This transform will return any non-transient write failures via a {@link PCollection + * PCollection<SplunkWriteError>}, where each {@link SplunkWriteError} captures the error that + * occurred while attempting to write to HEC. These can be published to a dead-letter sink or + * reprocessed. + * + * <p>For example: + * + * <pre>{@code + * PCollection<SplunkEvent> events = ...; + * + * PCollection<SplunkWriteError> errors = + * events.apply("WriteToSplunk", + * SplunkIO.writeBuilder() Review comment: In Beam, these PTransform builders are not really used as such. Do you think it makes sense to have a fluent, factory-type class that doesn't require a builder? e.g. ``` * SplunkIO.write() * .withToken(token) * .withUrl(url) * .withBatchCount(batchCount) * .withParallelism(parallelism) * .withDisableCertificateValidation(true) ``` ########## File path: sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java ########## @@ -0,0 +1,395 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.splunk; + +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 com.google.api.client.http.HttpResponse; +import com.google.api.client.http.HttpResponseException; +import com.google.auto.value.AutoValue; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.security.KeyManagementException; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.options.ValueProvider; +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.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** A {@link DoFn} to write {@link SplunkEvent}s to Splunk's HEC endpoint. */ +@AutoValue +abstract class SplunkEventWriter extends DoFn<KV<Integer, SplunkEvent>, SplunkWriteError> { + + private static final Integer DEFAULT_BATCH_COUNT = 1; + private static final Boolean DEFAULT_DISABLE_CERTIFICATE_VALIDATION = false; + private static final Logger LOG = LoggerFactory.getLogger(SplunkEventWriter.class); + private static final long DEFAULT_FLUSH_DELAY = 2; + private static final Counter INPUT_COUNTER = + Metrics.counter(SplunkEventWriter.class, "inbound-events"); + private static final Counter SUCCESS_WRITES = + Metrics.counter(SplunkEventWriter.class, "outbound-successful-events"); + private static final Counter FAILED_WRITES = + Metrics.counter(SplunkEventWriter.class, "outbound-failed-events"); + private static final String BUFFER_STATE_NAME = "buffer"; + private static final String COUNT_STATE_NAME = "count"; + private static final String TIME_ID_NAME = "expiry"; + + @StateId(BUFFER_STATE_NAME) + private final StateSpec<BagState<SplunkEvent>> buffer = StateSpecs.bag(); + + @StateId(COUNT_STATE_NAME) + private final StateSpec<ValueState<Long>> count = StateSpecs.value(); + + @TimerId(TIME_ID_NAME) + private final TimerSpec expirySpec = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + private Integer batchCount; + private Boolean disableValidation; + private HttpEventPublisher publisher; + + private static final Gson GSON = + new GsonBuilder().setFieldNamingStrategy(f -> f.getName().toLowerCase()).create(); + + /** A builder class for creating a {@link SplunkEventWriter}. */ + static Builder newBuilder() { + return new AutoValue_SplunkEventWriter.Builder(); + } + + @Nullable + abstract ValueProvider<String> url(); + + @Nullable + abstract ValueProvider<String> token(); + + @Nullable + abstract ValueProvider<Boolean> disableCertificateValidation(); + + @Nullable + abstract ValueProvider<Integer> inputBatchCount(); + + @Setup + public void setup() { + + checkArgument(url().isAccessible(), "url is required for writing events."); + checkArgument(token().isAccessible(), "Access token is required for writing events."); + + // Either user supplied or default batchCount. + if (batchCount == null) { + + if (inputBatchCount() != null) { + batchCount = inputBatchCount().get(); + } + + batchCount = MoreObjects.firstNonNull(batchCount, DEFAULT_BATCH_COUNT); + LOG.info("Batch count set to: {}", batchCount); + } + + // Either user supplied or default disableValidation. + if (disableValidation == null) { + + if (disableCertificateValidation() != null) { + disableValidation = disableCertificateValidation().get(); + } + + disableValidation = + MoreObjects.firstNonNull(disableValidation, DEFAULT_DISABLE_CERTIFICATE_VALIDATION); + LOG.info("Disable certificate validation set to: {}", disableValidation); + } + + try { + HttpEventPublisher.Builder builder = Review comment: TODO(pablo) - review http event publisher ########## File path: sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEvent.java ########## @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.splunk; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import com.google.auto.value.AutoValue; +import com.google.gson.annotations.SerializedName; +import javax.annotation.Nullable; + +/** + * A {@link SplunkEvent} describes a single payload sent to Splunk's Http Event Collector (HEC) + * endpoint. + * + * <p>Each object represents a single event and related metadata elements such as: + * + * <ul> + * <li>time + * <li>host + * <li>source + * <li>sourceType + * <li>index + * </ul> + */ +@AutoValue +public abstract class SplunkEvent { Review comment: see the *AutoValue* section of https://beam.apache.org/documentation/programming-guide/#creating-schemas - there's also a way to provide a Builder annotation ########## File path: sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkIO.java ########## @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.splunk; + +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 com.google.auto.value.AutoValue; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.options.ValueProvider; +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.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An unbounded sink for Splunk's Http Event Collector (HEC). + * + * <p>For more information, see the online documentation at <a + * href="https://dev.splunk.com/enterprise/docs/dataapps/httpeventcollector/">Splunk HEC</a>. + * + * <h3>Writing to Splunk's HEC</h3> + * + * <p>The {@link SplunkIO} class provides a {@link PTransform} that allows writing {@link + * SplunkEvent} messages into a Splunk HEC end point. + * + * <p>It takes as an input a {@link PCollection PCollection<SplunkEvent>}, where each {@link + * SplunkEvent} represents an event to be published to HEC. + * + * <p>To configure a {@link SplunkIO}, you must provide at a minimum: + * + * <ul> + * <li>url - HEC endpoint URL. + * <li>token - HEC endpoint token. + * </ul> + * + * <p>The {@link SplunkIO} transform can be customized further by optionally specifying: + * + * <ul> + * <li>parallelism - Number of parallel requests to the HEC. + * <li>batchCount - Number of events in a single batch. + * <li>disableCertificateValidation - Whether to disable ssl validation (useful for self-signed + * certificates) + * </ul> + * + * <p>This transform will return any non-transient write failures via a {@link PCollection + * PCollection<SplunkWriteError>}, where each {@link SplunkWriteError} captures the error that + * occurred while attempting to write to HEC. These can be published to a dead-letter sink or + * reprocessed. + * + * <p>For example: + * + * <pre>{@code + * PCollection<SplunkEvent> events = ...; + * + * PCollection<SplunkWriteError> errors = + * events.apply("WriteToSplunk", + * SplunkIO.writeBuilder() Review comment: note the example of PubsubIO, where the builder is used internally as part of the implementation, but externally, the built transform is passed around: https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java#L931-L965 ########## File path: sdks/java/io/splunk/build.gradle ########## @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +plugins { + id 'org.apache.beam.module' +} +applyJavaNature(automaticModuleName: 'org.apache.beam.sdk.io.splunk') + +description = "Apache Beam :: SDKs :: Java :: IO :: Splunk" +ext.summary = "IO to write events to Splunk Http Event Collector (HEC)" + +dependencies { + compile library.java.slf4j_api + compile project(path: ":sdks:java:core", configuration: "shadow") + compile group: "com.google.code.gson", name: "gson", version: "2.8.6" + compile group: "com.google.api-client", name: "google-api-client", version: "1.30.9" + compile group: "com.google.http-client", name: "google-http-client-apache-v2", version: "1.31.0" Review comment: TODO(pablo/sameer) - perhaps add these new dependencies to the project dependencies. Ensure that they are consistent with other com.google client/api dependencies. ---------------------------------------------------------------- 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