Attila Bukor has posted comments on this change. ( 
http://gerrit.cloudera.org:8080/14329 )

Change subject: [java] KUDU-2791: process communicates via protobuf-based 
protocol
......................................................................


Patch Set 4:

(5 comments)

http://gerrit.cloudera.org:8080/#/c/14329/4/java/kudu-subprocess/src/main/java/org/apache/kudu/subprocess/SubprocessConfiguration.java
File 
java/kudu-subprocess/src/main/java/org/apache/kudu/subprocess/SubprocessConfiguration.java:

http://gerrit.cloudera.org:8080/#/c/14329/4/java/kudu-subprocess/src/main/java/org/apache/kudu/subprocess/SubprocessConfiguration.java@95
PS4, Line 95:     return conf.getInt(MAX_MESSAGE_BYTES, 
MAX_MESSAGE_BYTES_DEFAULT);
I think this should be long just in case. In the C++ side we use uint but all 
integers in Java are signed, so if we specify a max message size that is 2^31-1 
< max_message_size < 2^32 it would be perfectly valid in the C++ side but would 
cause an overflow and a negative number here. Alternatively, we can make the 
C++ side signed.


http://gerrit.cloudera.org:8080/#/c/14329/4/java/kudu-subprocess/src/main/java/org/apache/kudu/subprocess/SubprocessMain.java
File 
java/kudu-subprocess/src/main/java/org/apache/kudu/subprocess/SubprocessMain.java:

http://gerrit.cloudera.org:8080/#/c/14329/4/java/kudu-subprocess/src/main/java/org/apache/kudu/subprocess/SubprocessMain.java@63
PS4, Line 63:     try {
maybe we can use try with resources here and then join the threads at the end 
of the try block so that they won't be closed immediately. In this case we 
don't need to handle closing the streams in the consumer and producer.


http://gerrit.cloudera.org:8080/#/c/14329/4/java/kudu-subprocess/src/main/java/org/apache/kudu/subprocess/SubprocessMain.java@80
PS4, Line 80:       MessageConsumer consumer = new 
MessageConsumer(blockingQueue, messageProcessor,
there should be a separate consumer instance for each thread.


http://gerrit.cloudera.org:8080/#/c/14329/4/java/kudu-subprocess/src/main/java/org/apache/kudu/subprocess/SubprocessMain.java@88
PS4, Line 88:       producerService.shutdown();
don't think these are needed, shutdown only means the executor service won't 
accept new tasks. To actually interrupt them we would need to run shutdownNow() 
as these are all infinite loops that won't normally terminate. The finally 
block will be called right after the futures are submitted, so we shouldn't do 
this here, but in a shutdown hook instead so that if the subprocess receives a 
signal we can gracefully shut down.


http://gerrit.cloudera.org:8080/#/c/14329/4/java/kudu-subprocess/src/test/java/org/apache/kudu/subprocess/TestMessageProcessor.java
File 
java/kudu-subprocess/src/test/java/org/apache/kudu/subprocess/TestMessageProcessor.java:

http://gerrit.cloudera.org:8080/#/c/14329/4/java/kudu-subprocess/src/test/java/org/apache/kudu/subprocess/TestMessageProcessor.java@78
PS4, Line 78:   public static class NonParameterizedTest {
this should be in its own class, but as Andrew said, we probably don't need 
parameterized anymore the two classes can be merged.



--
To view, visit http://gerrit.cloudera.org:8080/14329
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Iaf9ad24dbc9acc681284b6433836271b5b4c7982
Gerrit-Change-Number: 14329
Gerrit-PatchSet: 4
Gerrit-Owner: Hao Hao <[email protected]>
Gerrit-Reviewer: Adar Dembo <[email protected]>
Gerrit-Reviewer: Andrew Wong <[email protected]>
Gerrit-Reviewer: Attila Bukor <[email protected]>
Gerrit-Reviewer: Grant Henke <[email protected]>
Gerrit-Reviewer: Hao Hao <[email protected]>
Gerrit-Reviewer: Kudu Jenkins (120)
Gerrit-Comment-Date: Wed, 20 Nov 2019 15:36:35 +0000
Gerrit-HasComments: Yes

Reply via email to