You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@ratis.apache.org by GitBox <gi...@apache.org> on 2020/10/27 20:40:14 UTC

[GitHub] [incubator-ratis] amaliujia opened a new pull request #234: RATIS-1108. Add a RaftConfigKey to control whether datastream server is enabled

amaliujia opened a new pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234


   ## What changes were proposed in this pull request?
   
   Add a RaftConfigKey to control whether datastream server is enabled. It is because running a DataStream server in RaftServer has a cost thus it might be better to allow enable/disable it depends on needs. 
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/RATIS-1108
   
   ## How was this patch tested?
   
   UT
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513900913



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.ratis.datastream;
+
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.conf.RaftProperties;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class TestDataStreamDisabled extends TestDataStreamBase {
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    properties = new RaftProperties();
+    RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.DISABLED);
+  }
+
+  @Test
+  public void testDataStreamSingleServer() throws Exception {
+    exception.expect(UnsupportedOperationException.class);
+    exception.expectMessage("org.apache.ratis.server.impl.DisabledDataStreamFactory$1 does not support streamAsync");
+    runTestDataStream(1, 1_000_000, 100);

Review comment:
       Please make the test simpler.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#issuecomment-718895935


   Np. Thanks @runzhiwang  and @szetszwo  for your review!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513954074



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.ratis.datastream;
+
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.conf.RaftProperties;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class TestDataStreamDisabled extends TestDataStreamBase {
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    properties = new RaftProperties();
+    RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.DISABLED);
+  }
+
+  @Test
+  public void testDataStreamSingleServer() throws Exception {
+    exception.expect(UnsupportedOperationException.class);
+    exception.expectMessage("org.apache.ratis.server.impl.DisabledDataStreamFactory$1 does not support streamAsync");
+    runTestDataStream(1, 1_000_000, 100);

Review comment:
       yeah, I mentioned this in a previous comment:
    
   > If this is not enough, another approach is to expose `setupServer` and `setupClient` and test cases call these one by one to identify which step stops.
   
   I think you are asking to use the approach I mentioned in the last. Basically manually create a server, then create a client, then make calls.
   
    ok let me change to this more straightforward way then.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513964506



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.ratis.datastream;
+
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.conf.RaftProperties;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class TestDataStreamDisabled extends TestDataStreamBase {
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    properties = new RaftProperties();
+    RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.DISABLED);
+  }
+
+  @Test
+  public void testDataStreamSingleServer() throws Exception {
+    exception.expect(UnsupportedOperationException.class);
+    exception.expectMessage("org.apache.ratis.server.impl.DisabledDataStreamFactory$1 does not support streamAsync");
+    runTestDataStream(1, 1_000_000, 100);

Review comment:
       Have changed this test to a simpler version.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r514427918



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/DisabledDataStreamFactory.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.client.DataStreamClientFactory;
+import org.apache.ratis.client.DataStreamClientRpc;
+import org.apache.ratis.conf.Parameters;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.datastream.SupportedDataStreamType;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.DataStreamServerFactory;
+import org.apache.ratis.server.DataStreamServerRpc;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/** A stream factory that does nothing when data stream is disabled. */
+public class DisabledDataStreamFactory implements DataStreamServerFactory, DataStreamClientFactory {
+    public DisabledDataStreamFactory(Parameters parameters){}
+
+    @Override
+    public DataStreamClientRpc newDataStreamClientRpc(RaftPeer server, RaftProperties properties) {

Review comment:
       Oops... I must have forgot to push my last local commit that fixes indents before going to sleep. Sorry :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513875017



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/DisabledDataStreamFactory.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.client.DataStreamClientFactory;
+import org.apache.ratis.client.DataStreamClientRpc;
+import org.apache.ratis.conf.Parameters;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.datastream.SupportedDataStreamType;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.DataStreamServerFactory;
+import org.apache.ratis.server.DataStreamServerRpc;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/** A stream factory that does nothing when data stream is disabled. */
+public class DisabledDataStreamFactory implements DataStreamServerFactory, DataStreamClientFactory {

Review comment:
       Ah... I believe it is visual effect. My local files do not have the 1-space indents.
   
   And if you use cursor to select here, I think you will find there is no space even it looks like there are.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513900913



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.ratis.datastream;
+
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.conf.RaftProperties;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class TestDataStreamDisabled extends TestDataStreamBase {
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    properties = new RaftProperties();
+    RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.DISABLED);
+  }
+
+  @Test
+  public void testDataStreamSingleServer() throws Exception {
+    exception.expect(UnsupportedOperationException.class);
+    exception.expectMessage("org.apache.ratis.server.impl.DisabledDataStreamFactory$1 does not support streamAsync");
+    runTestDataStream(1, 1_000_000, 100);

Review comment:
       Please make the test simpler. At least, we do not need both `runTestDataStream(1, 1_000_000, 100);` and `runTestDataStream(1,1_000, 10_000);`. Besides, we do not need the whole  `private void runTestDataStream(int bufferSize, int bufferNum)` if only test the expected exception. It will confuse somebody. You can call the method which throw the expected exception directly.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513900913



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.ratis.datastream;
+
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.conf.RaftProperties;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class TestDataStreamDisabled extends TestDataStreamBase {
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    properties = new RaftProperties();
+    RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.DISABLED);
+  }
+
+  @Test
+  public void testDataStreamSingleServer() throws Exception {
+    exception.expect(UnsupportedOperationException.class);
+    exception.expectMessage("org.apache.ratis.server.impl.DisabledDataStreamFactory$1 does not support streamAsync");
+    runTestDataStream(1, 1_000_000, 100);

Review comment:
       Please make the test simpler. At least, we do not need both `runTestDataStream(1, 1_000_000, 100);` and `runTestDataStream(1,1_000, 10_000);`. Besides, we do not need the whole  `private void runTestDataStream(int bufferSize, int bufferNum)` if only test the expected exception.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#issuecomment-718189878


   @szetszwo  tried to refactor how does current data stream is tested and added a test for `DISABLED` type.
   
   Also rebased and quashed commits.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513900913



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.ratis.datastream;
+
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.conf.RaftProperties;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class TestDataStreamDisabled extends TestDataStreamBase {
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    properties = new RaftProperties();
+    RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.DISABLED);
+  }
+
+  @Test
+  public void testDataStreamSingleServer() throws Exception {
+    exception.expect(UnsupportedOperationException.class);
+    exception.expectMessage("org.apache.ratis.server.impl.DisabledDataStreamFactory$1 does not support streamAsync");
+    runTestDataStream(1, 1_000_000, 100);

Review comment:
       Please make the test simpler. At least, we do not need both `runTestDataStream(1, 1_000_000, 100);` and `runTestDataStream(1,1_000, 10_000);`. Besides, we do not need the whole  `private void runTestDataStream(int bufferSize, int bufferNum)` if only test the expected exception. It will confuse somebody.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513947188



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamBase.java
##########
@@ -206,15 +192,15 @@ void runTestDataStream(int bufferSize, int bufferNum) {
       final DataStreamReply reply = impl.getHeaderFuture().join();
       Assert.assertTrue(reply.isSuccess());
       Assert.assertEquals(0, reply.getBytesWritten());
-      Assert.assertEquals(reply.getType(), Type.STREAM_HEADER);
+      Assert.assertEquals(reply.getType(), DataStreamPacketHeaderProto.Type.STREAM_HEADER);

Review comment:
       Ah my IDE somehow made this change. I can revert it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r514050911



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/DisabledDataStreamFactory.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.client.DataStreamClientFactory;
+import org.apache.ratis.client.DataStreamClientRpc;
+import org.apache.ratis.conf.Parameters;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.datastream.SupportedDataStreamType;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.DataStreamServerFactory;
+import org.apache.ratis.server.DataStreamServerRpc;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/** A stream factory that does nothing when data stream is disabled. */
+public class DisabledDataStreamFactory implements DataStreamServerFactory, DataStreamClientFactory {
+    public DisabledDataStreamFactory(Parameters parameters){}
+
+    @Override
+    public DataStreamClientRpc newDataStreamClientRpc(RaftPeer server, RaftProperties properties) {

Review comment:
       I see what you mean now. Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#issuecomment-718389173


   LGTM


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] szetszwo commented on pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#issuecomment-718603453


   >  I pushed a commit to fixes indentation and remove IOException.
   
   Oops, I tried to post the above comment but accidentally added it to the commit log.  Sorry.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] szetszwo commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r514033664



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/DisabledDataStreamFactory.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.client.DataStreamClientFactory;
+import org.apache.ratis.client.DataStreamClientRpc;
+import org.apache.ratis.conf.Parameters;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.datastream.SupportedDataStreamType;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.DataStreamServerFactory;
+import org.apache.ratis.server.DataStreamServerRpc;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/** A stream factory that does nothing when data stream is disabled. */
+public class DisabledDataStreamFactory implements DataStreamServerFactory, DataStreamClientFactory {
+    public DisabledDataStreamFactory(Parameters parameters){}
+
+    @Override
+    public DataStreamClientRpc newDataStreamClientRpc(RaftPeer server, RaftProperties properties) {

Review comment:
       You are right that I was confused by the GitHub ui.  The comment "These few lines and some other lines below still have 4-space indentation."  is correct.  Please use 2-space indentation.

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/DisabledDataStreamFactory.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.client.DataStreamClientFactory;
+import org.apache.ratis.client.DataStreamClientRpc;
+import org.apache.ratis.conf.Parameters;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.datastream.SupportedDataStreamType;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.DataStreamServerFactory;
+import org.apache.ratis.server.DataStreamServerRpc;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/** A stream factory that does nothing when data stream is disabled. */
+public class DisabledDataStreamFactory implements DataStreamServerFactory, DataStreamClientFactory {

Review comment:
       You are right that I was confused by the GitHub ui.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED to indicate that datastream server is not enabled

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513184679



##########
File path: ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java
##########
@@ -61,6 +63,16 @@ static SupportedDataStreamType type(RaftProperties properties, Consumer<String>
     static void setType(RaftProperties properties, SupportedDataStreamType type) {
       set(properties::set, TYPE_KEY, type.name());
     }
+
+    String ENABLED_KEY = PREFIX + ".enabled";

Review comment:
       @szetszwo updated to use `SupportedDataStreamType.DISABLED`. Can you take a look?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513954074



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.ratis.datastream;
+
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.conf.RaftProperties;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class TestDataStreamDisabled extends TestDataStreamBase {
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    properties = new RaftProperties();
+    RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.DISABLED);
+  }
+
+  @Test
+  public void testDataStreamSingleServer() throws Exception {
+    exception.expect(UnsupportedOperationException.class);
+    exception.expectMessage("org.apache.ratis.server.impl.DisabledDataStreamFactory$1 does not support streamAsync");
+    runTestDataStream(1, 1_000_000, 100);

Review comment:
       yeah, I mentioned this in a previous comment:
    
   > If this is not enough, another approach is to expose `setupServer` and `setupClient` and test cases call these one by one to identify which step stops.
   
   I think you are asking to use the approach I mentioned in the last. Basically manually create a server, then create a client (which actually will hit the unsupported exception because the auto sending header operation by stream api).
   
    ok let me change to this more straightforward way then.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on pull request #234: RATIS-1108. Add a RaftConfigKey to control whether datastream server is enabled

Posted by GitBox <gi...@apache.org>.
amaliujia commented on pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#issuecomment-717569521


   R @szetszwo @runzhiwang 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add a RaftConfigKey to control whether datastream server is enabled

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513026049



##########
File path: ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java
##########
@@ -61,6 +63,16 @@ static SupportedDataStreamType type(RaftProperties properties, Consumer<String>
     static void setType(RaftProperties properties, SupportedDataStreamType type) {
       set(properties::set, TYPE_KEY, type.name());
     }
+
+    String ENABLED_KEY = PREFIX + ".enabled";
+    boolean ENABLED_DEFAULT = true;

Review comment:
       If adding this config key is ok, what should be the default value? Is by default enabling the stream server ok?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513964899



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.ratis.datastream;
+
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.conf.RaftProperties;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class TestDataStreamDisabled extends TestDataStreamBase {
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    properties = new RaftProperties();
+    RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.DISABLED);
+  }
+
+  @Test
+  public void testDataStreamSingleServer() throws Exception {
+    exception.expect(UnsupportedOperationException.class);
+    runTestDataStream(1, 1_000_000, 100);
+    runTestDataStream(1,1_000, 10_000);
+  }

Review comment:
       Actually have changed this test to the version that `setupServer` and `setupClient` then make a call.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513904413



##########
File path: ratis-common/src/main/java/org/apache/ratis/datastream/SupportedDataStreamType.java
##########
@@ -21,6 +21,7 @@
 import org.apache.ratis.util.ReflectionUtils;
 
 public enum SupportedDataStreamType implements DataStreamFactory {
+  DISABLED("org.apache.ratis.server.impl.DisabledDataStreamFactory"),
   NETTY("org.apache.ratis.netty.NettyDataStreamFactory");

Review comment:
       why not put DisabledDataStreamFactory and NettyDataStreamFactory in the same package ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] szetszwo merged pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
szetszwo merged pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] szetszwo commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513300082



##########
File path: ratis-common/src/main/java/org/apache/ratis/datastream/SupportedDataStreamType.java
##########
@@ -21,7 +21,8 @@
 import org.apache.ratis.util.ReflectionUtils;
 
 public enum SupportedDataStreamType implements DataStreamFactory {
-  NETTY("org.apache.ratis.netty.NettyDataStreamFactory");
+  NETTY("org.apache.ratis.netty.NettyDataStreamFactory"),
+  DISABLED("org.apache.ratis.server.impl.DisabledDataStreamFactory");

Review comment:
       Let's move DISABLED to be the first item.

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/DisabledDataStreamFactory.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.client.DataStreamClientFactory;
+import org.apache.ratis.client.DataStreamClientRpc;
+import org.apache.ratis.conf.Parameters;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.datastream.SupportedDataStreamType;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.DataStreamServerFactory;
+import org.apache.ratis.server.DataStreamServerRpc;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/** A stream factory that does nothing when data stream is disabled. */
+public class DisabledDataStreamFactory implements DataStreamServerFactory, DataStreamClientFactory {

Review comment:
       Please use 2-space indentation for the entire class.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] szetszwo commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513851483



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.ratis.datastream;
+
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.conf.RaftProperties;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class TestDataStreamDisabled extends TestDataStreamBase {
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    properties = new RaftProperties();
+    RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.DISABLED);
+  }
+
+  @Test
+  public void testDataStreamSingleServer() throws Exception {
+    exception.expect(UnsupportedOperationException.class);
+    runTestDataStream(1, 1_000_000, 100);
+    runTestDataStream(1,1_000, 10_000);
+  }

Review comment:
       It is not clear when we get UnsupportedOperationException.  In the test, we should make sure that server and client can start and work normally.  When the client uses the StreamApi, it will get UnsupportedOperationException.

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/DisabledDataStreamFactory.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.client.DataStreamClientFactory;
+import org.apache.ratis.client.DataStreamClientRpc;
+import org.apache.ratis.conf.Parameters;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.datastream.SupportedDataStreamType;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.DataStreamServerFactory;
+import org.apache.ratis.server.DataStreamServerRpc;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/** A stream factory that does nothing when data stream is disabled. */
+public class DisabledDataStreamFactory implements DataStreamServerFactory, DataStreamClientFactory {
+    public DisabledDataStreamFactory(Parameters parameters){}
+
+    @Override
+    public DataStreamClientRpc newDataStreamClientRpc(RaftPeer server, RaftProperties properties) {

Review comment:
       These few lines and some other lines below still have 4-space indentation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] szetszwo commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513854368



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/DisabledDataStreamFactory.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.client.DataStreamClientFactory;
+import org.apache.ratis.client.DataStreamClientRpc;
+import org.apache.ratis.conf.Parameters;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.datastream.SupportedDataStreamType;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.DataStreamServerFactory;
+import org.apache.ratis.server.DataStreamServerRpc;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/** A stream factory that does nothing when data stream is disabled. */
+public class DisabledDataStreamFactory implements DataStreamServerFactory, DataStreamClientFactory {

Review comment:
       Lines 1 to 35 now are 1-space indented.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513964349



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamBase.java
##########
@@ -206,15 +192,15 @@ void runTestDataStream(int bufferSize, int bufferNum) {
       final DataStreamReply reply = impl.getHeaderFuture().join();
       Assert.assertTrue(reply.isSuccess());
       Assert.assertEquals(0, reply.getBytesWritten());
-      Assert.assertEquals(reply.getType(), Type.STREAM_HEADER);
+      Assert.assertEquals(reply.getType(), DataStreamPacketHeaderProto.Type.STREAM_HEADER);

Review comment:
       Reverted.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513229302



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/DataStreamServerImpl.java
##########
@@ -35,31 +37,38 @@
 public class DataStreamServerImpl implements DataStreamServer {
   public static final Logger LOG = LoggerFactory.getLogger(DataStreamServerImpl.class);
 
-  private final DataStreamServerRpc serverRpc;
+  private Optional<DataStreamServerRpc> serverRpc;

Review comment:
       Thanks! 
   
   :) somehow I thought this approach complicates the code too much but I was wrong. This approach encapsulates code much well thus simplify the code.  




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#issuecomment-717753713


   @runzhiwang 
   Got it. Has shorten the title.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] szetszwo commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED to indicate that datastream server is not enabled

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513202703



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/DataStreamServerImpl.java
##########
@@ -35,31 +37,38 @@
 public class DataStreamServerImpl implements DataStreamServer {
   public static final Logger LOG = LoggerFactory.getLogger(DataStreamServerImpl.class);
 
-  private final DataStreamServerRpc serverRpc;
+  private Optional<DataStreamServerRpc> serverRpc;

Review comment:
       We do not need to change DataStreamServerImpl at all.  The idea is to implement a new RPC type for DISABLED, i.e. we should implement a DisabledDataStreamFactory.
   ```
   public class DisabledDataStreamFactory implements DataStreamServerFactory, DataStreamClientFactory {
     public DisabledDataStreamFactory(Parameters parameters){}
   
     @Override
     public SupportedDataStreamType getDataStreamType() {
       return SupportedDataStreamType.DISABLED;
     }
   
     @Override
     public DataStreamClientRpc newDataStreamClientRpc(RaftPeer server, RaftProperties properties) {
       return new DataStreamClientRpc() {
         @Override
         public void startClient() {}
   
         @Override
         public void closeClient() {}
       };
     }
   
     @Override
     public DataStreamServerRpc newDataStreamServerRpc(RaftPeer server, StateMachine stateMachine,
         RaftProperties properties) {
       return new DataStreamServerRpc() {
         @Override
         public void start() {}
   
         @Override
         public void close() {}
   
         @Override
         public void addRaftPeers(Collection<RaftPeer> peers) {}
       };
     }
   
     @Override
     public DataStreamServerRpc newDataStreamServerRpc(RaftServer server, StateMachine stateMachine,
         RaftProperties properties) {
       return new DataStreamServerRpc() {
         @Override
         public void start() {}
   
         @Override
         public void close() {}
   
         @Override
         public void addRaftPeers(Collection<RaftPeer> peers) {}
       };
     }
   }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513946132



##########
File path: ratis-common/src/main/java/org/apache/ratis/datastream/SupportedDataStreamType.java
##########
@@ -21,6 +21,7 @@
 import org.apache.ratis.util.ReflectionUtils;
 
 public enum SupportedDataStreamType implements DataStreamFactory {
+  DISABLED("org.apache.ratis.server.impl.DisabledDataStreamFactory"),
   NETTY("org.apache.ratis.netty.NettyDataStreamFactory");

Review comment:
       Because of dependencies. 
   
   `NettyDataStreamFactory` is in `ratis-netty`. A user who don't need stream server, thus choose to disable steam server, should not depend on `ratis-netty`. 
   
   Similarly, if in the future there is a `GrpcDataStreamFactory`, it likely will be in `ratis-grpc`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] szetszwo commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513852894



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/DisabledDataStreamFactory.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.client.DataStreamClientFactory;
+import org.apache.ratis.client.DataStreamClientRpc;
+import org.apache.ratis.conf.Parameters;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.datastream.SupportedDataStreamType;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.DataStreamServerFactory;
+import org.apache.ratis.server.DataStreamServerRpc;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/** A stream factory that does nothing when data stream is disabled. */
+public class DisabledDataStreamFactory implements DataStreamServerFactory, DataStreamClientFactory {
+    public DisabledDataStreamFactory(Parameters parameters){}
+
+    @Override
+    public DataStreamClientRpc newDataStreamClientRpc(RaftPeer server, RaftProperties properties) {

Review comment:
       Oops, this comment is incorrect.  Please ignore it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513899216



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamBase.java
##########
@@ -206,15 +192,15 @@ void runTestDataStream(int bufferSize, int bufferNum) {
       final DataStreamReply reply = impl.getHeaderFuture().join();
       Assert.assertTrue(reply.isSuccess());
       Assert.assertEquals(0, reply.getBytesWritten());
-      Assert.assertEquals(reply.getType(), Type.STREAM_HEADER);
+      Assert.assertEquals(reply.getType(), DataStreamPacketHeaderProto.Type.STREAM_HEADER);

Review comment:
       why need this change ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513954074



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.ratis.datastream;
+
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.conf.RaftProperties;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class TestDataStreamDisabled extends TestDataStreamBase {
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    properties = new RaftProperties();
+    RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.DISABLED);
+  }
+
+  @Test
+  public void testDataStreamSingleServer() throws Exception {
+    exception.expect(UnsupportedOperationException.class);
+    exception.expectMessage("org.apache.ratis.server.impl.DisabledDataStreamFactory$1 does not support streamAsync");
+    runTestDataStream(1, 1_000_000, 100);

Review comment:
       yeah, I mentioned this in a previous comment:
    
   > If this is not enough, another approach is to expose `setupServer` and `setupClient` and test cases call these one by one to identify which step stops.
   
   I think you are asking to use the approach I mentioned in the last. Basically manually create a server, then create a client (which actually will hit the unsupported exception because the auto sending header operation by stream api).
   
    




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED to indicate that datastream server is not enabled

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#issuecomment-717747738


   @amaliujia PR title is too long.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] szetszwo commented on a change in pull request #234: RATIS-1108. Add a RaftConfigKey to control whether datastream server is enabled

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513141802



##########
File path: ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java
##########
@@ -61,6 +63,16 @@ static SupportedDataStreamType type(RaftProperties properties, Consumer<String>
     static void setType(RaftProperties properties, SupportedDataStreamType type) {
       set(properties::set, TYPE_KEY, type.name());
     }
+
+    String ENABLED_KEY = PREFIX + ".enabled";

Review comment:
       Instead of add a new configuration, we may add a new type SupportedDataStreamType.DISABLED.  Then, we do not have to complicate the code. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513732613



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/DisabledDataStreamFactory.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.client.DataStreamClientFactory;
+import org.apache.ratis.client.DataStreamClientRpc;
+import org.apache.ratis.conf.Parameters;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.datastream.SupportedDataStreamType;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.DataStreamServerFactory;
+import org.apache.ratis.server.DataStreamServerRpc;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/** A stream factory that does nothing when data stream is disabled. */
+public class DisabledDataStreamFactory implements DataStreamServerFactory, DataStreamClientFactory {

Review comment:
       Done. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED to indicate that datastream server is not enabled

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513184679



##########
File path: ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java
##########
@@ -61,6 +63,16 @@ static SupportedDataStreamType type(RaftProperties properties, Consumer<String>
     static void setType(RaftProperties properties, SupportedDataStreamType type) {
       set(properties::set, TYPE_KEY, type.name());
     }
+
+    String ENABLED_KEY = PREFIX + ".enabled";

Review comment:
       @szetszwo updated to using `SupportedDataStreamType.DISABLED`. Can you take a look?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #234: RATIS-1108. Add SupportedDataStreamType.DISABLED

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #234:
URL: https://github.com/apache/incubator-ratis/pull/234#discussion_r513870575



##########
File path: ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.ratis.datastream;
+
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.conf.RaftProperties;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class TestDataStreamDisabled extends TestDataStreamBase {
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() {
+    properties = new RaftProperties();
+    RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.DISABLED);
+  }
+
+  @Test
+  public void testDataStreamSingleServer() throws Exception {
+    exception.expect(UnsupportedOperationException.class);
+    runTestDataStream(1, 1_000_000, 100);
+    runTestDataStream(1,1_000, 10_000);
+  }

Review comment:
       @szetszwo 
   
   I have push a one line of change to match the exception error message, which matches with the unsupported operation exception thrown from https://github.com/apache/incubator-ratis/blob/master/ratis-client/src/main/java/org/apache/ratis/client/DataStreamClientRpc.java#L34.
   
   The error message might have proved that server and client have started working and it fails when call `streamAsync`. Do you think whether this is enough?
   
   If this is not enough, another approach is to expose `setupServer` and `setupClient` and test cases call these one by one to identify which step stops. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org