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

[Feature][CDC] Support listening for message delayed events in cdc source #6634

Merged
merged 1 commit into from
Apr 7, 2024
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 @@ -29,4 +29,5 @@ public enum EventType {
LIFECYCLE_READER_OPEN,
LIFECYCLE_READER_CLOSE,
LIFECYCLE_WRITER_CLOSE,
READER_MESSAGE_DELAYED,
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* 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.seatunnel.api.source.event;

import org.apache.seatunnel.api.event.Event;
import org.apache.seatunnel.api.event.EventType;

import lombok.AllArgsConstructor;
import lombok.Getter;
import lombok.NoArgsConstructor;
import lombok.Setter;
import lombok.ToString;

@Getter
@Setter
@ToString
@AllArgsConstructor
@NoArgsConstructor
public class MessageDelayedEvent implements Event {
private long createdTime;
private String jobId;
private EventType eventType = EventType.READER_MESSAGE_DELAYED;

private long delayTime;
private String record;

public MessageDelayedEvent(long delayTime) {
this(delayTime, null);
}

public MessageDelayedEvent(long delayTime, String record) {
this.delayTime = delayTime;
this.record = record;
this.createdTime = System.currentTimeMillis();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,20 +21,23 @@
import org.apache.seatunnel.api.event.EventListener;
import org.apache.seatunnel.api.source.Collector;
import org.apache.seatunnel.api.source.SourceReader;
import org.apache.seatunnel.api.source.event.MessageDelayedEvent;
import org.apache.seatunnel.api.table.event.SchemaChangeEvent;
import org.apache.seatunnel.connectors.cdc.base.source.event.CompletedSnapshotPhaseEvent;
import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset;
import org.apache.seatunnel.connectors.cdc.base.source.offset.OffsetFactory;
import org.apache.seatunnel.connectors.cdc.base.source.split.SourceRecords;
import org.apache.seatunnel.connectors.cdc.base.source.split.state.IncrementalSplitState;
import org.apache.seatunnel.connectors.cdc.base.source.split.state.SourceSplitStateBase;
import org.apache.seatunnel.connectors.cdc.base.utils.MessageDelayedEventLimiter;
import org.apache.seatunnel.connectors.cdc.debezium.DebeziumDeserializationSchema;
import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordEmitter;

import org.apache.kafka.connect.source.SourceRecord;

import lombok.extern.slf4j.Slf4j;

import java.time.Duration;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
Expand Down Expand Up @@ -71,6 +74,8 @@ public class IncrementalSourceRecordEmitter<T>
protected final Counter recordFetchDelay;
protected final Counter recordEmitDelay;
protected final EventListener eventListener;
protected final MessageDelayedEventLimiter delayedEventLimiter =
new MessageDelayedEventLimiter(Duration.ofSeconds(1), 0.5d);

public IncrementalSourceRecordEmitter(
DebeziumDeserializationSchema<T> debeziumDeserializationSchema,
Expand Down Expand Up @@ -113,6 +118,11 @@ protected void reportMetrics(SourceRecord element) {
// report emit delay
long emitDelay = now - messageTimestamp;
recordEmitDelay.set(emitDelay > 0 ? emitDelay : 0);

// limit the emit event frequency
if (delayedEventLimiter.acquire(messageTimestamp)) {
eventListener.onEvent(new MessageDelayedEvent(emitDelay, element.toString()));
}
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* 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.seatunnel.connectors.cdc.base.utils;

import org.apache.seatunnel.shade.com.google.common.util.concurrent.RateLimiter;

import lombok.AllArgsConstructor;

import java.time.Duration;

@AllArgsConstructor
public class MessageDelayedEventLimiter {
private final long delayMs;
private final RateLimiter eventRateLimiter;

public MessageDelayedEventLimiter(Duration delayThreshold) {
this(delayThreshold, 1);
}

public MessageDelayedEventLimiter(Duration delayThreshold, double permitsPerSecond) {
this.delayMs = delayThreshold.toMillis();
this.eventRateLimiter = RateLimiter.create(permitsPerSecond);
}

public boolean acquire(long messageCreateTime) {
if (isDelayed(messageCreateTime)) {
return eventRateLimiter.tryAcquire();
}
return false;
}

private boolean isDelayed(long messageCreateTime) {
return delayMs != 0 && System.currentTimeMillis() - messageCreateTime >= delayMs;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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.seatunnel.connectors.cdc.base.utils;

import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;

import java.time.Duration;
import java.util.concurrent.TimeUnit;

public class MessageDelayedEventLimiterTest {

@Test
public void testAcquire() throws InterruptedException {
double permitsPerSecond = 0.5;
Duration delayThreshold = Duration.ofMillis(1000);
MessageDelayedEventLimiter delayedEventLimiter =
new MessageDelayedEventLimiter(delayThreshold, permitsPerSecond);

long endTime = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10);
long actualAcquiredCount = 0;
while (System.currentTimeMillis() < endTime) {
boolean acquired =
delayedEventLimiter.acquire(
System.currentTimeMillis() - (delayThreshold.toMillis() * 10));
if (acquired) {
actualAcquiredCount++;
}
Thread.sleep(1);
}
long expectedAcquiredCount = (long) (TimeUnit.SECONDS.toSeconds(10) * permitsPerSecond);

Assertions.assertTrue(expectedAcquiredCount >= actualAcquiredCount);
}

@Test
public void testNoAcquire() throws InterruptedException {
double permitsPerSecond = 0.5;
Duration delayThreshold = Duration.ofMillis(1000);
MessageDelayedEventLimiter delayedEventLimiter =
new MessageDelayedEventLimiter(delayThreshold, permitsPerSecond);

long endTime = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10);
long actualAcquiredCount = 0;
while (System.currentTimeMillis() < endTime) {
boolean acquired = delayedEventLimiter.acquire(System.currentTimeMillis());
if (acquired) {
actualAcquiredCount++;
}
Thread.sleep(1);
}

Assertions.assertTrue(actualAcquiredCount == 0);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -250,6 +250,7 @@ private static boolean isSystemThread(String s) {
|| s.contains("Java2D Disposer")
|| s.contains("OkHttp ConnectionPool")
|| s.startsWith("http-report-event-scheduler")
|| s.startsWith("event-forwarder")
|| s.contains(
"org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner")
|| s.startsWith("Log4j2-TF-")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@
package org.apache.seatunnel.engine.server;

import org.apache.seatunnel.api.common.metrics.MetricTags;
import org.apache.seatunnel.api.event.Event;
import org.apache.seatunnel.common.utils.ExceptionUtils;
import org.apache.seatunnel.common.utils.RetryUtils;
import org.apache.seatunnel.common.utils.StringFormatUtils;
import org.apache.seatunnel.engine.common.Constant;
import org.apache.seatunnel.engine.common.config.ConfigProvider;
Expand All @@ -28,6 +30,7 @@
import org.apache.seatunnel.engine.common.utils.PassiveCompletableFuture;
import org.apache.seatunnel.engine.core.classloader.ClassLoaderService;
import org.apache.seatunnel.engine.core.job.ConnectorJarIdentifier;
import org.apache.seatunnel.engine.server.event.JobEventReportOperation;
import org.apache.seatunnel.engine.server.exception.TaskGroupContextNotFoundException;
import org.apache.seatunnel.engine.server.execution.ExecutionState;
import org.apache.seatunnel.engine.server.execution.ProgressState;
Expand All @@ -46,10 +49,12 @@
import org.apache.seatunnel.engine.server.task.SeaTunnelTask;
import org.apache.seatunnel.engine.server.task.TaskGroupImmutableInformation;
import org.apache.seatunnel.engine.server.task.operation.NotifyTaskStatusOperation;
import org.apache.seatunnel.engine.server.utils.NodeEngineUtil;

import org.apache.commons.collections4.CollectionUtils;

import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.hazelcast.instance.impl.NodeState;
import com.hazelcast.internal.metrics.DynamicMetricsProvider;
import com.hazelcast.internal.metrics.MetricDescriptor;
Expand All @@ -68,13 +73,15 @@

import java.io.IOException;
import java.net.URL;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
Expand Down Expand Up @@ -140,6 +147,9 @@ public class TaskExecutionService implements DynamicMetricsProvider {

private final ServerConnectorPackageClient serverConnectorPackageClient;

private final BlockingQueue<Event> eventBuffer;
private final ExecutorService eventForwardService;

public TaskExecutionService(
ClassLoaderService classLoaderService,
NodeEngineImpl nodeEngine,
Expand All @@ -165,6 +175,43 @@ public TaskExecutionService(

serverConnectorPackageClient =
new ServerConnectorPackageClient(nodeEngine, seaTunnelConfig);

eventBuffer = new ArrayBlockingQueue<>(2048);
eventForwardService =
Executors.newSingleThreadExecutor(
new ThreadFactoryBuilder().setNameFormat("event-forwarder-%d").build());
eventForwardService.submit(
() -> {
List<Event> events = new ArrayList<>();
RetryUtils.RetryMaterial retryMaterial =
new RetryUtils.RetryMaterial(2, true, e -> true);
while (!Thread.currentThread().isInterrupted()) {
try {
events.clear();

Event first = eventBuffer.take();
events.add(first);

eventBuffer.drainTo(events, 500);
JobEventReportOperation operation = new JobEventReportOperation(events);

RetryUtils.retryWithException(
() ->
NodeEngineUtil.sendOperationToMasterNode(
nodeEngine, operation)
.join(),
retryMaterial);

logger.fine("Event forward success, events " + events.size());
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
logger.info("Event forward thread interrupted");
} catch (Throwable t) {
logger.warning(
"Event forward failed, discard events " + events.size(), t);
}
}
});
}

public void start() {
Expand All @@ -175,6 +222,7 @@ public void shutdown() {
isRunning = false;
executorService.shutdownNow();
scheduledExecutorService.shutdown();
eventForwardService.shutdownNow();
}

public TaskGroupContext getExecutionContext(TaskGroupLocation taskGroupLocation) {
Expand Down Expand Up @@ -619,6 +667,13 @@ public void printTaskExecutionRuntimeInfo() {
}
}

public void reportEvent(Event e) {
while (!eventBuffer.offer(e)) {
eventBuffer.poll();
logger.warning("Event buffer is full, discard the oldest event");
}
}

private final class BlockingWorker implements Runnable {

private final TaskTracker tracker;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.seatunnel.engine.server.event;

import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting;
import org.apache.seatunnel.shade.com.google.common.util.concurrent.ThreadFactoryBuilder;

import org.apache.seatunnel.api.event.Event;
Expand Down Expand Up @@ -104,7 +105,8 @@ public void handle(Event event) {
completionStage.toCompletableFuture().join();
}

private void report() throws IOException {
@VisibleForTesting
synchronized void report() throws IOException {
long headSequence = ringbuffer.headSequence();
if (headSequence > committedEventIndex) {
log.warn(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ public class JobEventListener implements EventListener {
@Override
public void onEvent(Event event) {
event.setJobId(String.valueOf(taskLocation.getJobId()));
JobEventReportOperation evenCollectOperation = new JobEventReportOperation(event);
taskExecutionContext.sendToMaster(evenCollectOperation).join();

taskExecutionContext.getTaskExecutionService().reportEvent(event);
}
}
Loading
Loading