[
https://issues.apache.org/jira/browse/BEAM-5309?focusedWorklogId=156919&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-156919
]
ASF GitHub Bot logged work on BEAM-5309:
----------------------------------------
Author: ASF GitHub Bot
Created on: 22/Oct/18 18:42
Start Date: 22/Oct/18 18:42
Worklog Time Spent: 10m
Work Description: b923 commented on a change in pull request #6691:
WIP:[BEAM-5309] Add streaming support for HadoopFormatIO
URL: https://github.com/apache/beam/pull/6691#discussion_r226975441
##########
File path:
sdks/java/io/hadoop-format/src/main/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIO.java
##########
@@ -0,0 +1,1247 @@
+/*
+ * 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.hadoop.format;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.Iterables;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Random;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.CombineFnBase;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link HadoopFormatIO} is a Transform for writing data to any sink which
implements Hadoop
+ * {@link OutputFormat}. For example - Cassandra, Elasticsearch, HBase, Redis,
Postgres etc. {@link
+ * HadoopFormatIO} has to make several performance trade-offs in connecting to
{@link OutputFormat},
+ * so if there is another Beam IO Transform specifically for connecting to
your data sink of choice,
+ * we would recommend using that one, but this IO Transform allows you to
connect to many data sinks
+ * that do not yet have a Beam IO Transform.
+ *
+ * <p>You will need to pass a Hadoop {@link Configuration} with parameters
specifying how the write
+ * will occur. Many properties of the Configuration are optional, and some are
required for certain
+ * {@link OutputFormat} classes, but the following properties must be set for
all OutputFormats:
+ *
+ * <ul>
+ * <li>{@code mapreduce.job.outputformat.class}: The {@link OutputFormat}
class used to connect to
+ * your data sink of choice.
+ * <li>{@code mapreduce.job.output.key.class}: The key class passed to the
{@link OutputFormat} in
+ * {@code mapreduce.job.outputformat.class}.
+ * <li>{@code mapreduce.job.output.value.class}: The value class passed to
the {@link
+ * OutputFormat} in {@code mapreduce.job.outputformat.class}.
+ * <li>{@code mapreduce.job.reduces}: Number of reduce tasks. Value is equal
to number of write
+ * tasks which will be genarated. This property is not required for
{@link
+ * Write.Builder#withConfigurationWithoutPartitioning(Configuration)}
write.
+ * <li>{@code mapreduce.job.partitioner.class}: Hadoop partitioner class
which will be used for
+ * distributing of records among partitions. This property is not
required for {@link
+ * Write.Builder#withConfigurationWithoutPartitioning(Configuration)}
write.
+ * </ul>
+ *
+ * <b>Note:</b> All mentioned values have appropriate constants. E.g.: {@link
+ * #OUTPUT_FORMAT_CLASS_ATTR}.
+ *
+ * <p>For example:
+ *
+ * <pre>{@code
+ * Configuration myHadoopConfiguration = new Configuration(false);
+ * // Set Hadoop OutputFormat, key and value class in configuration
+ * myHadoopConfiguration.setClass("mapreduce.job.outputformat.class",
+ * MyDbOutputFormatClass, OutputFormat.class);
+ * myHadoopConfiguration.setClass("mapreduce.job.output.key.class",
+ * MyDbOutputFormatKeyClass, Object.class);
+ * myHadoopConfiguration.setClass("mapreduce.job.output.value.class",
+ * MyDbOutputFormatValueClass, Object.class);
+ * myHadoopConfiguration.setClass("mapreduce.job.output.value.class",
+ * MyPartitionerClass, Object.class);
+ * myHadoopConfiguration.setInt("mapreduce.job.reduces", 2);
+ * }</pre>
+ *
+ * <p>You will need to set OutputFormat key and value class (i.e.
"mapreduce.job.output.key.class"
+ * and "mapreduce.job.output.value.class") in Hadoop {@link Configuration}
which are equal to {@code
+ * KeyT} and {@code ValueT}. If you set different OutputFormat key or value
class than
+ * OutputFormat's actual key or value class then, it will throw {@link
IllegalArgumentException}
+ *
+ * <h3>Writing using {@link HadoopFormatIO}</h3>
+ *
+ * <pre>{@code
+ * Pipeline p = ...; // Create pipeline.
+ * // Read data only with Hadoop configuration.
+ * p.apply("read",
+ * HadoopFormatIO.<OutputFormatKeyClass, OutputFormatKeyClass>write()
+ * .withConfiguration(myHadoopConfiguration);
+ * }</pre>
+ */
+@Experimental(Experimental.Kind.SOURCE_SINK)
+public class HadoopFormatIO {
+ private static final Logger LOGGER =
LoggerFactory.getLogger(HadoopFormatIO.class);
+
+ public static final String OUTPUT_FORMAT_CLASS_ATTR =
MRJobConfig.OUTPUT_FORMAT_CLASS_ATTR;
+ public static final String OUTPUT_KEY_CLASS = MRJobConfig.OUTPUT_KEY_CLASS;
+ public static final String OUTPUT_VALUE_CLASS =
MRJobConfig.OUTPUT_VALUE_CLASS;
+ public static final String NUM_REDUCES = MRJobConfig.NUM_REDUCES;
+ public static final String PARTITIONER_CLASS_ATTR =
MRJobConfig.PARTITIONER_CLASS_ATTR;
+
+ /**
+ * Creates an uninitialized {@link HadoopFormatIO.Write.Builder}. Before
use, the {@code Write}
+ * must be initialized with a {@link
Write.Builder#withConfiguration(Configuration)} or {@link
+ * Write.Builder#withConfigurationTransform(ConfigurationTransform)} or
{@link
+ * Write.Builder#withConfigurationWithoutPartitioning(Configuration)} that
specifies the sink.
+ *
+ * @param <KeyT> Type of keys to be written.
+ * @param <ValueT> Type of values to be written.
+ * @return Write builder
+ */
+ public static <KeyT, ValueT> Write.Builder<KeyT, ValueT> write() {
+ return new AutoValue_HadoopFormatIO_Write.Builder<>();
+ }
+
+ /**
+ * Interface for client definition of so called {@link Configuration}
"Map-Reduce" operation
+ * defined by methods {@link #getConfigTransform()} and {@link
#getConfigCombineFn()}
+ *
+ * <p>Client can define operations which will produce one particular
configuration from the input
+ * data by this interface. Generated configuration will be then used during
writing of data into
+ * one of the hadoop output formats.
+ *
+ * <p>This interface enables defining of special {@link Configuration} for
every particular
+ * window.
+ *
+ * @param <KeyT> Key type of writing data
+ * @param <ValueT> Value type of writing data
+ */
+ @FunctionalInterface
+ interface ConfigurationTransform<KeyT, ValueT> {
+
+ /** Default "reduce" function for extraction of one Configuration. */
+ Combine.IterableCombineFn<Configuration> DEFAULT_CONFIG_COMBINE_FN =
+ Combine.IterableCombineFn.of(
+ (configurations) -> {
+ Iterable<Configuration> filtered =
Iterables.filter(configurations, Objects::nonNull);
+ return Iterables.getFirst(filtered, null);
+ });
+
+ /**
+ * "Map" function which should transform one {@link KV} pair into hadoop
{@link Configuration}.
+ *
+ * <p><b>Note:</b> Default implementation of {@link #getConfigCombineFn()}
requires that from
+ * {@link KV} pair will be produced at least one {@link Configuration}
+ *
+ * @return transform function
+ */
+ PTransform<PCollection<? extends KV<KeyT, ValueT>>,
PCollection<Configuration>>
+ getConfigTransform();
+
+ /**
+ * "Reduce" function which collects all {@link Configuration}s created via
{@link
+ * #getConfigTransform()} and returns only one particular configuration
that will be used for
+ * storing of all {@link KV} pairs.
+ *
+ * @see #DEFAULT_CONFIG_COMBINE_FN
+ * @return Combine function
+ */
+ default CombineFnBase.GlobalCombineFn<Configuration, ?, Configuration>
getConfigCombineFn() {
+ return DEFAULT_CONFIG_COMBINE_FN;
+ }
+ }
+
+ /**
+ * Default implementation of Configuration transform. It requires only
particular {@link
+ * PTransform} to be specified.
+ *
+ * @param <KeyT> Key type which should be written
+ * @param <ValueT> Value type which should be written
+ */
+ private static class DefaultConfigurationTransform<KeyT, ValueT>
+ implements ConfigurationTransform<KeyT, ValueT> {
+
+ private PTransform<PCollection<? extends KV<KeyT, ValueT>>,
PCollection<Configuration>>
+ configTransform;
+
+ public DefaultConfigurationTransform(
+ PTransform<PCollection<? extends KV<KeyT, ValueT>>,
PCollection<Configuration>>
+ configTransform) {
+ this.configTransform = configTransform;
+ }
+
+ @Override
+ public PTransform<PCollection<? extends KV<KeyT, ValueT>>,
PCollection<Configuration>>
+ getConfigTransform() {
+ return configTransform;
+ }
+ }
+
+ /**
+ * Generates tasks for output pairs and groups them by this key.
+ *
+ * <p>This transformation is used when is configured write with partitioning.
+ *
+ * @param <KeyT> type of key
+ * @param <ValueT> type of value
+ */
+ private static class GroupDataByPartition<KeyT, ValueT>
+ extends PTransform<
+ PCollection<KV<KeyT, ValueT>>, PCollection<KV<Integer, KV<KeyT,
ValueT>>>> {
+
+ private PCollectionView<Configuration> configView;
+
+ private GroupDataByPartition(PCollectionView<Configuration> configView) {
+ this.configView = configView;
+ }
+
+ @Override
+ public PCollection<KV<Integer, KV<KeyT, ValueT>>>
expand(PCollection<KV<KeyT, ValueT>> input) {
+ return input
+ .apply(
+ "AssignTask",
+ ParDo.of(new AssignTaskFn<KeyT,
ValueT>(configView)).withSideInputs(configView))
+ .setTypeDescriptor(
+ TypeDescriptors.kvs(TypeDescriptors.integers(),
input.getTypeDescriptor()))
+ .apply("GroupByTaskId", GroupByKey.create())
+ .apply("FlattenGroupedTasks", ParDo.of(new FlattenGroupedTasks<>()));
+ }
+ }
+
+ /**
+ * Flattens grouped iterable {@link KV} pairs into triplets of
TaskID/Key/Value.
+ *
+ * @param <KeyT> Type of keys to be written.
+ * @param <ValueT> Type of values to be written.
+ */
+ private static class FlattenGroupedTasks<KeyT, ValueT>
+ extends DoFn<KV<Integer, Iterable<KV<KeyT, ValueT>>>, KV<Integer,
KV<KeyT, ValueT>>> {
+
+ @ProcessElement
+ public void processElement(
+ @Element KV<Integer, Iterable<KV<KeyT, ValueT>>> input,
+ OutputReceiver<KV<Integer, KV<KeyT, ValueT>>> outputReceiver) {
+ final Integer key = input.getKey();
+ for (KV<KeyT, ValueT> element :
+ requireNonNull(input.getValue(), "Iterable can not be null.")) {
+ outputReceiver.output(KV.of(key, element));
+ }
+ }
+ }
+
+ /**
+ * A {@link PTransform} that writes to any data sink which implements Hadoop
OutputFormat. For
+ * e.g. Cassandra, Elasticsearch, HBase, Redis, Postgres, etc. See the
class-level Javadoc on
+ * {@link HadoopFormatIO} for more information.
+ *
+ * @param <KeyT> Type of keys to be written.
+ * @param <ValueT> Type of values to be written.
+ * @see HadoopFormatIO
+ */
+ @AutoValue
+ public abstract static class Write<KeyT, ValueT>
+ extends PTransform<PCollection<KV<KeyT, ValueT>>, PDone> {
+
+ @Nullable
+ public abstract Configuration getConfiguration();
+
+ @Nullable
+ public abstract ConfigurationTransform<KeyT, ValueT> getConfigTransform();
+
+ public abstract boolean isWithPartitioning();
+
+ /**
+ * Builder for Write transformation.
+ *
+ * @param <KeyT> Key write type
+ * @param <ValueT> Value write type
+ */
+ @AutoValue.Builder
+ public abstract static class Builder<KeyT, ValueT> {
+
+ public abstract Builder<KeyT, ValueT> setConfiguration(Configuration
newConfiguration);
+
+ public abstract Builder<KeyT, ValueT> setConfigTransform(
+ ConfigurationTransform<KeyT, ValueT> newConfigTransform);
+
+ public abstract Builder<KeyT, ValueT> setWithPartitioning(boolean
newWithPartitioning);
+
+ abstract Write<KeyT, ValueT> build();
+
+ /**
+ * Writes to the sink using the options provided by the given hadoop
configuration.
+ *
+ * <p><b>Note:</b> Works only for {@link
+ * org.apache.beam.sdk.values.PCollection.IsBounded#BOUNDED} {@link
PCollection} with global
+ * {@link WindowingStrategy}.
+ *
+ * @param configuration hadoop configuration.
+ * @return Created write function
+ * @throws IllegalArgumentException when the configuration is null
+ */
+ @SuppressWarnings("unchecked")
+ public Write<KeyT, ValueT> withConfiguration(Configuration configuration)
+ throws IllegalArgumentException {
+ checkArgument(Objects.nonNull(configuration), "Configuration can not
be null");
+
+ return setConfiguration(new
Configuration(configuration)).setWithPartitioning(true).build();
+ }
+
+ /**
+ * Writes to the sink without need to partition output into specified
number of partitions.
+ *
+ * <p>This write operation doesn't do shuffle by the partition so it
saves transfer time
+ * before write operation itself. As a consequence it generates random
number of partitions.
+ *
+ * <p><b>Note:</b> Works only for {@link
+ * org.apache.beam.sdk.values.PCollection.IsBounded#BOUNDED} {@link
PCollection} with global
+ * {@link WindowingStrategy}.
+ *
+ * @param configuration hadoop configuration
+ * @return Created write function
+ * @throws IllegalArgumentException when the configuration is null
+ */
+ public Write<KeyT, ValueT>
withConfigurationWithoutPartitioning(Configuration configuration)
+ throws IllegalArgumentException {
+ checkArgument(Objects.nonNull(configuration), "Configuration can not
be null");
+
+ return setConfiguration(new Configuration(configuration))
+ .setWithPartitioning(false)
+ .build();
+ }
+
+ /**
+ * Writes to the sink using configuration created by provided {@code
+ * configurationTransformation}.
+ *
+ * <p>Parameter {@code configurationTransformation} should provide way
how to transform input
+ * data into {@link PCollection} of hadoop configurations.
+ *
+ * <p>PCollection of configuration is then reduced to one by {@link
+ * ConfigurationTransform#DEFAULT_CONFIG_COMBINE_FN}
+ *
+ * <p>This type is useful especially for processing unbounded windowed
data but can be used *
Review comment:
Deleted, thank you
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 156919)
Time Spent: 4h 10m (was: 4h)
> Add streaming support for HadoopOutputFormatIO
> ----------------------------------------------
>
> Key: BEAM-5309
> URL: https://issues.apache.org/jira/browse/BEAM-5309
> Project: Beam
> Issue Type: Sub-task
> Components: io-java-hadoop
> Reporter: Alexey Romanenko
> Assignee: David Hrbacek
> Priority: Minor
> Time Spent: 4h 10m
> Remaining Estimate: 0h
>
> design doc: https://s.apache.org/beam-streaming-hofio
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)