Conversation
|
kafka-trunk-git-pr #122 SUCCESS |
|
@lazyval Apologies for the commits history, I was fighting with git merge history back then from two branches and hence the commits was not well organized. I will create another PR with squashed commits after addressing the collected comments from this PR. |
There was a problem hiding this comment.
Can you add some documentation for some of these interfaces?
|
This is an excellent proposal, and after a quick pass this PR looks good. More detailed comments and questions to follow. |
There was a problem hiding this comment.
The run() and close() methods should not be synchronized. Because they are, then once run() is called it will block any other synchronized method, including close(), and because run() only completes when close() is called, run() will never complete. In other words, the thread will never stop.
You should be able to simply remove the synchronized keywords with the current code and maintain thread safety of the running volatile boolean field: the only method that reads that field is the private stillRunning() (called via private runLoop() which is called via public run()), while the only method that writes to the field is close().
|
Is there a way to make the sources and processors created by the topology aware of or dependent upon the supplied configuration? For example, let's say my job's configuration has a parameter that, if set one way, will result in a slight variation of the topology (e.g., an extra filtering processor between the source and my primary processor). Is that possible? |
There was a problem hiding this comment.
The ProducerRecord class has a constructor that takes the partition number, yet that doesn't appear to be exposed in these two send(...) methods. Am I missing how to specify the partitioning logic for each of the sent messages?
UPDATE: Okay, it's pretty obvious you can set the partitioner.class property in the producer's configuration to the name of the Partitioner implementation class. Doing this makes the KafkaProducer pass the message key to the Partitioner to determine the partition number. Is this a best practice, or is it still logical for our Processor implementation to determine the partition, perhaps based upon something other than they key. If so, then it'd be great to have additional send(...) methods that take the partition number.
There was a problem hiding this comment.
I think we can wrap the producer / consumer configs in the streaming / processor congis as you mentioned.
|
kafka-trunk-git-pr #189 FAILURE |
There was a problem hiding this comment.
The createSensor(...) method called on lines 68-75 uses the this.metrics field, and because this.metrics is not set until line 76 the result is a NullPointerException. To fix, simply move the this.metrics = context.metrics(); line before the first call to createSensor(...).
There was a problem hiding this comment.
Added a PR with the simple correction: confluentinc#38
|
@guozhangwang, I'm willing to help resolve issues, add test cases, make suggestions via patches, and even add JavaDoc. But I suspect that'd be easier after after you squash commit history. Please let me know what you think. |
There was a problem hiding this comment.
This would be easier to implement if the parameter to this method were an Iterable<Entry<K,V>> than a List<Entry<K,V>>. For example, the current RocksDBKeyValueStore uses byte[] for the keys and values, and it's pretty easy to wrap that with a parameterized class that uses provided Serializer and Deserializerinstances for the keys and values -- except that theputAllmethod cannot be easily implemented as a delegate if it takes aList`. (In essence, the list has to be fully-copied before the delegation can be made.
I'd be happy to provide a patch with this fix.
|
kafka-trunk-git-pr #229 FAILURE |
|
kafka-trunk-git-pr #232 FAILURE |
|
kafka-trunk-git-pr #239 FAILURE |
|
kafka-trunk-git-pr #241 FAILURE |
|
kafka-trunk-git-pr #245 FAILURE |
|
kafka-trunk-git-pr #249 FAILURE |
|
kafka-trunk-git-pr #251 FAILURE |
|
kafka-trunk-git-pr #252 FAILURE |
…river-updated Added a new ProcessorTopologyTestDriver class that makes it easier to unit test a ProcessorTopology
|
kafka-trunk-git-pr #508 FAILURE |
Fix delete in in-memory store, and let delete return old value
|
kafka-trunk-git-pr #509 FAILURE |
|
@rhauch I agree about ser-de, maybe we can allow users to pass in their (de)serializers upon construction of the key-value store. About |
|
@guozhangwang, see my draft PR for the serdes changes; should be up-to-date with the |
|
@guozhangwang, I created a PR for the InMemoryLRUCacheStore and a mini test framework for key-value stores, with new unit tests for all current |
flush records and state changelog before local state
|
kafka-trunk-git-pr #555 FAILURE |
Merge latest commits from apache again
|
kafka-trunk-git-pr #562 SUCCESS |
|
Update the PR body before merging, and the original message is here: Some open questions collected so far on the first patch. Thanks @gwenshap @jkreps @junrao .
So the implementation of KStream.filter look instead like this: The advantage is that the user code can now get rid of the whole Topology class with the builder. I think the order of execution for that API is quite unintuitive.
|
This doesn't matter in the present state, but there's an upcoming PR that make Controller use the Control plane, and duplicating merging default and override topic config there is cumbersome. With this change, the resolution happens earlier, in `AppendInterceptor`.
Remove unclean leader election warning

This work has been contributed by Jesse Anderson, Randall Hauch, Yasuhiro Matsuda and Guozhang Wang. The detailed design can be found in https://cwiki.apache.org/confluence/display/KAFKA/KIP-28+-+Add+a+processor+client.