-
Notifications
You must be signed in to change notification settings - Fork 13.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-3311/FLINK-3332] Add Cassandra connector #1771
Conversation
Great work 👍 |
<dependency> | ||
<groupId>com.google.guava</groupId> | ||
<artifactId>guava</artifactId> | ||
<version>16.0.1</version> |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please use the guava.version
variable here to use the same guava version across Flink
Hi, Hope it helps. |
@alkagin Thank you for reporting this, will look into it. I think i may already know how to fix it... |
} | ||
|
||
public enum ConsistencyLevel { | ||
At_LEAST_ONCE, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
typo: AT_LEAST_ONCE
* | ||
* @param <IN> Type of the elements emitted by this sink | ||
*/ | ||
public abstract class GenericAtLeastOnceSink<IN> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
GenericExactlyOnceSink
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I've struggled a bit with the name for this class, but came to the conclusion that since class itself does not provide exactly-once guarantees for all use-cases. As such the GenericAtLeastOnceSink name is intended.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ok, I misunderstood CassandraIdempotentExactlyOnceSink's inheritance.
Just pushed the following changes:
|
try { | ||
session.close(); | ||
} catch (Exception e) { | ||
LOG.info("Inputformat couldn't be closed - " + e.getMessage()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would make this a WARN and forward the cause to the log entry
* @return this builder | ||
*/ | ||
public CassandraSinkBuilder<IN> setClusterBuilder(ClusterBuilder builder) { | ||
if (builder != null) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this.builder
instead of builder
, correct?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes. i distinctly recall having fixed this...
Hi guys :) any update on this? |
I'll test the PR again |
I just tried the PR, but the recovery after a failure doesn't seem to work:
|
* 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> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Serial version uid missing
I finally was able to fix the restart issue. there were 2 massive bugs in the CassandraCommitter:
in addition i have made the following changes:
It's rather obvious that more tests are required. |
-revert final flag removal on ResultPartitionWriter -fixed type in CassandraCommitter -cached cpID is properly reset on close() -added test for CassandraCommitter -modified StreamTaskTestHarness to properly notice exceptions in the task thread -modified writeAheadSink test to close/open operator upon "failure"
I've reviewed the connector again.
Also, I would like to test the exactly once behavior on a cluster more thoroughly. Currently, I've only tested whether the connector is properly failing and restoring, but I didn't test if the written data is actually correct. However, since the code seems to be working under normal operation, I would suggest to merge the connector now, and then file follow up JIRAs for the remaining issues. Some log:
|
I tried out this branch and it works like it should in the scenario I set up: Overall, I agree with @rmetzger that it should be considered to merge this to get more users to test it and report issues. |
Thank you for testing the cassandra connector. I'll merge the pull request now. |
The merging will probably be a bit delayed because there are some memory issues:
I will take care of them! |
Looking at the code of Could you confirm that this isn't an issue with the implementation? |
You are correct that every CassandraSink instance opens a separate connection to the cluster. I don't see how we could avoid this without using Singletons, which should be avoided. |
Could somebody point to a documentation how we could manage state (pool of connections) when Sink works inside single JVM. |
This PR is a combination of #1640 and #1660, essentially providing a mid-merge view of both PR's. As they reside in the same module and expose similar functionalities, several things were changed in both commits to make the code more similar.
Such changes include formatting, field ordering, code in-lining, package declarations, file names, test unification, and the addition of a central method to create sinks (CassandraSink.addSink(DataStream input)) to reduce complexity for the user resulting from different sink implementations.