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

ASF GitHub Bot logged work on BEAM-5964:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 29/Nov/18 00:28
            Start Date: 29/Nov/18 00:28
    Worklog Time Spent: 10m 
      Work Description: kanterov commented on issue #7006: [BEAM-5964] Add 
ClickHouseIO.Write
URL: https://github.com/apache/beam/pull/7006#issuecomment-442658835
 
 
   @chamikaramj thanks! Everything makes sense, but I'm not sure about 
buffering.
   
   Let me clarify regarding deduplication first. ClickHouse itself does 
deduplication, and clients don't need to do deduplication, as soon as they 
insert blocks with the same rows in the same order. However, I don't clearly 
understand if there is a problem due to fusion and bundle splitting.
   
   Regarding buffering and flushing, the last time I tried it, it didn't work 
as well as streaming implemented in this PR. I'm a bit concerned to keep a huge 
buffer in memory because our typical bundles have millions of elements, and 
each row could have up to 100 fields. There is an option of limiting buffer 
size, but then we should retry flushing buffer because it can fail, but it 
doesn't feel right to do retries within DoFn, because DoFn itself is retriable.
   
   

----------------------------------------------------------------
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: 170503)
    Time Spent: 4.5h  (was: 4h 20m)

> 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: 4.5h
>  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)

Reply via email to