Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Improve CDCImporter process FinishedRecord, and advise CDC client #28953

Merged
merged 8 commits into from
Nov 8, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,8 @@
import org.apache.shardingsphere.data.pipeline.cdc.client.constant.ClientConnectionStatus;
import org.apache.shardingsphere.data.pipeline.cdc.client.context.ClientConnectionContext;
import org.apache.shardingsphere.data.pipeline.cdc.client.handler.CDCRequestHandler;
import org.apache.shardingsphere.data.pipeline.cdc.client.handler.ExceptionHandler;
import org.apache.shardingsphere.data.pipeline.cdc.client.handler.ServerErrorResultHandler;
import org.apache.shardingsphere.data.pipeline.cdc.client.parameter.CDCLoginParameter;
import org.apache.shardingsphere.data.pipeline.cdc.client.parameter.StartStreamingParameter;
import org.apache.shardingsphere.data.pipeline.cdc.client.util.RequestIdUtils;
Expand All @@ -49,6 +51,10 @@
import org.apache.shardingsphere.data.pipeline.cdc.protocol.request.StopStreamingRequestBody;
import org.apache.shardingsphere.data.pipeline.cdc.protocol.request.StreamDataRequestBody;
import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.CDCResponse;
import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.DataRecordResult.Record;

import java.util.List;
import java.util.function.Consumer;

/**
* CDC client.
Expand All @@ -65,9 +71,6 @@ public final class CDCClient implements AutoCloseable {
public CDCClient(final CDCClientConfiguration config) {
validateParameter(config);
this.config = config;
if (null != config.getExceptionHandler()) {
config.getExceptionHandler().setCDCClient(this);
}
}

private void validateParameter(final CDCClientConfiguration parameter) {
Expand All @@ -81,9 +84,13 @@ private void validateParameter(final CDCClientConfiguration parameter) {

/**
* Connect.
*
* @param dataConsumer data consumer
* @param exceptionHandler exception handler
* @param errorResultHandler error result handler
*/
@SneakyThrows(InterruptedException.class)
public void connect() {
public void connect(final Consumer<List<Record>> dataConsumer, final ExceptionHandler exceptionHandler, final ServerErrorResultHandler errorResultHandler) {
Bootstrap bootstrap = new Bootstrap();
group = new NioEventLoopGroup(1);
bootstrap.channel(NioSocketChannel.class)
Expand All @@ -98,7 +105,7 @@ protected void initChannel(final NioSocketChannel channel) {
channel.pipeline().addLast(new ProtobufDecoder(CDCResponse.getDefaultInstance()));
channel.pipeline().addLast(new ProtobufVarint32LengthFieldPrepender());
channel.pipeline().addLast(new ProtobufEncoder());
channel.pipeline().addLast(new CDCRequestHandler(config.getDataConsumer(), config.getExceptionHandler()));
channel.pipeline().addLast(new CDCRequestHandler(dataConsumer, exceptionHandler, errorResultHandler));
}
});
channel = bootstrap.connect(config.getAddress(), config.getPort()).sync().channel();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,6 @@

import lombok.Getter;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.data.pipeline.cdc.client.handler.ExceptionHandler;
import org.apache.shardingsphere.data.pipeline.cdc.protocol.response.DataRecordResult.Record;

import java.util.List;
import java.util.function.Consumer;

