[ 
https://issues.apache.org/jira/browse/CASSANDRA-15505?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17020531#comment-17020531
 ] 

Yifan Cai commented on CASSANDRA-15505:
---------------------------------------

Thanks [~ifesdjeen] for the patch. The message interceptor could be very useful 
to test Cassandra internals by intercepting the internode messages. 

My comments:

# Regarding message filtering semantics, should it allow having multiple 
filters that matches the same {{from}}, {{to}} and {{verb}} condition? In 
{{MessageFilters#process}}, the interceptors is allow to continue matching, 
meanwhile the filters are not by returning {{false}} immediately. So the 
semantics of {{interceptors}} and {{filters}} are not the same. For instance, 
one can have both {{interceptorA(1: from, 2: to, Mutation: verb, 
waitForCountdownLatch_1: runnable}} and {{interceptorA(1: from, 2: to, 
Mutation: verb, waitForCountdownLatch_2: runnable}}. The usage is confusing and 
the behavior is confusing and unpredictable. However, in the current 
implementation, such usage is allowed. Personally, I feel the filtering 
mechanism should only *allow 1 filter per matching condition*. 
# Nice to have a test that shows a filter can be disabled by calling 
{{Filter#off()}}
# Nice to have a test that shows having interceptors with blocking runnable 
does not block each other. Assuming, we do not want to clog the outing message 
sink.
# NIT: {{MessageFilters#DO_NOTHING}} can be {{final}}
# NIT: passing the reference of global filters when constructing a filter seems 
little strange. But it is required for supporting adding a filter to the global 
filters. Maybe making the constructor private, so a filter can only be 
constructed with the more informed builder. 



> Add message interceptors to in-jvm dtests
> -----------------------------------------
>
>                 Key: CASSANDRA-15505
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-15505
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Test/dtest
>            Reporter: Alex Petrov
>            Assignee: Alex Petrov
>            Priority: Normal
>
> Currently we only have means to filter messages in in-jvm tests. We need a 
> facility to intercept and modify the messages between nodes for testing 
> purposes.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to