[
https://issues.apache.org/jira/browse/BEAM-5964?focusedWorklogId=170504&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-170504
]
ASF GitHub Bot logged work on BEAM-5964:
----------------------------------------
Author: ASF GitHub Bot
Created on: 29/Nov/18 00:36
Start Date: 29/Nov/18 00:36
Worklog Time Spent: 10m
Work Description: chamikaramj commented on issue #7006: [BEAM-5964] Add
ClickHouseIO.Write
URL: https://github.com/apache/beam/pull/7006#issuecomment-442660230
Regarding fusion, assume there is step A before ParDo(WriteFn). Some runners
(for example, Dataflow) may fuse these steps. So if step A generate a different
set of data in a rerun (due to a failure) we may insert two different sets of
data. The way to fix this is to break fusion. This can be done by adding a
Reshuffle.viaRandomKey() step before writing.
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java#L64
It should be fine to do retries within a DoFn as long as this does not
result in duplicate data in the data store.
In fact, this is encouraged, (for example, you might want to use exponential
backoff in retries to prevent the data store from being overloaded). Bundle
level retries (four times for batch) is a runner specific last resort effort
and IO connectors should not depend on this behavior.
----------------------------------------------------------------
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: 170504)
Time Spent: 4h 40m (was: 4.5h)
> Add ClickHouseIO.Write
> ----------------------
>
> Key: BEAM-5964
> URL: https://issues.apache.org/jira/browse/BEAM-5964
> Project: Beam
> Issue Type: New Feature
> Components: io-ideas
> Reporter: Gleb Kanterov
> Assignee: Gleb Kanterov
> Priority: Major
> Time Spent: 4h 40m
> Remaining Estimate: 0h
>
> h3. Motivation
> ClickHouse is open-source columnar DBMS for OLAP. It allows analysis of data
> that is updated in real time. The project was released as open-source
> software under the Apache 2 license in June 2016.
> h3. Design and implementation
> 1. Do only writes, reads aren't useful because ClickHouse is designed for
> OLAP queries
> 2. For writes, do write in batches and rely on idempotent and atomic inserts
> supported by replicated tables in ClickHouse
> 3. Implement ClickHouseIO.Write as PTransform<PCollection<Row>, PDone>
> 4. Rely on having logic for casting rows between schemas in BEAM-5918, and
> don't put it in ClickHouseIO.Write
> h3. References
> [1]
> http://highscalability.com/blog/2017/9/18/evolution-of-data-structures-in-yandexmetrica.html
> [2]
> https://blog.cloudflare.com/how-cloudflare-analyzes-1m-dns-queries-per-second/
> [3]
> https://blog.cloudflare.com/http-analytics-for-6m-requests-per-second-using-clickhouse/
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)