You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by GitBox <gi...@apache.org> on 2022/06/15 09:00:16 UTC

[GitHub] [shardingsphere] azexcy opened a new pull request, #18375: Feature

azexcy opened a new pull request, #18375:
URL: https://github.com/apache/shardingsphere/pull/18375

   Fixes #18075.
   
   Changes proposed in this pull request:
   - Improve decode method, not consume incomplete data
   - skip checksum, not change the source ByteBuf.
   - Improve unit test
   


-- 
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: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] menghaoranss commented on pull request #18375: Feature

Posted by GitBox <gi...@apache.org>.
menghaoranss commented on PR #18375:
URL: https://github.com/apache/shardingsphere/pull/18375#issuecomment-1156283746

   @azexcy please modify the title to describe what this PR does


-- 
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: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #18375: Fix scaling increment task consumed incomplete data

Posted by GitBox <gi...@apache.org>.
sandynz commented on code in PR #18375:
URL: https://github.com/apache/shardingsphere/pull/18375#discussion_r898627540


##########
shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/client/netty/MySQLBinlogEventPacketDecoder.java:
##########
@@ -54,39 +55,48 @@ public MySQLBinlogEventPacketDecoder(final int checksumLength) {
     
     @Override
     protected void decode(final ChannelHandlerContext ctx, final ByteBuf in, final List<Object> out) {
-        MySQLPacketPayload payload = new MySQLPacketPayload(in, ctx.channel().attr(CommonConstants.CHARSET_ATTRIBUTE_KEY).get());
-        skipSequenceId(payload);
-        checkError(payload);
-        MySQLBinlogEventHeader binlogEventHeader = new MySQLBinlogEventHeader(payload);
-        removeChecksum(binlogEventHeader.getEventType(), in);
+        // readable bytes must grete + seqId(1b) + statusCode(1b) + header-length(19b) + 
+        while (in.readableBytes() >= 2 + MySQLBinlogEventHeader.MYSQL_BINLOG_EVENT_HEADER_LENGTH) {
+            in.markReaderIndex();
+            MySQLPacketPayload payload = new MySQLPacketPayload(in, ctx.channel().attr(CommonConstants.CHARSET_ATTRIBUTE_KEY).get());
+            skipSequenceId(payload);
+            checkError(payload);
+            MySQLBinlogEventHeader binlogEventHeader = new MySQLBinlogEventHeader(payload, binlogContext.getChecksumLength());
+            // make sure event has complete body
+            if (in.readableBytes() < binlogEventHeader.getEventSize() - MySQLBinlogEventHeader.MYSQL_BINLOG_EVENT_HEADER_LENGTH) {
+                log.debug("the event body is not complete, event size={}, readable bytes={}", binlogEventHeader.getEventSize(), in.readableBytes());
+                in.resetReaderIndex();
+                break;
+            }
+            out.add(decodeEvent(payload, binlogEventHeader));
+            skipChecksum(binlogEventHeader.getEventType(), in);
+        }
+    }
+    
+    private Object decodeEvent(final MySQLPacketPayload payload, final MySQLBinlogEventHeader binlogEventHeader) {
         switch (MySQLBinlogEventType.valueOf(binlogEventHeader.getEventType())) {
             case ROTATE_EVENT:
-                decodeRotateEvent(binlogEventHeader, payload);
-                break;
+                return decodeRotateEvent(binlogEventHeader, payload);
             case FORMAT_DESCRIPTION_EVENT:
-                new MySQLBinlogFormatDescriptionEventPacket(binlogEventHeader, payload);
-                break;
+                return new MySQLBinlogFormatDescriptionEventPacket(binlogEventHeader, payload);
             case TABLE_MAP_EVENT:
-                decodeTableMapEvent(binlogEventHeader, payload);
-                break;
+                return decodeTableMapEvent(binlogEventHeader, payload);

Review Comment:
   For ROTATE_EVENT etc, keep the original logic, just return null etc.



-- 
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: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #18375: Fix scaling increment task consumed incomplete data

Posted by GitBox <gi...@apache.org>.
sandynz commented on code in PR #18375:
URL: https://github.com/apache/shardingsphere/pull/18375#discussion_r897885545


##########
shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/client/netty/MySQLBinlogEventPacketDecoder.java:
##########
@@ -54,39 +55,48 @@ public MySQLBinlogEventPacketDecoder(final int checksumLength) {
     
     @Override
     protected void decode(final ChannelHandlerContext ctx, final ByteBuf in, final List<Object> out) {
-        MySQLPacketPayload payload = new MySQLPacketPayload(in, ctx.channel().attr(CommonConstants.CHARSET_ATTRIBUTE_KEY).get());
-        skipSequenceId(payload);
-        checkError(payload);
-        MySQLBinlogEventHeader binlogEventHeader = new MySQLBinlogEventHeader(payload);
-        removeChecksum(binlogEventHeader.getEventType(), in);
+        // readable bytes must grete + seqId(1b) + statusCode(1b) + header-length(19b) + 
+        while (in.readableBytes() >= 2 + MySQLBinlogEventHeader.MYSQL_BINLOG_EVENT_HEADER_LENGTH) {
+            in.markReaderIndex();
+            MySQLPacketPayload payload = new MySQLPacketPayload(in, ctx.channel().attr(CommonConstants.CHARSET_ATTRIBUTE_KEY).get());
+            skipSequenceId(payload);
+            checkError(payload);
+            MySQLBinlogEventHeader binlogEventHeader = new MySQLBinlogEventHeader(payload, binlogContext.getChecksumLength());
+            // make sure event has complete body
+            if (in.readableBytes() < binlogEventHeader.getEventSize() - MySQLBinlogEventHeader.MYSQL_BINLOG_EVENT_HEADER_LENGTH) {
+                log.debug("the event body is not complete, event size={}, readable bytes={}", binlogEventHeader.getEventSize(), in.readableBytes());
+                in.resetReaderIndex();
+                break;
+            }
+            out.add(decodeEvent(payload, binlogEventHeader));
+            skipChecksum(binlogEventHeader.getEventType(), in);
+        }
+    }
+    
+    private Object decodeEvent(final MySQLPacketPayload payload, final MySQLBinlogEventHeader binlogEventHeader) {
         switch (MySQLBinlogEventType.valueOf(binlogEventHeader.getEventType())) {
             case ROTATE_EVENT:
-                decodeRotateEvent(binlogEventHeader, payload);
-                break;
+                return decodeRotateEvent(binlogEventHeader, payload);
             case FORMAT_DESCRIPTION_EVENT:
-                new MySQLBinlogFormatDescriptionEventPacket(binlogEventHeader, payload);
-                break;
+                return new MySQLBinlogFormatDescriptionEventPacket(binlogEventHeader, payload);
             case TABLE_MAP_EVENT:
-                decodeTableMapEvent(binlogEventHeader, payload);
-                break;
+                return decodeTableMapEvent(binlogEventHeader, payload);

Review Comment:
   Some types of event didn't write packet to `out` before, need to verify there's no side effect if we write more packet in `out`.



##########
shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/client/netty/MySQLBinlogEventPacketDecoder.java:
##########
@@ -54,39 +56,51 @@ public MySQLBinlogEventPacketDecoder(final int checksumLength) {
     
     @Override
     protected void decode(final ChannelHandlerContext ctx, final ByteBuf in, final List<Object> out) {
-        MySQLPacketPayload payload = new MySQLPacketPayload(in, ctx.channel().attr(CommonConstants.CHARSET_ATTRIBUTE_KEY).get());
-        skipSequenceId(payload);
-        checkError(payload);
-        MySQLBinlogEventHeader binlogEventHeader = new MySQLBinlogEventHeader(payload);
-        removeChecksum(binlogEventHeader.getEventType(), in);
+        // readable bytes must grete + seqId(1b) + statusCode(1b) + header-length(19b) +
+        while (in.readableBytes() >= 2 + MySQLBinlogEventHeader.MYSQL_BINLOG_EVENT_HEADER_LENGTH) {
+            in.markReaderIndex();
+            MySQLPacketPayload payload = new MySQLPacketPayload(in, ctx.channel().attr(CommonConstants.CHARSET_ATTRIBUTE_KEY).get());
+            skipSequenceId(payload);
+            checkError(payload);
+            MySQLBinlogEventHeader binlogEventHeader = new MySQLBinlogEventHeader(payload, binlogContext.getChecksumLength());
+            // make sure event has complete body
+            if (in.readableBytes() < binlogEventHeader.getEventSize() - MySQLBinlogEventHeader.MYSQL_BINLOG_EVENT_HEADER_LENGTH) {
+                log.debug("the event body is not complete, event size={}, readable bytes={}", binlogEventHeader.getEventSize(), in.readableBytes());
+                in.resetReaderIndex();
+                break;
+            }
+            Object event = decodeEvent(payload, binlogEventHeader);
+            if (event instanceof AbstractBinlogEvent) {
+                out.add(event);
+            }

Review Comment:
   Seems PlaceholderEvent is instance of AbstractBinlogEvent, could it be improved?



##########
shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/client/netty/MySQLBinlogEventPacketDecoder.java:
##########
@@ -54,39 +56,51 @@ public MySQLBinlogEventPacketDecoder(final int checksumLength) {
     
     @Override
     protected void decode(final ChannelHandlerContext ctx, final ByteBuf in, final List<Object> out) {
-        MySQLPacketPayload payload = new MySQLPacketPayload(in, ctx.channel().attr(CommonConstants.CHARSET_ATTRIBUTE_KEY).get());
-        skipSequenceId(payload);
-        checkError(payload);
-        MySQLBinlogEventHeader binlogEventHeader = new MySQLBinlogEventHeader(payload);
-        removeChecksum(binlogEventHeader.getEventType(), in);
+        // readable bytes must grete + seqId(1b) + statusCode(1b) + header-length(19b) +

Review Comment:
   `grete` means greater than?



-- 
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: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #18375: Fix scaling increment task consumed incomplete data

Posted by GitBox <gi...@apache.org>.
sandynz commented on code in PR #18375:
URL: https://github.com/apache/shardingsphere/pull/18375#discussion_r898660203


##########
shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/client/netty/MySQLBinlogEventPacketDecoder.java:
##########
@@ -112,20 +127,22 @@ private String readRemainPacket(final MySQLPacketPayload payload) {
         return ByteBufUtil.hexDump(payload.readStringFixByBytes(payload.getByteBuf().readableBytes()));
     }
     
-    private void removeChecksum(final int eventType, final ByteBuf in) {
+    private void skipChecksum(final int eventType, final ByteBuf in) {
         if (0 < binlogContext.getChecksumLength() && MySQLBinlogEventType.FORMAT_DESCRIPTION_EVENT.getValue() != eventType) {
-            in.writerIndex(in.writerIndex() - binlogContext.getChecksumLength());
+            in.skipBytes(binlogContext.getChecksumLength());
         }
     }
     
-    private void decodeRotateEvent(final MySQLBinlogEventHeader binlogEventHeader, final MySQLPacketPayload payload) {
-        MySQLBinlogRotateEventPacket rotateEventPacket = new MySQLBinlogRotateEventPacket(binlogEventHeader, payload);
-        binlogContext.setFileName(rotateEventPacket.getNextBinlogName());
+    private AbstractMySQLBinlogEventPacket decodeRotateEvent(final MySQLBinlogEventHeader binlogEventHeader, final MySQLPacketPayload payload) {
+        MySQLBinlogRotateEventPacket result = new MySQLBinlogRotateEventPacket(binlogEventHeader, payload);
+        binlogContext.setFileName(result.getNextBinlogName());
+        return result;
     }
     
-    private void decodeTableMapEvent(final MySQLBinlogEventHeader binlogEventHeader, final MySQLPacketPayload payload) {
-        MySQLBinlogTableMapEventPacket tableMapEventPacket = new MySQLBinlogTableMapEventPacket(binlogEventHeader, payload);
-        binlogContext.putTableMapEvent(tableMapEventPacket.getTableId(), tableMapEventPacket);
+    private MySQLBinlogTableMapEventPacket decodeTableMapEvent(final MySQLBinlogEventHeader binlogEventHeader, final MySQLPacketPayload payload) {
+        MySQLBinlogTableMapEventPacket result = new MySQLBinlogTableMapEventPacket(binlogEventHeader, payload);
+        binlogContext.putTableMapEvent(result.getTableId(), result);
+        return result;
     }

Review Comment:
   Could we keep decodeRotateEvent and decodeTableMapEvent return void? Since it's not necessary to return result.



-- 
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: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #18375: Fix scaling increment task consumed incomplete data

Posted by GitBox <gi...@apache.org>.
sandynz commented on code in PR #18375:
URL: https://github.com/apache/shardingsphere/pull/18375#discussion_r898627337


##########
shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/client/netty/MySQLBinlogEventPacketDecoder.java:
##########
@@ -54,39 +56,51 @@ public MySQLBinlogEventPacketDecoder(final int checksumLength) {
     
     @Override
     protected void decode(final ChannelHandlerContext ctx, final ByteBuf in, final List<Object> out) {
-        MySQLPacketPayload payload = new MySQLPacketPayload(in, ctx.channel().attr(CommonConstants.CHARSET_ATTRIBUTE_KEY).get());
-        skipSequenceId(payload);
-        checkError(payload);
-        MySQLBinlogEventHeader binlogEventHeader = new MySQLBinlogEventHeader(payload);
-        removeChecksum(binlogEventHeader.getEventType(), in);
+        // readable bytes must grete + seqId(1b) + statusCode(1b) + header-length(19b) +
+        while (in.readableBytes() >= 2 + MySQLBinlogEventHeader.MYSQL_BINLOG_EVENT_HEADER_LENGTH) {
+            in.markReaderIndex();
+            MySQLPacketPayload payload = new MySQLPacketPayload(in, ctx.channel().attr(CommonConstants.CHARSET_ATTRIBUTE_KEY).get());
+            skipSequenceId(payload);
+            checkError(payload);
+            MySQLBinlogEventHeader binlogEventHeader = new MySQLBinlogEventHeader(payload, binlogContext.getChecksumLength());
+            // make sure event has complete body
+            if (in.readableBytes() < binlogEventHeader.getEventSize() - MySQLBinlogEventHeader.MYSQL_BINLOG_EVENT_HEADER_LENGTH) {
+                log.debug("the event body is not complete, event size={}, readable bytes={}", binlogEventHeader.getEventSize(), in.readableBytes());
+                in.resetReaderIndex();
+                break;
+            }
+            Object event = decodeEvent(payload, binlogEventHeader);
+            if (event instanceof AbstractBinlogEvent) {
+                out.add(event);
+            }

Review Comment:
   PlaceholderEvent should be kept.



-- 
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: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] codecov-commenter commented on pull request #18375: Fix scaling increment task consumed incomplete data

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #18375:
URL: https://github.com/apache/shardingsphere/pull/18375#issuecomment-1156439446

   # [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/18375?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#18375](https://codecov.io/gh/apache/shardingsphere/pull/18375?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9e511df) into [master](https://codecov.io/gh/apache/shardingsphere/commit/39acbd90293345a9928f92d563802cd122f29df8?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (39acbd9) will **decrease** coverage by `0.03%`.
   > The diff coverage is `80.43%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #18375      +/-   ##
   ============================================
   - Coverage     59.13%   59.10%   -0.04%     
   - Complexity     2194     2195       +1     
   ============================================
     Files          3737     3738       +1     
     Lines         54782    54802      +20     
     Branches       9311     9318       +7     
   ============================================
   - Hits          32397    32392       -5     
   - Misses        19628    19645      +17     
   - Partials       2757     2765       +8     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/shardingsphere/pull/18375?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../packet/binlog/row/MySQLBinlogRowsEventPacket.java](https://codecov.io/gh/apache/shardingsphere/pull/18375/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZGItcHJvdG9jb2wvc2hhcmRpbmdzcGhlcmUtZGItcHJvdG9jb2wtbXlzcWwvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2RiL3Byb3RvY29sL215c3FsL3BhY2tldC9iaW5sb2cvcm93L015U1FMQmlubG9nUm93c0V2ZW50UGFja2V0LmphdmE=) | `51.61% <0.00%> (-42.14%)` | :arrow_down: |
   | [.../packet/binlog/AbstractMySQLBinlogEventPacket.java](https://codecov.io/gh/apache/shardingsphere/pull/18375/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZGItcHJvdG9jb2wvc2hhcmRpbmdzcGhlcmUtZGItcHJvdG9jb2wtbXlzcWwvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2RiL3Byb3RvY29sL215c3FsL3BhY2tldC9iaW5sb2cvQWJzdHJhY3RNeVNRTEJpbmxvZ0V2ZW50UGFja2V0LmphdmE=) | `70.00% <60.00%> (-10.00%)` | :arrow_down: |
   | [...ket/binlog/row/MySQLBinlogTableMapEventPacket.java](https://codecov.io/gh/apache/shardingsphere/pull/18375/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZGItcHJvdG9jb2wvc2hhcmRpbmdzcGhlcmUtZGItcHJvdG9jb2wtbXlzcWwvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2RiL3Byb3RvY29sL215c3FsL3BhY2tldC9iaW5sb2cvcm93L015U1FMQmlubG9nVGFibGVNYXBFdmVudFBhY2tldC5qYXZh) | `93.33% <66.66%> (-3.10%)` | :arrow_down: |
   | [...st/client/netty/MySQLBinlogEventPacketDecoder.java](https://codecov.io/gh/apache/shardingsphere/pull/18375/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUta2VybmVsL3NoYXJkaW5nc3BoZXJlLWRhdGEtcGlwZWxpbmUvc2hhcmRpbmdzcGhlcmUtZGF0YS1waXBlbGluZS1kaWFsZWN0L3NoYXJkaW5nc3BoZXJlLWRhdGEtcGlwZWxpbmUtbXlzcWwvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2RhdGEvcGlwZWxpbmUvbXlzcWwvaW5nZXN0L2NsaWVudC9uZXR0eS9NeVNRTEJpbmxvZ0V2ZW50UGFja2V0RGVjb2Rlci5qYXZh) | `82.55% <85.29%> (-12.51%)` | :arrow_down: |
   | [...ol/mysql/packet/binlog/MySQLBinlogEventHeader.java](https://codecov.io/gh/apache/shardingsphere/pull/18375/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZGItcHJvdG9jb2wvc2hhcmRpbmdzcGhlcmUtZGItcHJvdG9jb2wtbXlzcWwvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2RiL3Byb3RvY29sL215c3FsL3BhY2tldC9iaW5sb2cvTXlTUUxCaW5sb2dFdmVudEhlYWRlci5qYXZh) | `94.44% <100.00%> (+0.32%)` | :arrow_up: |
   | [...inlog/management/MySQLBinlogRotateEventPacket.java](https://codecov.io/gh/apache/shardingsphere/pull/18375/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZGItcHJvdG9jb2wvc2hhcmRpbmdzcGhlcmUtZGItcHJvdG9jb2wtbXlzcWwvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2RiL3Byb3RvY29sL215c3FsL3BhY2tldC9iaW5sb2cvbWFuYWdlbWVudC9NeVNRTEJpbmxvZ1JvdGF0ZUV2ZW50UGFja2V0LmphdmE=) | `100.00% <100.00%> (ø)` | |
   | [...end/postgresql/err/PostgreSQLErrPacketFactory.java](https://codecov.io/gh/apache/shardingsphere/pull/18375/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtcHJveHkvc2hhcmRpbmdzcGhlcmUtcHJveHktZnJvbnRlbmQvc2hhcmRpbmdzcGhlcmUtcHJveHktZnJvbnRlbmQtcG9zdGdyZXNxbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvcHJveHkvZnJvbnRlbmQvcG9zdGdyZXNxbC9lcnIvUG9zdGdyZVNRTEVyclBhY2tldEZhY3RvcnkuamF2YQ==) | `37.50% <0.00%> (-3.88%)` | :arrow_down: |
   | [...tocol/postgresql/constant/PostgreSQLErrorCode.java](https://codecov.io/gh/apache/shardingsphere/pull/18375/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtZGItcHJvdG9jb2wvc2hhcmRpbmdzcGhlcmUtZGItcHJvdG9jb2wtcG9zdGdyZXNxbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvZGIvcHJvdG9jb2wvcG9zdGdyZXNxbC9jb25zdGFudC9Qb3N0Z3JlU1FMRXJyb3JDb2RlLmphdmE=) | `100.00% <0.00%> (ø)` | |
   | ... and [4 more](https://codecov.io/gh/apache/shardingsphere/pull/18375/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/shardingsphere/pull/18375?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/18375?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [39acbd9...9e511df](https://codecov.io/gh/apache/shardingsphere/pull/18375?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] azexcy commented on pull request #18375: Fix scaling increment task consumed incomplete data

Posted by GitBox <gi...@apache.org>.
azexcy commented on PR #18375:
URL: https://github.com/apache/shardingsphere/pull/18375#issuecomment-1156302687

   > @azexcy please modify the title to describe what this PR does
   
   changed


-- 
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: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz merged pull request #18375: Fix scaling increment task consumed incomplete data

Posted by GitBox <gi...@apache.org>.
sandynz merged PR #18375:
URL: https://github.com/apache/shardingsphere/pull/18375


-- 
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: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org