[ 
https://issues.apache.org/jira/browse/BEAM-5964?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16683742#comment-16683742
 ] 

Ismaël Mejía commented on BEAM-5964:
------------------------------------

Excellent for the binary protocol! The real deal is if partitioning is exposed 
in the client APIs and if the IO connector can benefit of it.
Yes, SDF is one of the two APIs to Read on Beam (the other one is the Source 
API), it is basically a DoFn that knows how to split itself (into new 
'partitions'), there is not a lot of documentation on how to write SDF based 
connectors yet (you can check at 
[HBaseIO.readAll|https://github.com/apache/beam/blob/8c99d2f527bdb952350f0ab91c7b792bcff0b554/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java#L296]
 for an example for Bounded reads, but don't hesitate to ask if you have doubts.
For writing you just go with the classical DoFn. Starting with the write is 
probably easier. Is there a way to test this easily in Java I mean like an 
embedded server? Otherwise you may need to simulate it or mock it for the Unit 
tests.


> 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
>
> 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)

Reply via email to