[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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); +
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public class CassandraExactlyOnceSink extends GenericExactlyOnceSink { --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public class CassandraExactlyOnceSink extends GenericExactlyOnceSink { --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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); +
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public class CassandraExactlyOnceSink extends GenericExactlyOnceSink { + 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public class CassandraExactlyOnceSink extends GenericExactlyOnceSink { + 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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); +
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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); +
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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); +
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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); +
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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); +
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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); +
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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); +
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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); +
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request: [FLINK-3332] Cassandra connector
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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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 + */ +public abstract class GenericExactlyOnceSink extends AbstractStreamOperator implements OneInputStreamOperator{ + private AbstractStateBackend.CheckpointStateOutputView out; + private TypeSerializer serializer; + protected TypeInformation 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 handle = out.closeAndGetHandle(); + if (state.pendingHandles.containsKey(checkpointId)) { + state.pendingHandles.get(checkpointId).add(handle); + } else { + ArrayList 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
[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: zentolDate: 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---