Skip to content
Open
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 @@ -89,6 +89,7 @@ public JobClient build() throws Exception {
tieringSourceBuilder.withPollTieringTableIntervalMs(
flussConfig.get(POLL_TIERING_TABLE_INTERVAL).toMillis());
}

TieringSource<?> tieringSource = tieringSourceBuilder.build();
DataStreamSource<?> source =
env.fromSource(
Expand Down
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.fluss.flink.tiering.event;

import org.apache.flink.api.connector.source.SourceEvent;

import java.util.Objects;

/**
* SourceEvent used to notify TieringSourceReader that a table has reached the maximum tiering
* duration and should be force completed.
*/
public class TieringReachMaxDurationEvent implements SourceEvent {

private static final long serialVersionUID = 1L;

private final long tableId;

public TieringReachMaxDurationEvent(long tableId) {
this.tableId = tableId;
}

public long getTableId() {
return tableId;
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (!(o instanceof TieringReachMaxDurationEvent)) {
return false;
}
TieringReachMaxDurationEvent that = (TieringReachMaxDurationEvent) o;
return tableId == that.tableId;
}

@Override
public int hashCode() {
return Objects.hashCode(tableId);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,8 @@
import org.apache.flink.api.connector.source.SourceReaderContext;
import org.apache.flink.api.connector.source.SplitEnumerator;
import org.apache.flink.api.connector.source.SplitEnumeratorContext;
import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
import org.apache.flink.core.io.SimpleVersionedSerializer;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.streaming.api.graph.StreamGraphHasherV2;
Expand Down Expand Up @@ -78,16 +80,15 @@ public Boundedness getBoundedness() {

@Override
public SplitEnumerator<TieringSplit, TieringSourceEnumeratorState> createEnumerator(
SplitEnumeratorContext<TieringSplit> splitEnumeratorContext) throws Exception {
SplitEnumeratorContext<TieringSplit> splitEnumeratorContext) {
return new TieringSourceEnumerator(
flussConf, splitEnumeratorContext, pollTieringTableIntervalMs);
}

@Override
public SplitEnumerator<TieringSplit, TieringSourceEnumeratorState> restoreEnumerator(
SplitEnumeratorContext<TieringSplit> splitEnumeratorContext,
TieringSourceEnumeratorState tieringSourceEnumeratorState)
throws Exception {
TieringSourceEnumeratorState tieringSourceEnumeratorState) {
// stateless operator
return new TieringSourceEnumerator(
flussConf, splitEnumeratorContext, pollTieringTableIntervalMs);
Expand All @@ -107,8 +108,11 @@ public SimpleVersionedSerializer<TieringSplit> getSplitSerializer() {
@Override
public SourceReader<TableBucketWriteResult<WriteResult>, TieringSplit> createReader(
SourceReaderContext sourceReaderContext) {
FutureCompletingBlockingQueue<RecordsWithSplitIds<TableBucketWriteResult<WriteResult>>>
elementsQueue = new FutureCompletingBlockingQueue<>();
Connection connection = ConnectionFactory.createConnection(flussConf);
return new TieringSourceReader<>(sourceReaderContext, connection, lakeTieringFactory);
return new TieringSourceReader<>(
elementsQueue, sourceReaderContext, connection, lakeTieringFactory);
}

/** This follows the operator uid hash generation logic of flink {@link StreamGraphHasherV2}. */
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
/*
* 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.fluss.flink.tiering.source;

import org.apache.fluss.flink.adapter.SingleThreadFetcherManagerAdapter;
import org.apache.fluss.flink.tiering.source.split.TieringSplit;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher;
import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherTask;
import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;

import java.util.Collection;
import java.util.function.Consumer;
import java.util.function.Supplier;

/**
* The SplitFetcherManager for tiering source. This class is needed to help notify a table reaches
* the max duration of tiering to {@link TieringSplitReader}.
*/
public class TieringSourceFetcherManager<WriteResult>
extends SingleThreadFetcherManagerAdapter<
TableBucketWriteResult<WriteResult>, TieringSplit> {

public TieringSourceFetcherManager(
FutureCompletingBlockingQueue<RecordsWithSplitIds<TableBucketWriteResult<WriteResult>>>
elementsQueue,
Supplier<SplitReader<TableBucketWriteResult<WriteResult>, TieringSplit>>
splitReaderSupplier,
Configuration configuration,
Consumer<Collection<String>> splitFinishedHook) {
super(elementsQueue, splitReaderSupplier, configuration, splitFinishedHook);
}

public void markTableReachTieringMaxDuration(long tableId) {
if (!fetchers.isEmpty()) {
// The fetcher thread is still running. This should be the majority of the cases.
fetchers.values()
.forEach(
splitFetcher ->
enqueueMarkTableReachTieringMaxDurationTask(
splitFetcher, tableId));
} else {
SplitFetcher<TableBucketWriteResult<WriteResult>, TieringSplit> splitFetcher =
createSplitFetcher();
enqueueMarkTableReachTieringMaxDurationTask(splitFetcher, tableId);
startFetcher(splitFetcher);
}
}

private void enqueueMarkTableReachTieringMaxDurationTask(
SplitFetcher<TableBucketWriteResult<WriteResult>, TieringSplit> splitFetcher,
long reachTieringDeadlineTable) {
splitFetcher.enqueueTask(
new SplitFetcherTask() {
@Override
public boolean run() {
((TieringSplitReader<WriteResult>) splitFetcher.getSplitReader())
.handleTableReachTieringMaxDuration(reachTieringDeadlineTable);
return true;
}

@Override
public void wakeUp() {
// do nothing
}
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,23 +18,31 @@
package org.apache.fluss.flink.tiering.source;

import org.apache.fluss.annotation.Internal;
import org.apache.fluss.annotation.VisibleForTesting;
import org.apache.fluss.client.Connection;
import org.apache.fluss.flink.adapter.SingleThreadMultiplexSourceReaderBaseAdapter;
import org.apache.fluss.flink.tiering.event.TieringReachMaxDurationEvent;
import org.apache.fluss.flink.tiering.source.split.TieringSplit;
import org.apache.fluss.flink.tiering.source.state.TieringSplitState;
import org.apache.fluss.lake.writer.LakeTieringFactory;

import org.apache.flink.api.connector.source.SourceEvent;
import org.apache.flink.api.connector.source.SourceReader;
import org.apache.flink.api.connector.source.SourceReaderContext;
import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase;
import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;

import java.time.Duration;
import java.util.Collections;
import java.util.List;
import java.util.Map;

import static org.apache.fluss.flink.tiering.source.TieringSplitReader.DEFAULT_POLL_TIMEOUT;

/** A {@link SourceReader} that read records from Fluss and write to lake. */
@Internal
public final class TieringSourceReader<WriteResult>
extends SingleThreadMultiplexSourceReaderBase<
extends SingleThreadMultiplexSourceReaderBaseAdapter<
TableBucketWriteResult<WriteResult>,
TableBucketWriteResult<WriteResult>,
TieringSplit,
Expand All @@ -43,11 +51,29 @@ public final class TieringSourceReader<WriteResult>
private final Connection connection;

public TieringSourceReader(
FutureCompletingBlockingQueue<RecordsWithSplitIds<TableBucketWriteResult<WriteResult>>>
elementsQueue,
SourceReaderContext context,
Connection connection,
LakeTieringFactory<WriteResult, ?> lakeTieringFactory) {
this(elementsQueue, context, connection, lakeTieringFactory, DEFAULT_POLL_TIMEOUT);
}

@VisibleForTesting
TieringSourceReader(
FutureCompletingBlockingQueue<RecordsWithSplitIds<TableBucketWriteResult<WriteResult>>>
elementsQueue,
SourceReaderContext context,
Connection connection,
LakeTieringFactory<WriteResult, ?> lakeTieringFactory,
Duration pollTimeout) {
super(
() -> new TieringSplitReader<>(connection, lakeTieringFactory),
elementsQueue,
new TieringSourceFetcherManager<>(
elementsQueue,
() -> new TieringSplitReader<>(connection, lakeTieringFactory, pollTimeout),
context.getConfiguration(),
(ignore) -> {}),
new TableBucketWriteResultEmitter<>(),
context.getConfiguration(),
context);
Expand Down Expand Up @@ -89,6 +115,17 @@ protected TieringSplit toSplitType(String splitId, TieringSplitState splitState)
return splitState.toSourceSplit();
}

@Override
public void handleSourceEvents(SourceEvent sourceEvent) {
if (sourceEvent instanceof TieringReachMaxDurationEvent) {
TieringReachMaxDurationEvent reachMaxDurationEvent =
(TieringReachMaxDurationEvent) sourceEvent;
long tableId = reachMaxDurationEvent.getTableId();
((TieringSourceFetcherManager<WriteResult>) splitFetcherManager)
.markTableReachTieringMaxDuration(tableId);
}
}

@Override
public void close() throws Exception {
super.close();
Expand Down
Loading