ifndef-SleePy commented on PR #5170: URL: https://github.com/apache/inlong/pull/5170#issuecomment-1195369139
> > Hi guys, thanks for reviewing. First, I'm sorry that I didn't explain my points behind the PR. > > As you can see, all the comments are focused on the scenario of flushing failure. I think we don't need care about the metrics after flushing failure. Because > > > > 1. If flushing fails, the job would falls into failover scenario. > > 2. Metric accuracy can not be guaranteed when failover happens. The consistency mode of `HBaseSinkFunction` is at-least-once. So some of produced records would probably be re-produced again. However the metric collection would not be reverted. All of these records would be counted. > > 3. Moreover, nobody knows how many records have been written into HBase Region Server when flushing fails. Flushing of `BufferedMutator` is not a transactional operation. If I remember correctly, HBase client spawns RPC calls to several Region Server concurrently. Any failure RPC call leads to a global failure. So when failure happens, we don't know how many RPC succeeded. It might be none, some or all records have been flushed successfully. > > > > My points here are: > > > > 1. We have to guarantee the metric accuracy if no failure happens. That's what we do in this PR. > > 2. Metric is not accurate if flushing fails, no matter what we do. So we don't need to involve any synchronization of these counters. Just make it as cheaper as possible. > > 3. A counter records the number of flushing failure times is meaningful. But I'm a bit confused of the counter name "dirty", so I removed the these ambiguous counters. Maybe we could introduce another counter to record the failure times. > > > > What do you guys think of this? @EMsnap @gong @yunqingmoswu > > As far as I know, the underlying logic of refresh should generate hfile first and then load, then refresh will only have this batch of data either all visible or all lost. If I understand correctly, then after refreshing the statistics, the data obtained will be more accurate, not only for dirty data, but also for normal sync data. 1. The `HFile` way you described is called "bulk loading" [1]. That's not the way worked here. `BufferedMutator` would not trigger bulk loading, there is another API for bulk loading. It would go through WAL and `MemStore` to be ingested into Region Server. 2. Bulk loading also could not guarantee "this batch of data either all visible or all lost", because there might be several `HFile` involved in the batch operation. 3. HBase could guarantee consistency in a row, but not cross rows, please check the ACID docs [2], "APIs that mutate several rows will _not_ be atomic across the multiple rows.". [1] https://hbase.apache.org/book.html#arch.bulk.load [2] https://hbase.apache.org/acid-semantics.html -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