/**
* CDC client configuration.
Expand All @@ -36,17 +31,5 @@ public final class CDCClientConfiguration {

private final int port;

private final Consumer<List<Record>> dataConsumer;

private final ExceptionHandler exceptionHandler;

private final int timeoutMills;

public CDCClientConfiguration(final String address, final int port, final Consumer<List<Record>> dataConsumer, final ExceptionHandler exceptionHandler) {
this.address = address;
this.port = port;
this.dataConsumer = dataConsumer;
this.exceptionHandler = exceptionHandler;
this.timeoutMills = 5000;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,8 @@ public final class CDCRequestHandler extends ChannelInboundHandlerAdapter {

private final ExceptionHandler exceptionHandler;

private final ServerErrorResultHandler errorResultHandler;

@Override
public void channelRegistered(final ChannelHandlerContext ctx) {
ClientConnectionContext context = new ClientConnectionContext();
Expand Down Expand Up @@ -78,7 +80,7 @@ public void channelRead(final ChannelHandlerContext ctx, final Object msg) {
future.countDown();
requestType = future.getRequestType();
}
exceptionHandler.handleServerException(ctx, new ServerErrorResult(response.getErrorCode(), response.getErrorMessage(), requestType));
errorResultHandler.handleServerError(ctx, new ServerErrorResult(response.getErrorCode(), response.getErrorMessage(), requestType));
responseFuture.ifPresent(future -> {
future.setErrorCode(response.getErrorCode());
future.setErrorMessage(response.getErrorMessage());
Expand Down Expand Up @@ -113,6 +115,6 @@ private void processDataRecords(final ChannelHandlerContext ctx, final DataRecor

@Override
public void exceptionCaught(final ChannelHandlerContext ctx, final Throwable cause) {
exceptionHandler.handleSocketException(ctx, cause);
exceptionHandler.handleException(ctx, cause);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,35 +18,17 @@
package org.apache.shardingsphere.data.pipeline.cdc.client.handler;

import io.netty.channel.ChannelHandlerContext;
import org.apache.shardingsphere.data.pipeline.cdc.client.CDCClient;
import org.apache.shardingsphere.data.pipeline.cdc.client.util.ServerErrorResult;

/**
* Exception handler.
*/
public interface ExceptionHandler {

/**
* Handle server exception.
*
* @param ctx channel handler context
* @param result error result
*/
void handleServerException(ChannelHandlerContext ctx, ServerErrorResult result);

/**
* Handle socket exception.
* Handle exception.
*
* @param ctx channel handler context
* @param throwable throwable
*/
void handleSocketException(ChannelHandlerContext ctx, Throwable throwable);

/**
* Set CDC client.
*
* @param cdcClient cdc client
*/
default void setCDCClient(CDCClient cdcClient) {
}
void handleException(ChannelHandlerContext ctx, Throwable throwable);
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,19 +23,19 @@
import org.apache.shardingsphere.data.pipeline.cdc.client.util.ServerErrorResult;

/**
* Logger exception handler, just print error.
* Logger exception error handler.
*/
@RequiredArgsConstructor
@Slf4j
public final class LoggerExceptionHandler implements ExceptionHandler {
public final class LoggerExceptionErrorHandler implements ExceptionHandler, ServerErrorResultHandler {

@Override
public void handleServerException(final ChannelHandlerContext ctx, final ServerErrorResult result) {
public void handleServerError(final ChannelHandlerContext ctx, final ServerErrorResult result) {
log.error("Server error, code: {}, message: {}", result.getErrorCode(), result.getErrorMessage());
}

@Override
public void handleSocketException(final ChannelHandlerContext ctx, final Throwable throwable) {
log.error("Socket error: ", throwable);
public void handleException(final ChannelHandlerContext ctx, final Throwable throwable) {
log.error("Exception error: ", throwable);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,10 @@
package org.apache.shardingsphere.data.pipeline.cdc.client.handler;

import io.netty.channel.ChannelHandlerContext;
import lombok.RequiredArgsConstructor;
import lombok.SneakyThrows;
import lombok.extern.slf4j.Slf4j;
import org.apache.shardingsphere.data.pipeline.cdc.client.CDCClient;
import org.apache.shardingsphere.data.pipeline.cdc.client.context.ClientConnectionContext;
import org.apache.shardingsphere.data.pipeline.cdc.client.util.ServerErrorResult;

import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
Expand All @@ -32,53 +30,45 @@
/**
* Retry streaming exception handler.
*/
@RequiredArgsConstructor
@Slf4j
public class RetryStreamingExceptionHandler implements ExceptionHandler {

private final AtomicInteger retryCount = new AtomicInteger(0);
private final CDCClient cdcClient;

private final AtomicInteger maxRetryTimes = new AtomicInteger(0);

private final int retryIntervalMills;

private CDCClient cdcClient;
private final int retryTimes;

public RetryStreamingExceptionHandler(final int retryCount, final int retryIntervalMills) {
this.retryCount.set(retryCount);
this.retryIntervalMills = retryIntervalMills;
}

@Override
public void setCDCClient(final CDCClient cdcClient) {
public RetryStreamingExceptionHandler(final CDCClient cdcClient, final int maxRetryTimes, final int retryIntervalMills) {
this.cdcClient = cdcClient;
this.maxRetryTimes.set(maxRetryTimes);
this.retryIntervalMills = retryIntervalMills;
retryTimes = 0;
}
azexcy marked this conversation as resolved.
Show resolved Hide resolved

@Override
public void handleServerException(final ChannelHandlerContext ctx, final ServerErrorResult result) {
log.error("Server error, code: {}, message: {}", result.getErrorCode(), result.getErrorMessage());
reconnect(ctx);
}

@Override
public void handleSocketException(final ChannelHandlerContext ctx, final Throwable throwable) {
public void handleException(final ChannelHandlerContext ctx, final Throwable throwable) {
log.error("Socket error: {}", throwable.getMessage());
azexcy marked this conversation as resolved.
Show resolved Hide resolved
reconnect(ctx);
}

@SneakyThrows(InterruptedException.class)
private void reconnect(final ChannelHandlerContext ctx) {
retryCount.incrementAndGet();
maxRetryTimes.incrementAndGet();
if (null == cdcClient) {
log.warn("CDC client is null, could not retry");
return;
}
ClientConnectionContext connectionContext = ctx.channel().attr(ClientConnectionContext.CONTEXT_KEY).get();
if (retryCount.get() > 5) {
if (retryTimes > maxRetryTimes.get()) {
log.warn("Retry times exceed 5, stop streaming");
connectionContext.getStreamingIds().forEach(each -> CompletableFuture.runAsync(() -> cdcClient.stopStreaming(each)));
return;
}
TimeUnit.MILLISECONDS.sleep(retryIntervalMills);
log.info("Retry to restart streaming, retry count: {}", retryCount.get());
log.info("Retry to restart streaming, retry count: {}", maxRetryTimes.get());
connectionContext.getStreamingIds().forEach(each -> CompletableFuture.runAsync(() -> cdcClient.restartStreaming(each)));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.shardingsphere.data.pipeline.cdc.client.handler;

import io.netty.channel.ChannelHandlerContext;
import org.apache.shardingsphere.data.pipeline.cdc.client.util.ServerErrorResult;

/**
* Server error result handler.
*/
public interface ServerErrorResultHandler {

/**
* Handle server ERROR.
*
* @param ctx channel handler context
* @param result error result
*/
void handleServerError(ChannelHandlerContext ctx, ServerErrorResult result);
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import lombok.extern.slf4j.Slf4j;
import org.apache.shardingsphere.data.pipeline.cdc.client.CDCClient;
import org.apache.shardingsphere.data.pipeline.cdc.client.config.CDCClientConfiguration;
import org.apache.shardingsphere.data.pipeline.cdc.client.handler.LoggerExceptionErrorHandler;
import org.apache.shardingsphere.data.pipeline.cdc.client.handler.RetryStreamingExceptionHandler;
import org.apache.shardingsphere.data.pipeline.cdc.client.parameter.CDCLoginParameter;
import org.apache.shardingsphere.data.pipeline.cdc.client.parameter.StartStreamingParameter;
Expand All @@ -41,14 +42,14 @@ public static void main(final String[] args) {
// Pay attention to the time zone, to avoid the problem of incorrect time zone, it is best to ensure that the time zone of the program is consistent with the time zone of the database server
// TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
String address = "127.0.0.1";
CDCClientConfiguration clientConfig = new CDCClientConfiguration(address, 33071, records -> log.info("records: {}", records), new RetryStreamingExceptionHandler(3, 5000));
int reconnectCount = 0;
int maxReconnectCount = 5;
while (reconnectCount < maxReconnectCount) {
try (CDCClient cdcClient = new CDCClient(clientConfig)) {
cdcClient.connect();
try (CDCClient cdcClient = new CDCClient(new CDCClientConfiguration(address, 33071, 5000))) {
LoggerExceptionErrorHandler loggerExceptionErrorHandler = new LoggerExceptionErrorHandler();
cdcClient.connect(records -> log.info("records: {}", records), new RetryStreamingExceptionHandler(cdcClient, 5, 5000), loggerExceptionErrorHandler);
azexcy marked this conversation as resolved.
Show resolved Hide resolved
cdcClient.login(new CDCLoginParameter("root", "root"));
String streamingId = cdcClient.startStreaming(new StartStreamingParameter("sharding_db", Collections.singleton(SchemaTable.newBuilder().setTable("t_order").build()), true));
String streamingId = cdcClient.startStreaming(new StartStreamingParameter("sharding_db", Collections.singleton(SchemaTable.newBuilder().setTable("t_order123").build()), true));
log.info("Streaming id={}", streamingId);
cdcClient.await();
// CHECKSTYLE:OFF
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
import org.apache.shardingsphere.data.pipeline.cdc.api.job.type.CDCJobType;
import org.apache.shardingsphere.data.pipeline.cdc.client.CDCClient;
import org.apache.shardingsphere.data.pipeline.cdc.client.config.CDCClientConfiguration;
import org.apache.shardingsphere.data.pipeline.cdc.client.handler.LoggerExceptionHandler;
import org.apache.shardingsphere.data.pipeline.cdc.client.handler.LoggerExceptionErrorHandler;
import org.apache.shardingsphere.data.pipeline.cdc.client.parameter.CDCLoginParameter;
import org.apache.shardingsphere.data.pipeline.cdc.client.parameter.StartStreamingParameter;
import org.apache.shardingsphere.data.pipeline.cdc.protocol.request.StreamDataRequestBody.SchemaTable;
Expand Down Expand Up @@ -169,10 +169,10 @@ private DataSource createStandardDataSource(final PipelineContainerComposer cont
private void startCDCClient(final PipelineContainerComposer containerComposer, final DialectDatabaseMetaData dialectDatabaseMetaData) {
DataSource dataSource = createStandardDataSource(containerComposer, PipelineContainerComposer.DS_4);
DataSourceRecordConsumer recordConsumer = new DataSourceRecordConsumer(dataSource, containerComposer.getDatabaseType());
CDCClientConfiguration cdcConfig = new CDCClientConfiguration("localhost", containerComposer.getContainerComposer().getProxyCDCPort(), recordConsumer, new LoggerExceptionHandler());
String schema = dialectDatabaseMetaData.isSchemaAvailable() ? "test" : "";
CDCClient cdcClient = new CDCClient(cdcConfig);
cdcClient.connect();
CDCClient cdcClient = new CDCClient(new CDCClientConfiguration("localhost", containerComposer.getContainerComposer().getProxyCDCPort(), 5000));
LoggerExceptionErrorHandler handler = new LoggerExceptionErrorHandler();
cdcClient.connect(recordConsumer, handler, handler);
azexcy marked this conversation as resolved.
Show resolved Hide resolved
cdcClient.login(new CDCLoginParameter(ProxyContainerConstants.USERNAME, ProxyContainerConstants.PASSWORD));
// TODO add full=false test case later
cdcClient.startStreaming(new StartStreamingParameter("sharding_db", new HashSet<>(Arrays.asList(SchemaTable.newBuilder().setTable(SOURCE_TABLE_NAME).setSchema(schema).build(),
Expand Down