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 all commits
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 Down Expand Up @@ -78,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 @@ -95,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 Expand Up @@ -165,9 +175,6 @@ public String startStreaming(final StartStreamingParameter parameter) {
* @param streamingId streaming id
*/
public void restartStreaming(final String streamingId) {
if (checkStreamingIdExist(streamingId)) {
stopStreaming(streamingId);
}
String requestId = RequestIdUtils.generateRequestId();
StartStreamingRequestBody body = StartStreamingRequestBody.newBuilder().setStreamingId(streamingId).build();
CDCRequest request = CDCRequest.newBuilder().setRequestId(requestId).setType(Type.START_STREAMING).setStartStreamingRequestBody(body).build();
Expand All @@ -179,16 +186,6 @@ public void restartStreaming(final String streamingId) {
log.info("Restart streaming success, streaming id: {}", streamingId);
}

private boolean checkStreamingIdExist(final String streamingId) {
checkChannelActive();
ClientConnectionContext connectionContext = channel.attr(ClientConnectionContext.CONTEXT_KEY).get();
if (null == connectionContext) {
log.warn("The connection context not exist");
return true;
}
return connectionContext.getStreamingIds().contains(streamingId);
}

/**
* Stop streaming.
*
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(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(cause);
exceptionHandler.handleException(ctx, cause);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,24 +17,18 @@

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

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

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

/**
* Handle server exception.
*
* @param result error result
*/
void handleServerException(ServerErrorResult result);

/**
* Handle socket exception.
* Handle exception.
*
* @param ctx channel handler context
* @param throwable throwable
*/
void handleSocketException(Throwable throwable);
void handleException(ChannelHandlerContext ctx, Throwable throwable);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* 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 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 java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;

/**
* Retry streaming exception handler.
*/
@Slf4j
public final class RetryStreamingExceptionHandler implements ExceptionHandler {

private final CDCClient cdcClient;

private final AtomicInteger maxRetryTimes;

private final int retryIntervalMills;

private final AtomicInteger retryTimes = new AtomicInteger(0);

public RetryStreamingExceptionHandler(final CDCClient cdcClient, final int maxRetryTimes, final int retryIntervalMills) {
this.cdcClient = cdcClient;
this.maxRetryTimes = new AtomicInteger(maxRetryTimes);
this.retryIntervalMills = retryIntervalMills;
}

@Override
public void handleException(final ChannelHandlerContext ctx, final Throwable throwable) {
log.error("Catch exception: ", throwable);
reconnect(ctx);
}

@SneakyThrows(InterruptedException.class)
private void reconnect(final ChannelHandlerContext ctx) {
retryTimes.incrementAndGet();
ClientConnectionContext connectionContext = ctx.channel().attr(ClientConnectionContext.CONTEXT_KEY).get();
if (retryTimes.get() > maxRetryTimes.get()) {
log.warn("Stop try to reconnect, stop streaming ids: {}", connectionContext.getStreamingIds());
connectionContext.getStreamingIds().forEach(each -> CompletableFuture.runAsync(() -> cdcClient.stopStreaming(each)));
return;
}
TimeUnit.MILLISECONDS.sleep(retryIntervalMills);
log.info("Retry to restart streaming, retry times: {}", retryTimes.get());
connectionContext.getStreamingIds().forEach(each -> CompletableFuture.runAsync(() -> cdcClient.restartStreaming(each)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,24 +17,19 @@

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

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

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

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

@Override
public void handleSocketException(final Throwable throwable) {
log.error("Socket error: ", throwable);
}
/**
* 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,7 +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.LoggerExceptionHandler;
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;
import org.apache.shardingsphere.data.pipeline.cdc.protocol.request.StreamDataRequestBody.SchemaTable;
Expand All @@ -41,9 +41,9 @@ 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 LoggerExceptionHandler());
try (CDCClient cdcClient = new CDCClient(clientConfig)) {
cdcClient.connect();
try (CDCClient cdcClient = new CDCClient(new CDCClientConfiguration(address, 33071, 10000))) {
cdcClient.connect(records -> log.info("records: {}", records), new RetryStreamingExceptionHandler(cdcClient, 5, 5000),
(ctx, result) -> log.error("Server error: {}", result.getErrorMessage()));
cdcClient.login(new CDCLoginParameter("root", "root"));
String streamingId = cdcClient.startStreaming(new StartStreamingParameter("sharding_db", Collections.singleton(SchemaTable.newBuilder().setTable("t_order").build()), true));
log.info("Streaming id={}", streamingId);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,6 @@

import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.PriorityQueue;
Expand Down Expand Up @@ -79,23 +78,20 @@ public final class CDCImporter extends AbstractPipelineLifecycleRunnable impleme
@Override
protected void runBlocking() {
CDCImporterManager.putImporter(this);
List<CDCChannelProgressPair> channelProgressPairs = new ArrayList<>(originalChannelProgressPairs);
while (isRunning()) {
if (needSorting) {
doWithSorting(channelProgressPairs);
doWithSorting(originalChannelProgressPairs);
} else {
doWithoutSorting(channelProgressPairs);
doWithoutSorting(originalChannelProgressPairs);
}
if (channelProgressPairs.isEmpty()) {
if (originalChannelProgressPairs.isEmpty()) {
break;
}
}
}

private void doWithoutSorting(final List<CDCChannelProgressPair> channelProgressPairs) {
Iterator<CDCChannelProgressPair> channelProgressPairsIterator = channelProgressPairs.iterator();
while (channelProgressPairsIterator.hasNext()) {
CDCChannelProgressPair channelProgressPair = channelProgressPairsIterator.next();
for (final CDCChannelProgressPair channelProgressPair : channelProgressPairs) {
PipelineChannel channel = channelProgressPair.getChannel();
List<Record> records = channel.fetchRecords(batchSize, timeout, timeUnit).stream().filter(each -> !(each instanceof PlaceholderRecord)).collect(Collectors.toList());
if (records.isEmpty()) {
Expand All @@ -108,9 +104,6 @@ private void doWithoutSorting(final List<CDCChannelProgressPair> channelProgress
ackCache.put(ackId, Collections.singletonList(Pair.of(channelProgressPair, new CDCAckPosition(records.get(records.size() - 1), getDataRecordsCount(records)))));
sink.write(ackId, records);
Record lastRecord = records.get(records.size() - 1);
if (lastRecord instanceof FinishedRecord) {
channelProgressPairsIterator.remove();
}
if (lastRecord instanceof FinishedRecord && records.stream().noneMatch(DataRecord.class::isInstance)) {
channel.ack(records);
channelProgressPair.getJobProgressListener().onProgressUpdated(new PipelineJobProgressUpdatedParameter(0));
Expand Down Expand Up @@ -236,9 +229,14 @@ public void ack(final String ackId) {
}
for (Pair<CDCChannelProgressPair, CDCAckPosition> each : channelPositionPairList) {
CDCAckPosition ackPosition = each.getRight();
each.getLeft().getChannel().ack(Collections.singletonList(ackPosition.getLastRecord()));
Record lastRecord = ackPosition.getLastRecord();
each.getLeft().getChannel().ack(Collections.singletonList(lastRecord));
if (lastRecord instanceof FinishedRecord) {
originalChannelProgressPairs.remove(each.getKey());
}
each.getLeft().getJobProgressListener().onProgressUpdated(new PipelineJobProgressUpdatedParameter(ackPosition.getDataRecordCount()));
}
ackCache.invalidate(ackId);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@
import java.util.LinkedList;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
Expand All @@ -75,7 +76,7 @@ public final class CDCJobPreparer {
public void initTasks(final Collection<CDCJobItemContext> jobItemContexts) {
// TODO Use pipeline tree to build it
AtomicBoolean inventoryImporterUsed = new AtomicBoolean();
List<CDCChannelProgressPair> inventoryChannelProgressPairs = new LinkedList<>();
List<CDCChannelProgressPair> inventoryChannelProgressPairs = new CopyOnWriteArrayList<>();
AtomicBoolean incrementalImporterUsed = new AtomicBoolean();
List<CDCChannelProgressPair> incrementalChannelProgressPairs = new LinkedList<>();
for (CDCJobItemContext each : jobItemContexts) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,12 +44,13 @@
import org.apache.shardingsphere.data.pipeline.core.exception.param.PipelineInvalidParameterException;
import org.apache.shardingsphere.distsql.handler.exception.rule.MissingRequiredRuleException;
import org.apache.shardingsphere.infra.autogen.version.ShardingSphereVersion;
import org.apache.shardingsphere.infra.executor.audit.exception.SQLAuditException;
import org.apache.shardingsphere.infra.metadata.user.Grantee;
import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
import org.apache.shardingsphere.infra.exception.core.ShardingSpherePreconditions;
import org.apache.shardingsphere.infra.exception.core.external.sql.ShardingSphereSQLException;
import org.apache.shardingsphere.infra.exception.core.external.sql.sqlstate.XOpenSQLState;
import org.apache.shardingsphere.infra.exception.core.external.sql.type.kernel.category.PipelineSQLException;
import org.apache.shardingsphere.infra.executor.audit.exception.SQLAuditException;
import org.apache.shardingsphere.infra.metadata.user.Grantee;
import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
import org.apache.shardingsphere.proxy.backend.context.ProxyContext;

import java.net.InetSocketAddress;
Expand Down Expand Up @@ -172,8 +173,12 @@ private void processStreamDataRequest(final ChannelHandlerContext ctx, final CDC
throw new CDCExceptionWrapper(request.getRequestId(), new PipelineInvalidParameterException("Source schema table is empty"));
}
checkPrivileges(request.getRequestId(), connectionContext.getCurrentUser().getGrantee(), requestBody.getDatabase());
CDCResponse response = backendHandler.streamData(request.getRequestId(), requestBody, connectionContext, ctx.channel());
ctx.writeAndFlush(response);
try {
CDCResponse response = backendHandler.streamData(request.getRequestId(), requestBody, connectionContext, ctx.channel());
ctx.writeAndFlush(response);
} catch (final PipelineSQLException ex) {
throw new CDCExceptionWrapper(request.getRequestId(), ex);
}
}

private void processAckStreamingRequest(final CDCRequest request) {
Expand Down
Loading