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

Pipe: support reporting progress by UserDefinedEvent #12000

Merged
merged 7 commits into from
Jan 29, 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
@@ -0,0 +1,58 @@
/*
* 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.iotdb.pipe.api.event;

/**
* User defined event is used to wrap data generated by users, keeping a source event to
* automatically report the processing progress to pipe engine.
*/
public abstract class UserDefinedEvent implements Event {

/** The user defined event is generated from this source event. */
protected final Event sourceEvent;

/**
* @param sourceEvent The source event of this user defined event which is used to report the
* processing progress to pipe engine. Please notice that the source event should satisfy the
* following conditions: 1. A source event can only be assigned to one user defined event. 2.
* If more than one user defined events are generated from the same source event, only the
* last generated user defined event can be assigned with the source event, others should be
* assigned {@code null}, or call {@link #UserDefinedEvent()} to generate a user defined event
* without source event.
*/
protected UserDefinedEvent(Event sourceEvent) {
this.sourceEvent = parseRootSourceEvent(sourceEvent);
}

/** Generate a user defined event without source event. */
protected UserDefinedEvent() {
this.sourceEvent = null;
}

private Event parseRootSourceEvent(Event sourceEvent) {
return sourceEvent instanceof UserDefinedEvent
? ((UserDefinedEvent) sourceEvent).getSourceEvent()
: sourceEvent;
}

public Event getSourceEvent() {
return sourceEvent;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -175,6 +175,10 @@ public final String getPipeName() {
return pipeName;
}

public final PipeTaskMeta getPipeTaskMeta() {
return pipeTaskMeta;
}

/**
* Get the pattern of this event.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/*
* 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.iotdb.db.pipe.event;

import org.apache.iotdb.commons.consensus.index.ProgressIndex;
import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta;
import org.apache.iotdb.pipe.api.event.Event;
import org.apache.iotdb.pipe.api.event.UserDefinedEvent;

public class UserDefinedEnrichedEvent extends EnrichedEvent {

private final UserDefinedEvent userDefinedEvent;
private final EnrichedEvent enrichedEvent;

public static Event maybeOf(Event event) {
return event instanceof UserDefinedEvent
&& ((UserDefinedEvent) event).getSourceEvent() instanceof EnrichedEvent
? new UserDefinedEnrichedEvent(
(UserDefinedEvent) event, (EnrichedEvent) ((UserDefinedEvent) event).getSourceEvent())
: event;
}

private UserDefinedEnrichedEvent(UserDefinedEvent userDefinedEvent, EnrichedEvent enrichedEvent) {
super(
enrichedEvent.getPipeName(),
enrichedEvent.getPipeTaskMeta(),
enrichedEvent.getPattern(),
enrichedEvent.getStartTime(),
enrichedEvent.getEndTime());
this.userDefinedEvent = userDefinedEvent;
this.enrichedEvent = enrichedEvent;
}

public UserDefinedEvent getUserDefinedEvent() {
return userDefinedEvent;
}

@Override
public boolean internallyIncreaseResourceReferenceCount(String holderMessage) {
return enrichedEvent.internallyIncreaseResourceReferenceCount(holderMessage);
}

@Override
public boolean internallyDecreaseResourceReferenceCount(String holderMessage) {
return enrichedEvent.internallyDecreaseResourceReferenceCount(holderMessage);
}

@Override
public ProgressIndex getProgressIndex() {
return enrichedEvent.getProgressIndex();
}

@Override
public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport(
String pipeName, PipeTaskMeta pipeTaskMeta, String pattern, long startTime, long endTime) {
return enrichedEvent.shallowCopySelfAndBindPipeTaskMetaForProgressReport(
pipeName, pipeTaskMeta, pattern, startTime, endTime);
}

@Override
public boolean isGeneratedByPipe() {
return enrichedEvent.isGeneratedByPipe();
}

@Override
public boolean isEventTimeOverlappedWithTimeRange() {
return enrichedEvent.isEventTimeOverlappedWithTimeRange();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue;
import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBThriftAsyncConnector;
import org.apache.iotdb.db.pipe.event.EnrichedEvent;
import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent;
import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent;
import org.apache.iotdb.db.pipe.metric.PipeConnectorMetrics;
import org.apache.iotdb.db.pipe.task.connection.PipeEventCollector;
Expand Down Expand Up @@ -120,7 +121,10 @@ protected boolean executeOnce() {
return false;
}

final Event event = lastEvent != null ? lastEvent : inputPendingQueue.waitedPoll();
final Event event =
lastEvent != null
? lastEvent
: UserDefinedEnrichedEvent.maybeOf(inputPendingQueue.waitedPoll());
// Record this event for retrying on connection failure or other exceptions
setLastEvent(event);

Expand All @@ -142,7 +146,10 @@ protected boolean executeOnce() {
} else if (event instanceof PipeHeartbeatEvent) {
transferHeartbeatEvent((PipeHeartbeatEvent) event);
} else {
outputPipeConnector.transfer(event);
outputPipeConnector.transfer(
event instanceof UserDefinedEnrichedEvent
? ((UserDefinedEnrichedEvent) event).getUserDefinedEvent()
: event);
}

releaseLastEvent(true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import org.apache.iotdb.commons.pipe.execution.scheduler.PipeSubtaskScheduler;
import org.apache.iotdb.commons.pipe.task.EventSupplier;
import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent;
import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent;
import org.apache.iotdb.db.pipe.metric.PipeProcessorMetrics;
import org.apache.iotdb.db.pipe.task.connection.PipeEventCollector;
Expand Down Expand Up @@ -97,7 +98,10 @@ protected boolean executeOnce() throws Exception {
return false;
}

final Event event = lastEvent != null ? lastEvent : inputEventSupplier.supply();
final Event event =
lastEvent != null
? lastEvent
: UserDefinedEnrichedEvent.maybeOf(inputEventSupplier.supply());
// Record the last event for retry when exception occurs
setLastEvent(event);
if (
Expand Down Expand Up @@ -125,7 +129,11 @@ protected boolean executeOnce() throws Exception {
((PipeHeartbeatEvent) event).onProcessed();
PipeProcessorMetrics.getInstance().markPipeHeartbeatEvent(taskID);
} else {
pipeProcessor.process(event, outputEventCollector);
pipeProcessor.process(
event instanceof UserDefinedEnrichedEvent
? ((UserDefinedEnrichedEvent) event).getUserDefinedEvent()
: event,
outputEventCollector);
}
}

Expand Down
Loading