You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by zentol <gi...@git.apache.org> on 2016/02/10 16:19:38 UTC

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

GitHub user zentol opened a pull request:

    https://github.com/apache/flink/pull/1620

    [FLINK-3332] Cassandra connector

    This PR adds an Exactly-Once Cassandra Sink.
    
    The Exactly-once guarantee is made by saving incoming records in the OperatorState, and only committing them into Cassandra when a checkpoint completes. Note that a job failure while the data is being committed will cause duplicate data to be committed, but the chance of this happening is much smaller than for a naive At-Least-once implementation.
    
    The CassandraExactlyOnceSink is implemented as a custom operator to get access to the Statebackend. Values are committed with single inserts using a PreparedStatement that is supplied by the user, similiar to the Batch JDBC-Outputformat.
    
    The Exactly-Once logic is completely contained in a GenericExactlyOnceSink class that can be used by virtually every sink, requiring no knowledge about the checkpointing mechamism.
    
    The CassandraExactlyOnceSink and GenericExactlyOnceSink are covered by tests that use the OneInputStreamTaskHarness to generate a task environment, verifying that stored data is discarded when a state is restored; all data is being committed when a notify is missed; and of course that everything works when nothing goes wrong.
    
    Note: This PR currently subsumes #1609 (the change to ResultPartitionWriter), so that the tests run properly.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/zentol/flink 3332_cassandra

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1620.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1620
    
