Skip to content

Commit

Permalink
[FLINK-19696] Add committer operator for BATCH mode for the new sink API
Browse files Browse the repository at this point in the history
This patch introduce two operators:
1. ‘BatchCommitterOperator’ is the runtime operator for executing the sink’s ‘Committer’ in the batch execution mode.
2. ‘BatchGlobalCommitterOperator’ is the runtime operator for executing the sink’s ‘GlobalCommitter’ in the batch execution mode

This closes apache#13703.
  • Loading branch information
guoweiM authored and kl0u committed Oct 24, 2020
1 parent ec04072 commit 8dba1a8
Show file tree
Hide file tree
Showing 6 changed files with 510 additions and 0 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* 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
*
* https://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.flink.streaming.runtime.operators.sink;

import org.apache.flink.api.connector.sink.Committer;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.BoundedOneInput;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;

import java.util.ArrayList;
import java.util.List;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Runtime {@link org.apache.flink.streaming.api.operators.StreamOperator} for executing
* {@link Committer} in the batch execution mode.
*
* @param <CommT> The committable type of the {@link Committer}.
*/
final class BatchCommitterOperator<CommT> extends AbstractStreamOperator<CommT>
implements OneInputStreamOperator<CommT, CommT>, BoundedOneInput {

/** Responsible for committing the committable to the external system. */
private final Committer<CommT> committer;

/** Record all the committables until the end of the input. */
private final List<CommT> allCommittables;

public BatchCommitterOperator(Committer<CommT> committer) {
this.committer = checkNotNull(committer);
this.allCommittables = new ArrayList<>();
}

@Override
public void processElement(StreamRecord<CommT> element) {
allCommittables.add(element.getValue());
}

@Override
public void endInput() throws Exception {
if (!allCommittables.isEmpty()) {
final List<CommT> neededRetryCommittables = committer.commit(allCommittables);
if (!neededRetryCommittables.isEmpty()) {
throw new UnsupportedOperationException("Currently does not support the re-commit!");
}
for (CommT committable : allCommittables) {
output.collect(new StreamRecord<>(committable));
}
}
}

@Override
public void close() throws Exception {
super.close();
committer.close();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
/*
* 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
*
* https://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.flink.streaming.runtime.operators.sink;

import org.apache.flink.api.connector.sink.Committer;
import org.apache.flink.api.connector.sink.Sink;
import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.StreamOperatorParameters;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* A {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} for
* {@link BatchCommitterOperator}.
*
* @param <CommT> The committable type of the {@link Committer}.
*/
public final class BatchCommitterOperatorFactory<CommT> extends AbstractStreamOperatorFactory<CommT>
implements OneInputStreamOperatorFactory<CommT, CommT> {

private final Sink<?, CommT, ?, ?> sink;

public BatchCommitterOperatorFactory(Sink<?, CommT, ?, ?> sink) {
this.sink = checkNotNull(sink);
}

@Override
@SuppressWarnings("unchecked")
public <T extends StreamOperator<CommT>> T createStreamOperator(
StreamOperatorParameters<CommT> parameters) {
final BatchCommitterOperator<CommT> committerOperator =
new BatchCommitterOperator<>(
sink.createCommitter().orElseThrow(
() -> new IllegalStateException(
"Could not create committer from the sink")));
committerOperator.setup(
parameters.getContainingTask(),
parameters.getStreamConfig(),
parameters.getOutput());
return (T) committerOperator;
}

@Override
@SuppressWarnings("rawtypes")
public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
return BatchCommitterOperator.class;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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
*
* https://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.flink.streaming.runtime.operators.sink;

import org.apache.flink.api.connector.sink.GlobalCommitter;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.BoundedOneInput;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;

import java.util.ArrayList;
import java.util.Collections;
import java.util.List;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Runtime {@link org.apache.flink.streaming.api.operators.StreamOperator} for executing
* {@link GlobalCommitter} in the batch execution mode.
*
* @param <CommT> The committable type of the {@link GlobalCommitter}
* @param <GlobalCommT> The committable type of the {@link GlobalCommitter}
*/
final class BatchGlobalCommitterOperator<CommT, GlobalCommT> extends AbstractStreamOperator<GlobalCommT>
implements OneInputStreamOperator<CommT, GlobalCommT>, BoundedOneInput {

/** Aggregate committables to global committables and commit the global committables to the external system. */
private final GlobalCommitter<CommT, GlobalCommT> globalCommitter;

/** Record all the committables until the end of the input. */
private final List<CommT> allCommittables;

BatchGlobalCommitterOperator(GlobalCommitter<CommT, GlobalCommT> globalCommitter) {
this.globalCommitter = checkNotNull(globalCommitter);
this.allCommittables = new ArrayList<>();
}

@Override
public void processElement(StreamRecord<CommT> element) {
allCommittables.add(element.getValue());
}

@Override
public void endInput() throws Exception {
if (!allCommittables.isEmpty()) {
final GlobalCommT globalCommittable = globalCommitter.combine(allCommittables);
final List<GlobalCommT> neededRetryCommittables = globalCommitter.commit(
Collections.singletonList(globalCommittable));
if (!neededRetryCommittables.isEmpty()) {
throw new UnsupportedOperationException("Currently does not support the re-commit!");
}
}
globalCommitter.endOfInput();
}

@Override
public void close() throws Exception {
super.close();
globalCommitter.close();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* 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
*
* https://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.flink.streaming.runtime.operators.sink;

import org.apache.flink.api.connector.sink.GlobalCommitter;
import org.apache.flink.api.connector.sink.Sink;
import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.StreamOperatorParameters;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* A {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} for
* {@link BatchGlobalCommitterOperator}.
*
* @param <CommT> The committable type of the {@link GlobalCommitter}
* @param <GlobalCommT> The committable type of the {@link GlobalCommitter}
*/
public final class BatchGlobalCommitterOperatorFactory<CommT, GlobalCommT> extends AbstractStreamOperatorFactory<GlobalCommT>
implements OneInputStreamOperatorFactory<CommT, GlobalCommT> {

private final Sink<?, CommT, ?, GlobalCommT> sink;

public BatchGlobalCommitterOperatorFactory(Sink<?, CommT, ?, GlobalCommT> sink) {
this.sink = checkNotNull(sink);
}

@Override
@SuppressWarnings("unchecked")
public <T extends StreamOperator<GlobalCommT>> T createStreamOperator(StreamOperatorParameters<GlobalCommT> parameters) {
final BatchGlobalCommitterOperator<CommT, GlobalCommT> batchGlobalCommitterOperator =
new BatchGlobalCommitterOperator<>(
sink.createGlobalCommitter().orElseThrow(
() -> new IllegalStateException(
"Could not create global committer from the sink")));

batchGlobalCommitterOperator.setup(
parameters.getContainingTask(),
parameters.getStreamConfig(),
parameters.getOutput());
return (T) batchGlobalCommitterOperator;
}

@Override
@SuppressWarnings("rawtypes")
public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
return BatchGlobalCommitterOperator.class;
}
}
Loading

0 comments on commit 8dba1a8

Please sign in to comment.