[ https://issues.apache.org/jira/browse/FLINK-3311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15296337#comment-15296337 ]
ASF GitHub Bot commented on FLINK-3311: --------------------------------------- Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/1771#discussion_r64217845 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericAtLeastOnceSink.java --- @@ -0,0 +1,192 @@ +/** + * 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 + * <p/> + * http://www.apache.org/licenses/LICENSE-2.0 + * <p/> + * 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; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.runtime.io.disk.InputViewIterator; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.runtime.util.ReusingMutableToRegularIteratorWrapper; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTaskState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashSet; +import java.util.Set; +import java.util.TreeMap; +import java.util.UUID; + +/** + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing + * mechanism and can provide exactly-once guarantees; depending on the storage backend and sink/committer implementation. + * <p/> + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a + * checkpoint is completed. + * + * @param <IN> Type of the elements emitted by this sink + */ +public abstract class GenericAtLeastOnceSink<IN> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> { + protected static final Logger LOG = LoggerFactory.getLogger(GenericAtLeastOnceSink.class); + private final CheckpointCommitter committer; + private transient AbstractStateBackend.CheckpointStateOutputView out; + protected final TypeSerializer<IN> serializer; + private final String id; + + private ExactlyOnceState state = new ExactlyOnceState(); + + public GenericAtLeastOnceSink(CheckpointCommitter committer, TypeSerializer<IN> serializer, String jobID) throws Exception { + this.committer = committer; + this.serializer = serializer; + this.id = UUID.randomUUID().toString(); + this.committer.setJobId(jobID); + this.committer.createResource(); + } + + @Override + public void open() throws Exception { + committer.setOperatorId(id); + committer.setOperatorSubtaskId(getRuntimeContext().getIndexOfThisSubtask()); + committer.open(); + } + + public void close() throws Exception { + committer.close(); + } + + /** + * Saves a handle in the state. + * + * @param checkpointId + * @throws IOException + */ + private void saveHandleInState(final long checkpointId, final long timestamp) throws Exception { + //only add handle if a new OperatorState was created since the last snapshot + if (out != null) { + StateHandle<DataInputView> handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + //we already have a checkpoint stored for that ID that may have been partially written, + //so we discard this "alternate version" and use the stored checkpoint + handle.discardState(); + } else { + state.pendingHandles.put(checkpointId, new Tuple2<>(timestamp, handle)); + } + out = null; + } + } + + @Override + public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception { + StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp); + saveHandleInState(checkpointId, timestamp); + taskState.setFunctionState(state); + return taskState; + } + + @Override + public void restoreState(StreamTaskState state, long recoveryTimestamp) throws Exception { + super.restoreState(state, recoveryTimestamp); + this.state = (ExactlyOnceState) state.getFunctionState(); + + out = null; + } + + @Override + public void notifyOfCompletedCheckpoint(long checkpointId) throws Exception { + super.notifyOfCompletedCheckpoint(checkpointId); + + synchronized (state.pendingHandles) { + Set<Long> pastCheckpointIds = state.pendingHandles.keySet(); + Set<Long> checkpointsToRemove = new HashSet<>(); + for (Long pastCheckpointId : pastCheckpointIds) { + if (pastCheckpointId <= checkpointId) { + if (!committer.isCheckpointCommitted(pastCheckpointId)) { --- End diff -- I've implemented this. > Add a connector for streaming data into Cassandra > ------------------------------------------------- > > Key: FLINK-3311 > URL: https://issues.apache.org/jira/browse/FLINK-3311 > Project: Flink > Issue Type: New Feature > Components: Streaming Connectors > Reporter: Robert Metzger > Assignee: Andrea Sella > > We had users in the past asking for a Flink+Cassandra integration. > It seems that there is a well-developed java client for connecting into > Cassandra: https://github.com/datastax/java-driver (ASL 2.0) > There are also tutorials out there on how to start a local cassandra instance > (for the tests): > http://prettyprint.me/prettyprint.me/2010/02/14/running-cassandra-as-an-embedded-service/index.html > For the data types, I think we should support TupleX types, and map standard > java types to the respective cassandra types. > In addition, it seems that there is a object mapper from datastax to store > POJOs in Cassandra (there are annotations for defining the primary key and > types) -- This message was sent by Atlassian JIRA (v6.3.4#6332)