----
commit 64f0b32c9292f1c5957badbcee30476b663eb5a1
Author: zentol <s....@web.de>
Date:   2016-02-10T13:14:18Z

    [FLINK-3332] Cassandra connector

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52623142
  
    --- Diff: flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraExactlyOnceSink.java ---
    @@ -0,0 +1,100 @@
    +/**
    + * 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.connectors.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSet;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.google.common.util.concurrent.FutureCallback;
    +import com.google.common.util.concurrent.Futures;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.streaming.runtime.operators.GenericExactlyOnceSink;
    +
    +/**
    + * Sink that emits its input elements into a Cassandra database. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public class CassandraExactlyOnceSink<IN extends Tuple> extends GenericExactlyOnceSink<IN> {
    --- End diff --
    
    Why does it only works for `Tuple` and not a generic type?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52630950
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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);
    +		saveHandleInState(checkpointId);
    +
    +		synchronized (state.pendingHandles) {
    +			Set<Long> pastCheckpointIds = state.pendingHandles.keySet();
    +			Set<Long> checkpointsToRemove = new HashSet<>();
    +			for (Long pastCheckpointId : pastCheckpointIds) {
    +				if (pastCheckpointId <= checkpointId) {
    +					List<StateHandle<DataInputView>> handles = state.pendingHandles.get(pastCheckpointId);
    +					for (StateHandle<DataInputView> handle : handles) {
    +						DataInputView in = handle.getState(getUserCodeClassloader());
    +						while (in.readByte() == 1) {
    +							IN value = serializer.deserialize(in);
    +							sendValue(value);
    +						}
    +						checkpointsToRemove.add(pastCheckpointId);
    +					}
    +				}
    +			}
    +			for (Long toRemove : checkpointsToRemove) {
    +				state.pendingHandles.remove(toRemove);
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Write the given element into the backend.
    +	 * @param value value to be written
    +	 * @throws Exception
    +     */
    +	protected abstract void sendValue(IN value) throws Exception;
    +
    +	@Override
    +	public void processElement(StreamRecord<IN> element) throws Exception {
    +		IN value = element.getValue();
    +		if (serializer == null) {
    +			typeInfo = TypeExtractor.getForObject(value);
    +			serializer = typeInfo.createSerializer(new ExecutionConfig());
    --- End diff --
    
    the executionconfig is sent along the cluster, correct? so i could make the TypeInfo transient, and call createSerializer with the proper executionconfig.
    
    The user shouldn't have to pass the serializer separately, we should instantiate that one on our own.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52630685
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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 {
    --- End diff --
    
    I guess I have missed something, but how do prevent the following from happening: You've completed a checkpoint and written all the elements up to this point to the external system. Now the processing continues, but before you can complete the next checkpoint, the system crashes. As a result, your state will be rolled back to the latest checkpoint. However, this means that you will write all elements from the last checkpoint again if the next completed checkpoint message arrives, won't you?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52623666
  
    --- Diff: flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraExactlyOnceSink.java ---
    @@ -0,0 +1,100 @@
    +/**
    + * 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.connectors.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSet;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.google.common.util.concurrent.FutureCallback;
    +import com.google.common.util.concurrent.Futures;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.streaming.runtime.operators.GenericExactlyOnceSink;
    +
    +/**
    + * Sink that emits its input elements into a Cassandra database. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public class CassandraExactlyOnceSink<IN extends Tuple> extends GenericExactlyOnceSink<IN> {
    --- End diff --
    
    you're right, i forgot about POJO's.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52634496
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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 {
    --- End diff --
    
    I have to look into this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52631298
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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);
    +		saveHandleInState(checkpointId);
    +
    +		synchronized (state.pendingHandles) {
    +			Set<Long> pastCheckpointIds = state.pendingHandles.keySet();
    +			Set<Long> checkpointsToRemove = new HashSet<>();
    +			for (Long pastCheckpointId : pastCheckpointIds) {
    +				if (pastCheckpointId <= checkpointId) {
    +					List<StateHandle<DataInputView>> handles = state.pendingHandles.get(pastCheckpointId);
    +					for (StateHandle<DataInputView> handle : handles) {
    +						DataInputView in = handle.getState(getUserCodeClassloader());
    +						while (in.readByte() == 1) {
    +							IN value = serializer.deserialize(in);
    +							sendValue(value);
    +						}
    +						checkpointsToRemove.add(pastCheckpointId);
    +					}
    +				}
    +			}
    +			for (Long toRemove : checkpointsToRemove) {
    +				state.pendingHandles.remove(toRemove);
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Write the given element into the backend.
    +	 * @param value value to be written
    +	 * @throws Exception
    +     */
    +	protected abstract void sendValue(IN value) throws Exception;
    +
    +	@Override
    +	public void processElement(StreamRecord<IN> element) throws Exception {
    +		IN value = element.getValue();
    +		if (serializer == null) {
    +			typeInfo = TypeExtractor.getForObject(value);
    +			serializer = typeInfo.createSerializer(new ExecutionConfig());
    --- End diff --
    
    Of course you should automatically extract the `TypeInformation` and then create the corresponding `TypeSerializer` when you create the sink. But you shouldn't pass the `TypeInformation` into the sink and ship it to the cluster.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52625293
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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);
    +		saveHandleInState(checkpointId);
    --- End diff --
    
    I'm not so sure whether this is correct to do here. I think calling `saveHandleInState` will also include new elements which have arrived after the `checkpointId` barrier in the current `checkpointId` state. This would entail duplicate elements in case of a failure.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52627131
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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);
    +		saveHandleInState(checkpointId);
    +
    +		synchronized (state.pendingHandles) {
    +			Set<Long> pastCheckpointIds = state.pendingHandles.keySet();
    +			Set<Long> checkpointsToRemove = new HashSet<>();
    +			for (Long pastCheckpointId : pastCheckpointIds) {
    +				if (pastCheckpointId <= checkpointId) {
    +					List<StateHandle<DataInputView>> handles = state.pendingHandles.get(pastCheckpointId);
    +					for (StateHandle<DataInputView> handle : handles) {
    +						DataInputView in = handle.getState(getUserCodeClassloader());
    +						while (in.readByte() == 1) {
    +							IN value = serializer.deserialize(in);
    +							sendValue(value);
    +						}
    +						checkpointsToRemove.add(pastCheckpointId);
    +					}
    +				}
    +			}
    +			for (Long toRemove : checkpointsToRemove) {
    +				state.pendingHandles.remove(toRemove);
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Write the given element into the backend.
    +	 * @param value value to be written
    +	 * @throws Exception
    +     */
    +	protected abstract void sendValue(IN value) throws Exception;
    +
    +	@Override
    +	public void processElement(StreamRecord<IN> element) throws Exception {
    +		IN value = element.getValue();
    +		if (serializer == null) {
    +			typeInfo = TypeExtractor.getForObject(value);
    +			serializer = typeInfo.createSerializer(new ExecutionConfig());
    +		}
    +		//generate initial operator state
    +		if (out == null) {
    +			out = getStateBackend().createCheckpointStateOutputView(0, 0);
    +		}
    +		out.writeByte(1);
    +		serializer.serialize(value, out);
    +	}
    +
    +	@Override
    +	public void processWatermark(Watermark mark) throws Exception {
    +		//don't do anything, since no are a sink
    +	}
    +
    +	/**
    +	 * This state is used to keep a list of all StateHandles (essentially references to past OperatorStates) that were
    +	 * used since the last completed checkpoint.
    +	 **/
    +	public class ExactlyOnceState implements StateHandle<Serializable> {
    +		protected HashMap<Long, ArrayList<StateHandle<DataInputView>>> pendingHandles;
    +
    +		public ExactlyOnceState() {
    +			pendingHandles = new HashMap<>();
    +		}
    +
    +		@Override
    +		public HashMap<Long, ArrayList<StateHandle<DataInputView>>> getState(ClassLoader userCodeClassLoader) throws Exception {
    +			return pendingHandles;
    +		}
    +
    +		@Override
    +		public void discardState() throws Exception {
    +			pendingHandles = new HashMap<>();
    +		}
    +
    +		@Override
    +		public long getStateSize() throws Exception {
    +			return 0;
    --- End diff --
    
    We should be able to calculate the state as the sum of the state handles stored in `pendingHandles`, shouldn't we?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52622167
  
    --- Diff: flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraExactlyOnceSink.java ---
    @@ -0,0 +1,100 @@
    +/**
    + * 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.connectors.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSet;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.google.common.util.concurrent.FutureCallback;
    +import com.google.common.util.concurrent.Futures;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.streaming.runtime.operators.GenericExactlyOnceSink;
    +
    +/**
    + * Sink that emits its input elements into a Cassandra database. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public class CassandraExactlyOnceSink<IN extends Tuple> extends GenericExactlyOnceSink<IN> {
    +	private String host;
    +	private String createQuery;
    +	private String insertQuery;
    +
    +	private transient Cluster cluster;
    +	private transient Session session;
    +	private transient PreparedStatement preparedStatement;
    +
    +	private transient Throwable exception = null;
    +
    +	public CassandraExactlyOnceSink(String host, String insertQuery) {
    +		this(host, null, insertQuery);
    +	}
    +
    +	public CassandraExactlyOnceSink(String host, String createQuery, String insertQuery) {
    +		this.host = host;
    +		this.createQuery = createQuery;
    +		this.insertQuery = insertQuery;
    --- End diff --
    
    Adding null checks and non empty checks for the `host` and the `insertQuery` could be helpful.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52623063
  
    --- Diff: flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraExactlyOnceSink.java ---
    @@ -0,0 +1,100 @@
    +/**
    + * 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.connectors.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSet;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.google.common.util.concurrent.FutureCallback;
    +import com.google.common.util.concurrent.Futures;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.streaming.runtime.operators.GenericExactlyOnceSink;
    +
    +/**
    + * Sink that emits its input elements into a Cassandra database. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    --- End diff --
    
    param description missing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52625364
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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);
    +		saveHandleInState(checkpointId);
    +
    +		synchronized (state.pendingHandles) {
    +			Set<Long> pastCheckpointIds = state.pendingHandles.keySet();
    +			Set<Long> checkpointsToRemove = new HashSet<>();
    +			for (Long pastCheckpointId : pastCheckpointIds) {
    +				if (pastCheckpointId <= checkpointId) {
    +					List<StateHandle<DataInputView>> handles = state.pendingHandles.get(pastCheckpointId);
    +					for (StateHandle<DataInputView> handle : handles) {
    +						DataInputView in = handle.getState(getUserCodeClassloader());
    +						while (in.readByte() == 1) {
    +							IN value = serializer.deserialize(in);
    +							sendValue(value);
    +						}
    +						checkpointsToRemove.add(pastCheckpointId);
    +					}
    +				}
    +			}
    +			for (Long toRemove : checkpointsToRemove) {
    +				state.pendingHandles.remove(toRemove);
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Write the given element into the backend.
    +	 * @param value value to be written
    +	 * @throws Exception
    +     */
    +	protected abstract void sendValue(IN value) throws Exception;
    +
    +	@Override
    +	public void processElement(StreamRecord<IN> element) throws Exception {
    +		IN value = element.getValue();
    +		if (serializer == null) {
    +			typeInfo = TypeExtractor.getForObject(value);
    +			serializer = typeInfo.createSerializer(new ExecutionConfig());
    +		}
    +		//generate initial operator state
    +		if (out == null) {
    +			out = getStateBackend().createCheckpointStateOutputView(0, 0);
    +		}
    +		out.writeByte(1);
    +		serializer.serialize(value, out);
    +	}
    +
    +	@Override
    +	public void processWatermark(Watermark mark) throws Exception {
    +		//don't do anything, since no are a sink
    --- End diff --
    
    typo: no = we


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52621606
  
    --- Diff: flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraExactlyOnceSink.java ---
    @@ -0,0 +1,100 @@
    +/**
    + * 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.connectors.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSet;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.google.common.util.concurrent.FutureCallback;
    +import com.google.common.util.concurrent.Futures;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.streaming.runtime.operators.GenericExactlyOnceSink;
    +
    +/**
    + * Sink that emits its input elements into a Cassandra database. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public class CassandraExactlyOnceSink<IN extends Tuple> extends GenericExactlyOnceSink<IN> {
    +	private String host;
    +	private String createQuery;
    +	private String insertQuery;
    --- End diff --
    
    These fields could be made final, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the pull request:

    https://github.com/apache/flink/pull/1620#issuecomment-182959361
  
    Good work @zentol. I had some inline comments.
    
    My main concern is that upon recovery you'll write all elements of the previously written checkpoint again to the external store. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by spdrnl <gi...@git.apache.org>.
Github user spdrnl commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r54090368
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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 {
    --- End diff --
    
    Apart from Flink specifics it is possible to get Cassandra to behave idempotent with the right index/table structure. See and example below, the eid field is used as part of the index to identity unique records.
    
    ```
    cqlsh:mykeyspace> CREATE TABLE IF NOT EXISTS timeline (         
    eid uuid,         
    timesegment bigint,         
    from_url text,         
    to_url  text, 
    PRIMARY KEY ((timesegment) , eid));
    
    cqlsh:mykeyspace> INSERT INTO timeline (eid, timesegment, from_url, to_url) VALUES(50554d6e-29bb-11e5-b345-feff819cdc9f, 1, 'a', 'b') ;
    cqlsh:mykeyspace> SELECT * FROM timeline;
    
     timesegment | eid                                  | from_url | to_url
    -------------+--------------------------------------+----------+--------
               1 | 50554d6e-29bb-11e5-b345-feff819cdc9f |        a |      b
    
    (1 rows)
    cqlsh:mykeyspace> INSERT INTO timeline (eid, timesegment, from_url, to_url) VALUES(50554d6e-29bb-11e5-b345-feff819cdc9f, 1, 'a', 'b') ;
    cqlsh:mykeyspace> SELECT * FROM timeline;
     timesegment | eid                                  | from_url | to_url
    -------------+--------------------------------------+----------+--------
               1 | 50554d6e-29bb-11e5-b345-feff819cdc9f |        a |      b
    
    (1 rows)
    cqlsh:mykeyspace> INSERT INTO timeline (eid, timesegment, from_url, to_url) VALUES(uuid(), 1, 'a', 'b') ;
    cqlsh:mykeyspace> SELECT * FROM timeline;
    
     timesegment | eid                                  | from_url | to_url
    -------------+--------------------------------------+----------+--------
               1 | 50554d6e-29bb-11e5-b345-feff819cdc9f |        a |      b
               1 | 35761e5e-06b3-4ce6-9038-c9014e498281 |        a |      b
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52622956
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    --- End diff --
    
    param description missing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52575628
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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);
    +		saveHandleInState(checkpointId);
    +
    +		synchronized (state.pendingHandles) {
    +			Set<Long> pastCheckpointIds = state.pendingHandles.keySet();
    +			Set<Long> checkpointsToRemove = new HashSet<>();
    +			for (Long pastCheckpointId : pastCheckpointIds) {
    +				if (pastCheckpointId <= checkpointId) {
    +					List<StateHandle<DataInputView>> handles = state.pendingHandles.get(pastCheckpointId);
    +					for (StateHandle<DataInputView> handle : handles) {
    +						DataInputView in = handle.getState(getUserCodeClassloader());
    +						while (in.readByte() == 1) {
    +							IN value = serializer.deserialize(in);
    +							sendValue(value);
    +						}
    +						checkpointsToRemove.add(pastCheckpointId);
    +					}
    +				}
    +			}
    +			for (Long toRemove : checkpointsToRemove) {
    +				state.pendingHandles.remove(toRemove);
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Write the given element into the backend.
    +	 * @param value value to be written
    +	 * @throws Exception
    +     */
    +	protected abstract void sendValue(IN value) throws Exception;
    --- End diff --
    
    for more flexibility we should probably pass an Iterable<IN> here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52625876
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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);
    +		saveHandleInState(checkpointId);
    +
    +		synchronized (state.pendingHandles) {
    +			Set<Long> pastCheckpointIds = state.pendingHandles.keySet();
    +			Set<Long> checkpointsToRemove = new HashSet<>();
    +			for (Long pastCheckpointId : pastCheckpointIds) {
    +				if (pastCheckpointId <= checkpointId) {
    +					List<StateHandle<DataInputView>> handles = state.pendingHandles.get(pastCheckpointId);
    +					for (StateHandle<DataInputView> handle : handles) {
    +						DataInputView in = handle.getState(getUserCodeClassloader());
    +						while (in.readByte() == 1) {
    +							IN value = serializer.deserialize(in);
    +							sendValue(value);
    +						}
    +						checkpointsToRemove.add(pastCheckpointId);
    +					}
    +				}
    +			}
    +			for (Long toRemove : checkpointsToRemove) {
    +				state.pendingHandles.remove(toRemove);
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Write the given element into the backend.
    +	 * @param value value to be written
    +	 * @throws Exception
    +     */
    +	protected abstract void sendValue(IN value) throws Exception;
    +
    +	@Override
    +	public void processElement(StreamRecord<IN> element) throws Exception {
    +		IN value = element.getValue();
    +		if (serializer == null) {
    +			typeInfo = TypeExtractor.getForObject(value);
    +			serializer = typeInfo.createSerializer(new ExecutionConfig());
    --- End diff --
    
    This won't work here. To always create the right `TypeSerializer` you have to initialize it with the proper `ExecutionConfig`. E.g. Kryo requires that the types are registered in the right order. The order is specified in the `ExecutionConfig`. I think you should provide the proper serializer when you create the sink. Then you can also kick out the unnecessary `TypeInformation` which shouldn't be sent to the cluster in the first place.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52626379
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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);
    +		saveHandleInState(checkpointId);
    +
    +		synchronized (state.pendingHandles) {
    +			Set<Long> pastCheckpointIds = state.pendingHandles.keySet();
    +			Set<Long> checkpointsToRemove = new HashSet<>();
    +			for (Long pastCheckpointId : pastCheckpointIds) {
    +				if (pastCheckpointId <= checkpointId) {
    +					List<StateHandle<DataInputView>> handles = state.pendingHandles.get(pastCheckpointId);
    +					for (StateHandle<DataInputView> handle : handles) {
    +						DataInputView in = handle.getState(getUserCodeClassloader());
    +						while (in.readByte() == 1) {
    +							IN value = serializer.deserialize(in);
    +							sendValue(value);
    +						}
    +						checkpointsToRemove.add(pastCheckpointId);
    +					}
    +				}
    +			}
    +			for (Long toRemove : checkpointsToRemove) {
    +				state.pendingHandles.remove(toRemove);
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Write the given element into the backend.
    +	 * @param value value to be written
    +	 * @throws Exception
    +     */
    +	protected abstract void sendValue(IN value) throws Exception;
    +
    +	@Override
    +	public void processElement(StreamRecord<IN> element) throws Exception {
    +		IN value = element.getValue();
    +		if (serializer == null) {
    +			typeInfo = TypeExtractor.getForObject(value);
    +			serializer = typeInfo.createSerializer(new ExecutionConfig());
    +		}
    +		//generate initial operator state
    +		if (out == null) {
    +			out = getStateBackend().createCheckpointStateOutputView(0, 0);
    +		}
    +		out.writeByte(1);
    +		serializer.serialize(value, out);
    +	}
    +
    +	@Override
    +	public void processWatermark(Watermark mark) throws Exception {
    +		//don't do anything, since no are a sink
    +	}
    +
    +	/**
    +	 * This state is used to keep a list of all StateHandles (essentially references to past OperatorStates) that were
    +	 * used since the last completed checkpoint.
    +	 **/
    +	public class ExactlyOnceState implements StateHandle<Serializable> {
    +		protected HashMap<Long, ArrayList<StateHandle<DataInputView>>> pendingHandles;
    +
    +		public ExactlyOnceState() {
    +			pendingHandles = new HashMap<>();
    +		}
    +
    +		@Override
    +		public HashMap<Long, ArrayList<StateHandle<DataInputView>>> getState(ClassLoader userCodeClassLoader) throws Exception {
    +			return pendingHandles;
    +		}
    +
    +		@Override
    +		public void discardState() throws Exception {
    +			pendingHandles = new HashMap<>();
    --- End diff --
    
    What about the `StateHandles` in the `pendingHandles` map? Shouldn't we call discard on them as well?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by zentol <gi...@git.apache.org>.
Github user zentol closed the pull request at:

    https://github.com/apache/flink/pull/1620


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52626959
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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);
    +		saveHandleInState(checkpointId);
    +
    +		synchronized (state.pendingHandles) {
    +			Set<Long> pastCheckpointIds = state.pendingHandles.keySet();
    +			Set<Long> checkpointsToRemove = new HashSet<>();
    +			for (Long pastCheckpointId : pastCheckpointIds) {
    +				if (pastCheckpointId <= checkpointId) {
    +					List<StateHandle<DataInputView>> handles = state.pendingHandles.get(pastCheckpointId);
    +					for (StateHandle<DataInputView> handle : handles) {
    +						DataInputView in = handle.getState(getUserCodeClassloader());
    +						while (in.readByte() == 1) {
    +							IN value = serializer.deserialize(in);
    +							sendValue(value);
    +						}
    +						checkpointsToRemove.add(pastCheckpointId);
    +					}
    +				}
    +			}
    +			for (Long toRemove : checkpointsToRemove) {
    +				state.pendingHandles.remove(toRemove);
    --- End diff --
    
    Don't we also have to discard the state once we have written it to the external system?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52623292
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    --- End diff --
    
    Can't the input type be a generic type?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52624579
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    +
    +	private ExactlyOnceState state = new ExactlyOnceState();
    +
    +	/**
    +	 * Saves a handle in the state.
    +	 * @param checkpointId
    +	 * @throws IOException
    +	 */
    +	private void saveHandleInState(final long checkpointId) throws IOException {
    +		//only add handle if a new OperatorState was created since the last snapshot/notify
    +		if (out != null) {
    +			out.writeByte(0); //EOF-byte
    +			StateHandle<DataInputView> handle = out.closeAndGetHandle();
    +			if (state.pendingHandles.containsKey(checkpointId)) {
    +				state.pendingHandles.get(checkpointId).add(handle);
    +			} else {
    +				ArrayList<StateHandle<DataInputView>> list = new ArrayList<>();
    +				list.add(handle);
    +				state.pendingHandles.put(checkpointId, list);
    +			}
    +			out = null;
    +		}
    +	}
    +
    +	@Override
    +	public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
    +		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
    +		saveHandleInState(checkpointId);
    +		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);
    +		saveHandleInState(checkpointId);
    +
    +		synchronized (state.pendingHandles) {
    +			Set<Long> pastCheckpointIds = state.pendingHandles.keySet();
    +			Set<Long> checkpointsToRemove = new HashSet<>();
    +			for (Long pastCheckpointId : pastCheckpointIds) {
    +				if (pastCheckpointId <= checkpointId) {
    +					List<StateHandle<DataInputView>> handles = state.pendingHandles.get(pastCheckpointId);
    +					for (StateHandle<DataInputView> handle : handles) {
    +						DataInputView in = handle.getState(getUserCodeClassloader());
    +						while (in.readByte() == 1) {
    +							IN value = serializer.deserialize(in);
    +							sendValue(value);
    +						}
    +						checkpointsToRemove.add(pastCheckpointId);
    --- End diff --
    
    This can moved out of the loop, I guess.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52625891
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    --- End diff --
    
    I'm not sure if it's really needed at all actually, will give it some thought.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3332] Cassandra connector

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1620#discussion_r52623648
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericExactlyOnceSink.java ---
    @@ -0,0 +1,173 @@
    +/**
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +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 java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. This sink is integrated with the checkpointing
    + * mechanism to provide exactly once semantics.
    + *
    + * Incoming records are stored within a {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed. Should a job fail while the data is being committed, no exactly once guarantee can be made.
    + * @param <IN>
    + */
    +public abstract class GenericExactlyOnceSink<IN extends Tuple> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +	private AbstractStateBackend.CheckpointStateOutputView out;
    +	private TypeSerializer<IN> serializer;
    +	protected TypeInformation<IN> typeInfo;
    --- End diff --
    
    The `TypeInformation` shouldn't actually be sent to the cluster. Why do you need it here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---