Repository: samza Updated Branches: refs/heads/master 608e4e0b7 -> cb92cf18b
SAMZA-1512: Documentation on the multi-stage batch processing Add overview documentation to explain how partitionBy(), checkpoint and state works in batch. Also organized the existing hdfs consumer/producer docs into the same hadoop folder under documentation. Author: xinyuiscool <[email protected]> Reviewers: Jake Maes <[email protected]> Closes #381 from xinyuiscool/SAMZA-1512 Project: http://git-wip-us.apache.org/repos/asf/samza/repo Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/cb92cf18 Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/cb92cf18 Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/cb92cf18 Branch: refs/heads/master Commit: cb92cf18b5b1707d98a442a32a9f6c3767e13527 Parents: 608e4e0 Author: Xinyu Liu <[email protected]> Authored: Tue Dec 12 17:07:07 2017 -0800 Committer: xiliu <[email protected]> Committed: Tue Dec 12 17:07:07 2017 -0800 ---------------------------------------------------------------------- .../documentation/hadoop/multi_stage_batch.png | Bin 0 -> 65364 bytes .../hadoop/unified_batch_streaming.png | Bin 0 -> 17371 bytes .../documentation/versioned/hadoop/consumer.md | 110 +++++++++++++++++++ .../documentation/versioned/hadoop/overview.md | 46 ++++++++ .../documentation/versioned/hadoop/producer.md | 68 ++++++++++++ .../documentation/versioned/hdfs/consumer.md | 110 ------------------- .../documentation/versioned/hdfs/producer.md | 70 ------------ docs/learn/documentation/versioned/index.html | 15 ++- .../versioned/jobs/configuration-table.html | 4 +- .../versioned/yarn/yarn-security.md | 1 - 10 files changed, 235 insertions(+), 189 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/samza/blob/cb92cf18/docs/img/versioned/learn/documentation/hadoop/multi_stage_batch.png ---------------------------------------------------------------------- diff --git a/docs/img/versioned/learn/documentation/hadoop/multi_stage_batch.png b/docs/img/versioned/learn/documentation/hadoop/multi_stage_batch.png new file mode 100644 index 0000000..1142540 Binary files /dev/null and b/docs/img/versioned/learn/documentation/hadoop/multi_stage_batch.png differ http://git-wip-us.apache.org/repos/asf/samza/blob/cb92cf18/docs/img/versioned/learn/documentation/hadoop/unified_batch_streaming.png ---------------------------------------------------------------------- diff --git a/docs/img/versioned/learn/documentation/hadoop/unified_batch_streaming.png b/docs/img/versioned/learn/documentation/hadoop/unified_batch_streaming.png new file mode 100644 index 0000000..0fd2899 Binary files /dev/null and b/docs/img/versioned/learn/documentation/hadoop/unified_batch_streaming.png differ http://git-wip-us.apache.org/repos/asf/samza/blob/cb92cf18/docs/learn/documentation/versioned/hadoop/consumer.md ---------------------------------------------------------------------- diff --git a/docs/learn/documentation/versioned/hadoop/consumer.md b/docs/learn/documentation/versioned/hadoop/consumer.md new file mode 100644 index 0000000..653dcf9 --- /dev/null +++ b/docs/learn/documentation/versioned/hadoop/consumer.md @@ -0,0 +1,110 @@ +--- +layout: page +title: Reading from HDFS +--- +<!-- + 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. +--> + +You can configure your Samza job to read from HDFS files. The [HdfsSystemConsumer](https://github.com/apache/samza/blob/master/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java) can read from HDFS files. Avro encoded records are supported out of the box and it is easy to extend to support other formats (plain text, csv, json etc). See `Event format` section below. + +### Environment + +Your job needs to run on the same YARN cluster which hosts the HDFS you want to consume from. + +### Partitioning + +Partitioning works at the level of individual HDFS files. Each file is treated as a stream partition, while a directory that contains these files is a stream. For example, if you want to read from a HDFS path which contains 10 individual files, there will naturally be 10 partitions created. You can configure up to 10 Samza containers to process these partitions. If you want to read from a single HDFS file, there is currently no way to break down the consumption - you can only have one container to process the file. + +### Event format + +[HdfsSystemConsumer](https://github.com/apache/samza/blob/master/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java) currently supports reading from avro files. The received [IncomingMessageEnvelope](../api/javadocs/org/apache/samza/system/IncomingMessageEnvelope.html) contains three significant fields: + +1. The key which is empty +2. The message which is set to the avro [GenericRecord](https://avro.apache.org/docs/1.7.6/api/java/org/apache/avro/generic/GenericRecord.html) +3. The stream partition which is set to the name of the HDFS file + +To extend the support beyond avro files (e.g. json, csv, etc.), you can implement the interface [SingleFileHdfsReader](https://github.com/apache/samza/blob/master/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/reader/SingleFileHdfsReader.java) (take a look at the implementation of [AvroFileHdfsReader](https://github.com/apache/samza/blob/master/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/reader/AvroFileHdfsReader.java) as a sample). + +### End of stream support + +One major difference between HDFS data and Kafka data is that while a kafka topic has an unbounded stream of messages, HDFS files are bounded and have a notion of EOF. + +You can choose to implement [EndOfStreamListenerTask](../api/javadocs/org/apache/samza/task/EndOfStreamListenerTask.html) to receive a callback when all partitions are at end of stream. When all partitions being processed by the task are at end of stream (i.e. EOF has been reached for all files), the Samza job exits automatically. + +### Basic Configuration + +Here is a few of the basic configs to set up HdfsSystemConsumer: + +``` +# The HDFS system consumer is implemented under the org.apache.samza.system.hdfs package, +# so use HdfsSystemFactory as the system factory for your system +systems.hdfs-clickstream.samza.factory=org.apache.samza.system.hdfs.HdfsSystemFactory + +# You need to specify the path of files you want to consume in task.inputs +task.inputs=hdfs-clickstream.hdfs:/data/clickstream/2016/09/11 + +# You can specify a white list of files you want your job to process (in Java Pattern style) +systems.hdfs-clickstream.partitioner.defaultPartitioner.whitelist=.*avro + +# You can specify a black list of files you don't want your job to process (in Java Pattern style), +# by default it's empty. +# Note that you can have both white list and black list, in which case both will be applied. +systems.hdfs-clickstream.partitioner.defaultPartitioner.blacklist=somefile.avro + +``` + +### Security Configuration + +The following additional configs are required when accessing HDFS clusters that have kerberos enabled: + +``` +# Use the SamzaYarnSecurityManagerFactory, which fetches and renews the Kerberos delegation tokens when the job is running in a secure environment. +job.security.manager.factory=org.apache.samza.job.yarn.SamzaYarnSecurityManagerFactory + +# Kerberos principal +yarn.kerberos.principal=your-principal-name + +# Path of the keytab file (local path) +yarn.kerberos.keytab=/tmp/keytab +``` + +### Advanced Configuration + +Some of the advanced configuration you might need to set up: + +``` +# Specify the group pattern for advanced partitioning. +systems.hdfs-clickstream.partitioner.defaultPartitioner.groupPattern=part-[id]-.* +``` + +The advanced partitioning goes beyond the basic assumption that each file is a partition. With advanced partitioning you can group files into partitions arbitrarily. For example, if you have a set of files as [part-01-a.avro, part-01-b.avro, part-02-a.avro, part-02-b.avro, part-03-a.avro] that you want to organize into three partitions as (part-01-a.avro, part-01-b.avro), (part-02-a.avro, part-02-b.avro), (part-03-a.avro), where the numbers in the middle act as a "group identifier", you can then set this property to be "part-[id]-.*" (note that **[id]** is a reserved term here, i.e. you have to literally put it as **[id]**). The partitioner will apply this pattern to all file names and extract the "group identifier" ("[id]" in the pattern), then use the "group identifier" to group files into partitions. + +``` +# Specify the type of files your job want to process (support avro only for now) +systems.hdfs-clickstream.consumer.reader=avro + +# Max number of retries (per-partition) before the container fails. +system.hdfs-clickstream.consumer.numMaxRetries=10 + +``` + +For the list of all configs, check out the configuration table page [here](../jobs/configuration-table.html) + +### More Information +[HdfsSystemConsumer design doc](https://issues.apache.org/jira/secure/attachment/12827670/HDFSSystemConsumer.pdf) + +## [Writing to HDFS »](./producer.html) \ No newline at end of file http://git-wip-us.apache.org/repos/asf/samza/blob/cb92cf18/docs/learn/documentation/versioned/hadoop/overview.md ---------------------------------------------------------------------- diff --git a/docs/learn/documentation/versioned/hadoop/overview.md b/docs/learn/documentation/versioned/hadoop/overview.md new file mode 100644 index 0000000..0820127 --- /dev/null +++ b/docs/learn/documentation/versioned/hadoop/overview.md @@ -0,0 +1,46 @@ +--- +layout: page +title: Batch Processing Overview +--- +<!-- + 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. +--> + +Samza provides a unified data processing model for both stream and batch processing. The primary difference between batch and streaming is whether the input size is bounded or unbounded. Batch data sources are typically bounded (e.g. static files on HDFS), whereas streams are unbounded (e.g. a topic in Kafka). Under the hood, the same highly-efficient stream-processing engine handles both types. + +<img src="/img/{{site.version}}/learn/documentation/hadoop/unified_batch_streaming.png" alt="Unified Batch and Streaming" style="max-width: 100%; height: auto;" onclick="window.open(this.src)"> + +### Unified API for Batch and Streaming + +Samza provides a single set of APIs for both batch and stream processing. This unified programming API makes it convenient for you to focus on the processing logic, without treating bounded and unbounded sources differently. Switching between batch and streaming only requires config change, e.g. [Kafka](../api/overview.html) to [HDFS](./consumer.html), instead of any code change. + +### Multi-stage Batch Pipeline + +Complex data pipelines usually consist multiple stages, with data shuffled (repartitioned) between stages to enable key-based operations such as windowing, aggregation, and join. Samza [high-level API](/startup/preview/index.html) provides an operator named `partitionBy` to create such multi-stage pipelines. Internally, Samza creates a physical stream, called an âintermediate streamâ, based on the system configured as in `job.default.system`. Samza repartitions the output of the previous stage by sending it to the intermediate stream with the appropriate partition count and partition key. It then feeds it to the next stage of the pipeline. The lifecycle of intermediate streams is completely managed by Samza so from the user perspective the data shuffling is automatic. + +For a single-stage pipeline, dealing with bounded data sets is straightforward: the system consumer âknowsâ the end of a particular partition, and it will emit end-of-stream token once a partition is complete. Samza will shut down the container when all its input partitions are complete. + +For a multi-stage pipeline, however, things become tricky since intermediate streams are often physically unbounded data streams, e.g. Kafka, and the downstream stages don't know when to shut down since unbounded streams don't have an end. To solve this problem, Samza uses in-band end-of-stream control messages in the intermediate stream along with user data messages. The upstream stage broadcasts end-of-stream control messages to every partition of the intermediate stream, and the downstream stage will aggregate the end-of-stream messages for each partition. When one end-of-stream message has been received for every upstream task in a partition, the downstream stage will conclude that the partition has no more messages, and the task will shut down. For pipelines with more than 2 stages, the end-of-stream control messages will be propagated from the source to the last stage, and each stage will perform the end-of-stream aggregation and then shuts down. The following diagram shows th e flow: + +<img src="/img/{{site.version}}/learn/documentation/hadoop/multi_stage_batch.png" alt="Multi-stage Batch Processing" style="max-width: 100%; height: auto;" onclick="window.open(this.src)"> + +### State and Fault-tolerance + +Samzaâs [state management](../container/state-management.html) and [fault-tolerance](../container/checkpointing.html) apply the same to batch. You can use in-memory or RocksDb as your local state store which can be persisted by changelog streams. In case of any container failures, Samza will restart the container by reseeding the local store from changelog streams, and resume processing from the previous checkpoints. + +During a job restart, batch processing behaves completely different from streaming. In batch, it is expected to be a re-run and all the internal streams, including intermediate, checkpoint and changelog streams, need to be fresh. Since some systems only support retention-based stream cleanup, e.g. Kafka without deletion enabled, Samza creates a new set of internal streams for each job run. To achieve this, Samza internally generates a unique **run.id** to each job run. The **run.id** is appended to the physical names of the internal streams, which will be used in the job in each run. Samza also performs due diligence to delete/purge the streams from previous run. The cleanup happens when the job is restarted. + +## [Reading from HDFS »](./consumer.html) \ No newline at end of file http://git-wip-us.apache.org/repos/asf/samza/blob/cb92cf18/docs/learn/documentation/versioned/hadoop/producer.md ---------------------------------------------------------------------- diff --git a/docs/learn/documentation/versioned/hadoop/producer.md b/docs/learn/documentation/versioned/hadoop/producer.md new file mode 100644 index 0000000..a94d5de --- /dev/null +++ b/docs/learn/documentation/versioned/hadoop/producer.md @@ -0,0 +1,68 @@ +--- +layout: page +title: Writing to HDFS +--- +<!-- + 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. +--> + +The `samza-hdfs` module implements a Samza Producer to write to HDFS. The current implementation includes a ready-to-use `HdfsSystemProducer`, and three `HdfsWriter`s: One that writes messages of raw bytes to a `SequenceFile` of `BytesWritable` keys and values. Another writes UTF-8 `String`s to a `SequenceFile` with `LongWritable` keys and `Text` values. +The last one writes out Avro data files including the schema automatically reflected from the POJO objects fed to it. + +### Configuring an HdfsSystemProducer + +You can configure an HdfsSystemProducer like any other Samza system: using configuration keys and values set in a `job.properties` file. +You might configure the system producer for use by your `StreamTasks` like this: + +``` +# set the SystemFactory implementation to instantiate HdfsSystemProducer aliased to 'hdfs-clickstream' +systems.hdfs-clickstream.samza.factory=org.apache.samza.system.hdfs.HdfsSystemFactory + +# define a serializer/deserializer for the hdfs-clickstream system +# DO NOT define (i.e. comment out) a SerDe when using the AvroDataFileHdfsWriter so it can reflect the schema +systems.hdfs-clickstream.samza.msg.serde=some-serde-impl + +# consumer configs not needed for HDFS system, reader is not implemented yet + +# Assign a Metrics implementation via a label we defined earlier in the props file +systems.hdfs-clickstream.streams.metrics.samza.msg.serde=some-metrics-impl + +# Assign the implementation class for this system's HdfsWriter +systems.hdfs-clickstream.producer.hdfs.writer.class=org.apache.samza.system.hdfs.writer.TextSequenceFileHdfsWriter +#systems.hdfs-clickstream.producer.hdfs.writer.class=org.apache.samza.system.hdfs.writer.AvroDataFileHdfsWriter + +# Set compression type supported by chosen Writer. Only BLOCK compression is supported currently +# AvroDataFileHdfsWriter supports snappy, bzip2, deflate or none (null, anything other than the first three) +systems.hdfs-clickstream.producer.hdfs.compression.type=snappy + +# The base dir for HDFS output. The default Bucketer for SequenceFile HdfsWriters +# is currently /BASE/JOB_NAME/DATE_PATH/FILES, where BASE is set below +systems.hdfs-clickstream.producer.hdfs.base.output.dir=/user/me/analytics/clickstream_data + +# Assign the implementation class for the HdfsWriter's Bucketer +systems.hdfs-clickstream.producer.hdfs.bucketer.class=org.apache.samza.system.hdfs.writer.JobNameDateTimeBucketer + +# Configure the DATE_PATH the Bucketer will set to bucket output files by day for this job run. +systems.hdfs-clickstream.producer.hdfs.bucketer.date.path.format=yyyy_MM_dd + +# Optionally set the max output bytes (records for AvroDataFileHdfsWriter) per file. +# A new file will be cut and output continued on the next write call each time this many bytes +# (records for AvroDataFileHdfsWriter) are written. +systems.hdfs-clickstream.producer.hdfs.write.batch.size.bytes=134217728 +#systems.hdfs-clickstream.producer.hdfs.write.batch.size.records=10000 +``` + +The above configuration assumes a Metrics and Serde implemnetation has been properly configured against the `some-serde-impl` and `some-metrics-impl` labels somewhere else in the same `job.properties` file. Each of these properties has a reasonable default, so you can leave out the ones you don't need to customize for your job run. http://git-wip-us.apache.org/repos/asf/samza/blob/cb92cf18/docs/learn/documentation/versioned/hdfs/consumer.md ---------------------------------------------------------------------- diff --git a/docs/learn/documentation/versioned/hdfs/consumer.md b/docs/learn/documentation/versioned/hdfs/consumer.md deleted file mode 100644 index 41a1a51..0000000 --- a/docs/learn/documentation/versioned/hdfs/consumer.md +++ /dev/null @@ -1,110 +0,0 @@ ---- -layout: page -title: Reading from HDFS ---- -<!-- - 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. ---> - -You can configure your Samza job to read from HDFS files. The [HdfsSystemConsumer](https://github.com/apache/samza/blob/master/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java) can read from HDFS files. Avro encoded records are supported out of the box and it is easy to extend to support other formats (plain text, csv, json etc). See `Event format` section below. - -### Environment - -Your job needs to run on the same YARN cluster which hosts the HDFS you want to consume from. - -### Partitioning - -Partitioning works at the level of individual HDFS files. Each file is treated as a stream partition, while a directory that contains these files is a stream. For example, if you want to read from a HDFS path which contains 10 individual files, there will naturally be 10 partitions created. You can configure up to 10 Samza containers to process these partitions. If you want to read from a single HDFS file, there is currently no way to break down the consumption - you can only have one container to process the file. - -### Event format - -[HdfsSystemConsumer](https://github.com/apache/samza/blob/master/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java) currently supports reading from avro files. The received [IncomingMessageEnvelope](../api/javadocs/org/apache/samza/system/IncomingMessageEnvelope.html) contains three significant fields: - -1. The key which is empty -2. The message which is set to the avro [GenericRecord](https://avro.apache.org/docs/1.7.6/api/java/org/apache/avro/generic/GenericRecord.html) -3. The stream partition which is set to the name of the HDFS file - -To extend the support beyond avro files (e.g. json, csv, etc.), you can implement the interface [SingleFileHdfsReader](https://github.com/apache/samza/blob/master/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/reader/SingleFileHdfsReader.java) (take a look at the implementation of [AvroFileHdfsReader](https://github.com/apache/samza/blob/master/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/reader/AvroFileHdfsReader.java) as a sample). - -### End of stream support - -One major difference between HDFS data and Kafka data is that while a kafka topic has an unbounded stream of messages, HDFS files are bounded and have a notion of EOF. - -You can choose to implement [EndOfStreamListenerTask](../api/javadocs/org/apache/samza/task/EndOfStreamListenerTask.html) to receive a callback when all partitions are at end of stream. When all partitions being processed by the task are at end of stream (i.e. EOF has been reached for all files), the Samza job exits automatically. - -### Basic Configuration - -Here is a few of the basic configs to set up HdfsSystemConsumer: - -``` -# The HDFS system consumer is implemented under the org.apache.samza.system.hdfs package, -# so use HdfsSystemFactory as the system factory for your system -systems.hdfs-clickstream.samza.factory=org.apache.samza.system.hdfs.HdfsSystemFactory - -# You need to specify the path of files you want to consume in task.inputs -task.inputs=hdfs-clickstream.hdfs:/data/clickstream/2016/09/11 - -# You can specify a white list of files you want your job to process (in Java Pattern style) -systems.hdfs-clickstream.partitioner.defaultPartitioner.whitelist=.*avro - -# You can specify a black list of files you don't want your job to process (in Java Pattern style), -# by default it's empty. -# Note that you can have both white list and black list, in which case both will be applied. -systems.hdfs-clickstream.partitioner.defaultPartitioner.blacklist=somefile.avro - -``` - -### Security Configuration - -The following additional configs are required when accessing HDFS clusters that have kerberos enabled: - -``` -# Use the SamzaYarnSecurityManagerFactory, which fetches and renews the Kerberos delegation tokens when the job is running in a secure environment. -job.security.manager.factory=org.apache.samza.job.yarn.SamzaYarnSecurityManagerFactory - -# Kerberos principal -yarn.kerberos.principal=your-principal-name - -# Path of the keytab file (local path) -yarn.kerberos.keytab=/tmp/keytab -``` - -### Advanced Configuration - -Some of the advanced configuration you might need to set up: - -``` -# Specify the group pattern for advanced partitioning. -systems.hdfs-clickstream.partitioner.defaultPartitioner.groupPattern=part-[id]-.* -``` - -The advanced partitioning goes beyond the basic assumption that each file is a partition. With advanced partitioning you can group files into partitions arbitrarily. For example, if you have a set of files as [part-01-a.avro, part-01-b.avro, part-02-a.avro, part-02-b.avro, part-03-a.avro] that you want to organize into three partitions as (part-01-a.avro, part-01-b.avro), (part-02-a.avro, part-02-b.avro), (part-03-a.avro), where the numbers in the middle act as a "group identifier", you can then set this property to be "part-[id]-.*" (note that **[id]** is a reserved term here, i.e. you have to literally put it as **[id]**). The partitioner will apply this pattern to all file names and extract the "group identifier" ("[id]" in the pattern), then use the "group identifier" to group files into partitions. - -``` -# Specify the type of files your job want to process (support avro only for now) -systems.hdfs-clickstream.consumer.reader=avro - -# Max number of retries (per-partition) before the container fails. -system.hdfs-clickstream.consumer.numMaxRetries=10 - -``` - -For the list of all configs, check out the configuration table page [here](../jobs/configuration-table.html) - -### More Information -[HdfsSystemConsumer design doc](https://issues.apache.org/jira/secure/attachment/12827670/HDFSSystemConsumer.pdf) - -## [Security »](../operations/security.html) \ No newline at end of file http://git-wip-us.apache.org/repos/asf/samza/blob/cb92cf18/docs/learn/documentation/versioned/hdfs/producer.md ---------------------------------------------------------------------- diff --git a/docs/learn/documentation/versioned/hdfs/producer.md b/docs/learn/documentation/versioned/hdfs/producer.md deleted file mode 100644 index a157cd8..0000000 --- a/docs/learn/documentation/versioned/hdfs/producer.md +++ /dev/null @@ -1,70 +0,0 @@ ---- -layout: page -title: Writing to HDFS ---- -<!-- - 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. ---> - -The `samza-hdfs` module implements a Samza Producer to write to HDFS. The current implementation includes a ready-to-use `HdfsSystemProducer`, and three `HdfsWriter`s: One that writes messages of raw bytes to a `SequenceFile` of `BytesWritable` keys and values. Another writes UTF-8 `String`s to a `SequenceFile` with `LongWritable` keys and `Text` values. -The last one writes out Avro data files including the schema automatically reflected from the POJO objects fed to it. - -### Configuring an HdfsSystemProducer - -You can configure an HdfsSystemProducer like any other Samza system: using configuration keys and values set in a `job.properties` file. -You might configure the system producer for use by your `StreamTasks` like this: - -``` -# set the SystemFactory implementation to instantiate HdfsSystemProducer aliased to 'hdfs-clickstream' -systems.hdfs-clickstream.samza.factory=org.apache.samza.system.hdfs.HdfsSystemFactory - -# define a serializer/deserializer for the hdfs-clickstream system -# DO NOT define (i.e. comment out) a SerDe when using the AvroDataFileHdfsWriter so it can reflect the schema -systems.hdfs-clickstream.samza.msg.serde=some-serde-impl - -# consumer configs not needed for HDFS system, reader is not implemented yet - -# Assign a Metrics implementation via a label we defined earlier in the props file -systems.hdfs-clickstream.streams.metrics.samza.msg.serde=some-metrics-impl - -# Assign the implementation class for this system's HdfsWriter -systems.hdfs-clickstream.producer.hdfs.writer.class=org.apache.samza.system.hdfs.writer.TextSequenceFileHdfsWriter -#systems.hdfs-clickstream.producer.hdfs.writer.class=org.apache.samza.system.hdfs.writer.AvroDataFileHdfsWriter - -# Set compression type supported by chosen Writer. Only BLOCK compression is supported currently -# AvroDataFileHdfsWriter supports snappy, bzip2, deflate or none (null, anything other than the first three) -systems.hdfs-clickstream.producer.hdfs.compression.type=snappy - -# The base dir for HDFS output. The default Bucketer for SequenceFile HdfsWriters -# is currently /BASE/JOB_NAME/DATE_PATH/FILES, where BASE is set below -systems.hdfs-clickstream.producer.hdfs.base.output.dir=/user/me/analytics/clickstream_data - -# Assign the implementation class for the HdfsWriter's Bucketer -systems.hdfs-clickstream.producer.hdfs.bucketer.class=org.apache.samza.system.hdfs.writer.JobNameDateTimeBucketer - -# Configure the DATE_PATH the Bucketer will set to bucket output files by day for this job run. -systems.hdfs-clickstream.producer.hdfs.bucketer.date.path.format=yyyy_MM_dd - -# Optionally set the max output bytes (records for AvroDataFileHdfsWriter) per file. -# A new file will be cut and output continued on the next write call each time this many bytes -# (records for AvroDataFileHdfsWriter) are written. -systems.hdfs-clickstream.producer.hdfs.write.batch.size.bytes=134217728 -#systems.hdfs-clickstream.producer.hdfs.write.batch.size.records=10000 -``` - -The above configuration assumes a Metrics and Serde implemnetation has been properly configured against the `some-serde-impl` and `some-metrics-impl` labels somewhere else in the same `job.properties` file. Each of these properties has a reasonable default, so you can leave out the ones you don't need to customize for your job run. - -## [Reading from HDFS »](../hdfs/consumer.html) \ No newline at end of file http://git-wip-us.apache.org/repos/asf/samza/blob/cb92cf18/docs/learn/documentation/versioned/index.html ---------------------------------------------------------------------- diff --git a/docs/learn/documentation/versioned/index.html b/docs/learn/documentation/versioned/index.html index a710383..e23b29f 100644 --- a/docs/learn/documentation/versioned/index.html +++ b/docs/learn/documentation/versioned/index.html @@ -83,12 +83,15 @@ title: Documentation <li><a href="yarn/yarn-host-affinity.html">Host Affinity & Yarn</a></li> <li><a href="yarn/yarn-resource-localization.html">Resource Localization</a></li> <li><a href="yarn/yarn-security.html">Yarn Security</a></li> - <li><a href="hdfs/producer.html">Writing to HDFS</a></li> - <li><a href="hdfs/consumer.html">Reading from HDFS</a></li> -<!-- TODO write yarn pages - <li><a href="">Fault Tolerance</a></li> - <li><a href="">Security</a></li> ---> + +</ul> + +<h4>Hadoop</h4> + +<ul class="documentation-list"> + <li><a href="hadoop/overview.html">Batch Processing Overview</a></li> + <li><a href="hadoop/consumer.html">Reading from HDFS</a></li> + <li><a href="hadoop/producer.html">Writing to HDFS</a></li> </ul> <h4>Operations</h4> http://git-wip-us.apache.org/repos/asf/samza/blob/cb92cf18/docs/learn/documentation/versioned/jobs/configuration-table.html ---------------------------------------------------------------------- diff --git a/docs/learn/documentation/versioned/jobs/configuration-table.html b/docs/learn/documentation/versioned/jobs/configuration-table.html index b6ae0d9..ef81887 100644 --- a/docs/learn/documentation/versioned/jobs/configuration-table.html +++ b/docs/learn/documentation/versioned/jobs/configuration-table.html @@ -2170,7 +2170,7 @@ </tr> <tr> - <th colspan="3" class="section" id="hdfs-system-producer"><a href="../hdfs/producer.html">Writing to HDFS</a></th> + <th colspan="3" class="section" id="hdfs-system-producer"><a href="../hadoop/producer.html">Writing to HDFS</a></th> </tr> <tr> @@ -2210,7 +2210,7 @@ </tr> <tr> - <th colspan="3" class="section" id="hdfs-system-consumer"><a href="../hdfs/consumer.html">Reading from HDFS</a></th> + <th colspan="3" class="section" id="hdfs-system-consumer"><a href="../hadoop/consumer.html">Reading from HDFS</a></th> </tr> <tr> http://git-wip-us.apache.org/repos/asf/samza/blob/cb92cf18/docs/learn/documentation/versioned/yarn/yarn-security.md ---------------------------------------------------------------------- diff --git a/docs/learn/documentation/versioned/yarn/yarn-security.md b/docs/learn/documentation/versioned/yarn/yarn-security.md index 7b66ed8..0aecefe 100644 --- a/docs/learn/documentation/versioned/yarn/yarn-security.md +++ b/docs/learn/documentation/versioned/yarn/yarn-security.md @@ -91,4 +91,3 @@ yarn.token.renewal.interval.seconds=86400 </property> {% endhighlight %} -## [Writing to HDFS »](../hdfs/producer.html)
