Skip to content

Conversation

@stevenzwu
Copy link
Contributor

No description provided.

@stevenzwu
Copy link
Contributor Author

This PR has no unit test as it mainly contains the plumbing classes for Iceberg source. We will have full test with Iceberg source.

@github-actions github-actions bot added the flink label Mar 4, 2022

IcebergSourceSplit nextSplit = splits.poll();
if (nextSplit == null) {
throw new IOException("No split remaining");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe add a comment to explain why this is the right behavior. It seems odd to me that it should throw an exception when it hasn't received new splits. Maybe that is the expectation from how fetch is called, but it is definitely concerning to see.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a great question. I looked into Flink code again. Throwing an IOException will cause the fetcher to be closed so that we don't have to poll the fetcher again. that is not a problem, because new split later will create new fetcher.

This is the FetchTask that calls splitReader.fetch. We can see that the return value can't be null. I agree that this is a little weird behavior. cc @tweise

    public boolean run() throws IOException {
        try {
            if (!isWakenUp() && lastRecords == null) {
                lastRecords = splitReader.fetch();
            }

            if (!isWakenUp()) {
                // The order matters here. We must first put the last records into the queue.
                // This ensures the handling of the fetched records is atomic to wakeup.
                if (elementsQueue.put(fetcherIndex, lastRecords)) {
                    if (!lastRecords.finishedSplits().isEmpty()) {
                        // The callback does not throw InterruptedException.
                        splitFinishedCallback.accept(lastRecords.finishedSplits());
                    }
                    lastRecords = null;
                }
            }
        }

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

aded a method comment section to explain the IOException behavior

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is probably better as an inline comment.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

changed to inline comment

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor Author

@stevenzwu stevenzwu Mar 25, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@yittg that is a good question. In theory, I agree it is a good behavior. I haven't figured a good way to implement the wakeUp behavior

Say there is no split available from the queue. if fetch() method doesn't return, it would need to poll the splits queue to check for new split in some loop with sleep. if the wakerUp is called and set the wakeUp flag for the fetch method to break out of the loop and return an empty RecordsWithSplitIds with no records and no finished splits.

We can't use wait-notify to signal btw wakeUp and fetch. Otherwise, fetch method won't do a splits queue polling and discover new splits during wait. That is also undesirable.

Actually fetch, add splits, wakeup tasks are all enqueued executed by SingleThreadFetcherManager. not sure if we can leverage notify here.

Maybe there is other way I haven't thought about.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My colleague suggested a good way. We can just return an empty result in this case. it will cause fetcher to be idle and SplitFetcherManager will closes idle fetcher. At least, it is like a normal flow (than IOException). I pushed a commit for the change.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SGTM roughly, thanks

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Forgot to mention. For the change of return an empty result for no split case, it has been validated by the unit test in the uber draft PR, which contains MiniCluster unit tests for both batch and streaming executions. #2105

private IcebergSourceSplit currentSplit;
private String currentSplitId;

IcebergSourceSplitReader(ReaderFunction<T> readerFunction,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It may be better to call this openSplitFunction?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I still feel ReaderFunction is more accurate. here is the interface from previously merged PR. It basically read a split and return an iterator of record batch.

public interface ReaderFunction<T> extends Serializable,
    Function<IcebergSourceSplit, CloseableIterator<RecordsWithSplitIds<RecordAndPosition<T>>>> {
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

renamed the variable to openSplitFunction

}

/** Sets the position without setting a record. */
public void position(int newFileOffset, long newRecordOffset) {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

removing an unused method

@stevenzwu stevenzwu force-pushed the SourceReader branch 2 times, most recently from efbd7a7 to 849346d Compare March 14, 2022 04:08
public void handleSplitsChanges(SplitsChange<IcebergSourceSplit> splitsChange) {
if (!(splitsChange instanceof SplitsAddition)) {
throw new UnsupportedOperationException(String.format(
"The SplitChange type of %s is not supported.", splitsChange.getClass()));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unsupported split change: %s

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thx. done

"The SplitChange type of %s is not supported.", splitsChange.getClass()));
}

LOG.info("Add splits to reader: {}", splitsChange.splits());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

May want to convert this to a count rather than locations.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

…e split fetcher to be idle. SplitFetcherManager closes idle fetcher.
@rdblue rdblue merged commit 86b42fe into apache:master Apr 1, 2022
kbendick pushed a commit to kbendick/iceberg that referenced this pull request Apr 4, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants