You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@geode.apache.org by pivotal-amurmann <gi...@git.apache.org> on 2017/08/16 23:16:13 UTC

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

GitHub user pivotal-amurmann opened a pull request:

    https://github.com/apache/geode/pull/716

    GEODE-3406: Locator accepts Protobuf requests

    Also addresses GEODE-3400, GEODE-3399
    This allows the locator to respond to Protobuf requests. Currently it
    will only be able to respond to getAvailableServers.
    
    To enable this we are introducing a new value of "0" that will be sent
    in place of the Gossip version. After it we expect the same magic byte
    ("110") as in AcceptorImpl.
    
    This also is gated by the `geode.feature-protobuf-protocol` system
    property.
    
    The getAvailableServers request handler now uses the locator directly,
    since we are on the locator.
    
    Signed-off-by: Brian Rowe <br...@pivotal.io>
    
    Thank you for submitting a contribution to Apache Geode.
    
    In order to streamline the review of the contribution we ask you
    to ensure the following steps have been taken:
    
    ### For all changes:
    - [✔️ ] Is there a JIRA ticket associated with this PR? Is it referenced in the commit message?
    GEODE-3400, GEODE-3399, GEODE-3406
    
    - [✔️ ] Has your PR been rebased against the latest commit within the target branch (typically `develop`)?
    
    - [✔️] Is your initial contribution a single, squashed commit?
    
    - [✔️] Does `gradlew build` run cleanly?
    
    - [✔️] Have you written or updated unit tests to verify your changes?
    
    - [x] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
    N/A
    
    ### Note:
    Please ensure that once the PR is submitted, you check travis-ci for build issues and
    submit an update to your PR as soon as possible. If you need help, please send an
    email to dev@geode.apache.org.
    
    @kohlmu-pivotal @bschuchardt @galen-pivotal @WireBaron @hiteshk25 

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/pivotal-amurmann/geode feature/GEODE-3400/locator_support_protobuf

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/geode/pull/716.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #716
    
----
commit a1d10ffdc6eda365eafa11d055cbce5096abcec2
Author: Alexander Murmann <am...@pivotal.io>
Date:   2017-08-14T22:08:14Z

    GEODE-3406: Locator accepts Protobuf requests
    
    Also addresses GEODE-3400, GEODE-3399
    This allows the locator to respond to Protobuf requests. Currently it
    will only be able to respond to getAvailableServers.
    
    To enable this we are introducing a new value of "0" that will be sent
    in place of the Gossip version. After it we expect the same magic byte
    ("110") as in AcceptorImpl.
    
    This also is gated by the `geode.feature-protobuf-protocol` system
    property.
    
    The getAvailableServers request handler now uses the locator directly,
    since we are on the locator.
    
    Signed-off-by: Brian Rowe <br...@pivotal.io>

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by WireBaron <gi...@git.apache.org>.
Github user WireBaron commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133597359
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtoclMessageHandlerLoader.java ---
    @@ -0,0 +1,64 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import java.io.IOException;
    +import java.net.Socket;
    +import java.util.Iterator;
    +import java.util.ServiceLoader;
    +
    +import org.apache.geode.internal.cache.InternalCache;
    +import org.apache.geode.internal.cache.tier.Acceptor;
    +import org.apache.geode.internal.cache.tier.CachedRegionHelper;
    +import org.apache.geode.internal.security.SecurityService;
    +
    +/**
    + * Creates instances of ServerConnection based on the connection mode provided.
    + */
    +public class ClientProtoclMessageHandlerLoader {
    --- End diff --
    
    Typo: s/Protocl/Protocol
    
    Actually, this looks like a second copy of this code, so should just be deleted in favor of the correctly spelled version.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133778501
  
    --- Diff: geode-core/src/test/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java ---
    @@ -392,7 +388,7 @@ public void testLocatorUpdateIntervalZero() throws Exception {
     
       private void startFakeLocator() throws UnknownHostException, IOException, InterruptedException {
         server = new TcpServer(port, InetAddress.getLocalHost(), null, null, handler, new FakeHelper(),
    -        Thread.currentThread().getThreadGroup(), "Tcp Server");
    +        Thread.currentThread().getThreadGroup(), "Tcp Server", null);
    --- End diff --
    
    Why is it valid to pass in a 'null' InternalLocator?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by pivotal-amurmann <gi...@git.apache.org>.
Github user pivotal-amurmann commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r135089666
  
    --- Diff: geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/operations/GetAvailableServersOperationHandler.java ---
    @@ -50,51 +37,23 @@
       @Override
       public Result<ServerAPI.GetAvailableServersResponse> process(
           SerializationService serializationService, ServerAPI.GetAvailableServersRequest request,
    -      Cache cache) {
    -
    -    InternalDistributedSystem distributedSystem =
    -        (InternalDistributedSystem) cache.getDistributedSystem();
    -    Properties properties = distributedSystem.getProperties();
    -    String locatorsString = properties.getProperty(ConfigurationProperties.LOCATORS);
    +      MessageExecutionContext executionContext) throws InvalidExecutionContextException {
     
    -    HashSet<DistributionLocatorId> locators = new HashSet();
    -    StringTokenizer stringTokenizer = new StringTokenizer(locatorsString, ",");
    -    while (stringTokenizer.hasMoreTokens()) {
    -      String locator = stringTokenizer.nextToken();
    -      if (StringUtils.isNotEmpty(locator)) {
    -        locators.add(new DistributionLocatorId(locator));
    -      }
    +    InternalLocator locator = executionContext.getLocator();
    +    ArrayList serversFromSnapshot =
    --- End diff --
    
    I very much agree with Galen that this should be refactored. This is a big demeter violation which is pointing at some worse code in ServerLocator which currently know how to answered requests and how to get the information to answer them. If that was split out into one class that can talk whatever protocol it's talking and another class that can get information from the locator this could get cleaned up quite a bit and also make unit tests much easier. Since we are talking about switching to Netty that might be wasted effort at this time. On the other hand we should extract the business logic from the transport logic anyways when moving to Netty and doing this beforehand might make that move easier.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by pivotal-amurmann <gi...@git.apache.org>.
Github user pivotal-amurmann commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134066027
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/InvalidExecutionContextException.java ---
    @@ -0,0 +1,33 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    --- End diff --
    
    Where would be a better place for this? I am honestly just plain confused by the existing structure.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134005525
  
    --- Diff: geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java ---
    @@ -143,11 +150,12 @@
     
       public TcpServer(int port, InetAddress bind_address, Properties sslConfig,
           DistributionConfigImpl cfg, TcpHandler handler, PoolStatHelper poolHelper,
    -      ThreadGroup threadGroup, String threadName) {
    +      ThreadGroup threadGroup, String threadName, InternalLocator internalLocator) {
    --- End diff --
    
    Is there a reason why we pass in the specific `InternalLocator` rather than a more generic `Locator`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134007584
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/InvalidExecutionContextException.java ---
    @@ -0,0 +1,33 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    --- End diff --
    
    Is this class in the wrong package?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by galen-pivotal <gi...@git.apache.org>.
Github user galen-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134270934
  
    --- Diff: geode-core/src/test/java/org/apache/geode/test/dunit/standalone/DUnitLauncher.java ---
    @@ -297,6 +297,8 @@ public Object call() throws IOException {
             // able to do so successfully anyway
             p.setProperty(DISABLE_AUTO_RECONNECT, "true");
     
    +        System.setProperty("geode.feature-protobuf-protocol", "true");
    --- End diff --
    
    Do we want to do this for all DUnit tests? On the one hand, it might help shake some bugs out; on the other, we're setting a system property for all tests.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by pivotal-amurmann <gi...@git.apache.org>.
Github user pivotal-amurmann commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134005011
  
    --- Diff: geode-core/src/test/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java ---
    @@ -392,7 +388,7 @@ public void testLocatorUpdateIntervalZero() throws Exception {
     
       private void startFakeLocator() throws UnknownHostException, IOException, InterruptedException {
         server = new TcpServer(port, InetAddress.getLocalHost(), null, null, handler, new FakeHelper(),
    -        Thread.currentThread().getThreadGroup(), "Tcp Server");
    +        Thread.currentThread().getThreadGroup(), "Tcp Server", null);
    --- End diff --
    
    We added another parameter "protobuf handler" for TcpServer constructor to process protobuf messages. And it is nothing to do with existing code base and is only relevant for Protobuf requests. That's why we are passing null for old code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133780406
  
    --- Diff: geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/operations/GetAvailableServersOperationHandler.java ---
    @@ -50,51 +37,19 @@
       @Override
       public Result<ServerAPI.GetAvailableServersResponse> process(
           SerializationService serializationService, ServerAPI.GetAvailableServersRequest request,
    -      Cache cache) {
    -
    -    InternalDistributedSystem distributedSystem =
    -        (InternalDistributedSystem) cache.getDistributedSystem();
    -    Properties properties = distributedSystem.getProperties();
    -    String locatorsString = properties.getProperty(ConfigurationProperties.LOCATORS);
    -
    -    HashSet<DistributionLocatorId> locators = new HashSet();
    -    StringTokenizer stringTokenizer = new StringTokenizer(locatorsString, ",");
    -    while (stringTokenizer.hasMoreTokens()) {
    -      String locator = stringTokenizer.nextToken();
    -      if (StringUtils.isNotEmpty(locator)) {
    -        locators.add(new DistributionLocatorId(locator));
    -      }
    -    }
    +      ExecutionContext executionContext) throws InvalidExecutionContextException {
     
    -    TcpClient tcpClient = getTcpClient();
    -    for (DistributionLocatorId locator : locators) {
    -      try {
    -        return getGetAvailableServersFromLocator(tcpClient, locator.getHost());
    -      } catch (IOException | ClassNotFoundException e) {
    -        // try the next locator
    -      }
    -    }
    -    return Failure.of(ProtobufResponseUtilities.makeErrorResponse(
    -        ProtocolErrorCode.DATA_UNREACHABLE.codeValue, "Unable to find a locator"));
    -  }
    +    InternalLocator locator = executionContext.getLocator();
    +    ArrayList servers2 = locator.getServerLocatorAdvisee().getLoadSnapshot().getServers(null);
    --- End diff --
    
    why 'servers2'? What does the numeric denote? Maybe a different descriptive variable name is required... like 'availableServers'


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133778013
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ExecutionContext.java ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import org.apache.geode.cache.Cache;
    +import org.apache.geode.distributed.internal.InternalLocator;
    +
    +public class ExecutionContext {
    --- End diff --
    
    Would this not be a 'MessageExecutionContext' rather than a generic 'ExecutionContext'?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by galen-pivotal <gi...@git.apache.org>.
Github user galen-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134358612
  
    --- Diff: geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/operations/GetAvailableServersOperationHandler.java ---
    @@ -50,51 +37,23 @@
       @Override
       public Result<ServerAPI.GetAvailableServersResponse> process(
           SerializationService serializationService, ServerAPI.GetAvailableServersRequest request,
    -      Cache cache) {
    -
    -    InternalDistributedSystem distributedSystem =
    -        (InternalDistributedSystem) cache.getDistributedSystem();
    -    Properties properties = distributedSystem.getProperties();
    -    String locatorsString = properties.getProperty(ConfigurationProperties.LOCATORS);
    +      MessageExecutionContext executionContext) throws InvalidExecutionContextException {
     
    -    HashSet<DistributionLocatorId> locators = new HashSet();
    -    StringTokenizer stringTokenizer = new StringTokenizer(locatorsString, ",");
    -    while (stringTokenizer.hasMoreTokens()) {
    -      String locator = stringTokenizer.nextToken();
    -      if (StringUtils.isNotEmpty(locator)) {
    -        locators.add(new DistributionLocatorId(locator));
    -      }
    +    InternalLocator locator = executionContext.getLocator();
    +    ArrayList serversFromSnapshot =
    --- End diff --
    
    I don't like that this has to go so far in to get a list of servers. I'd like to see the DUnit test (or an integration test using a real Locator) come back just so we can have some guarantee that `InternalLocator` doesn't change on us.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134053987
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/InvalidExecutionContextException.java ---
    @@ -0,0 +1,33 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    --- End diff --
    
    What does an ExecutionContext have to do with socket?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by WireBaron <gi...@git.apache.org>.
Github user WireBaron commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133599146
  
    --- Diff: geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAvailableServersOperationHandlerJUnitTest.java ---
    @@ -32,75 +30,48 @@
     import org.junit.Test;
     import org.junit.experimental.categories.Category;
     
    -import java.io.IOException;
     import java.util.ArrayList;
    -import java.util.Properties;
     
     import static org.junit.Assert.assertEquals;
     import static org.junit.Assert.assertTrue;
     import static org.mockito.ArgumentMatchers.any;
    -import static org.mockito.ArgumentMatchers.anyBoolean;
    -import static org.mockito.ArgumentMatchers.anyInt;
     import static org.mockito.Mockito.mock;
     import static org.mockito.Mockito.when;
     
     @Category(UnitTest.class)
     public class GetAvailableServersOperationHandlerJUnitTest extends OperationHandlerJUnitTest {
    --- End diff --
    
    Are we getting anything of value from the OperationHandlerJUnitTest now that we're using a locator instead of a cache?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by WireBaron <gi...@git.apache.org>.
Github user WireBaron commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133599245
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ExecutionContext.java ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import org.apache.geode.cache.Cache;
    +import org.apache.geode.distributed.internal.InternalLocator;
    +
    +public class ExecutionContext {
    --- End diff --
    
    Should this be marked experimental like the exception?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by galen-pivotal <gi...@git.apache.org>.
Github user galen-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134022599
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtoclMessageHandlerLoader.java ---
    @@ -0,0 +1,64 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import java.io.IOException;
    +import java.net.Socket;
    +import java.util.Iterator;
    +import java.util.ServiceLoader;
    +
    +import org.apache.geode.internal.cache.InternalCache;
    +import org.apache.geode.internal.cache.tier.Acceptor;
    +import org.apache.geode.internal.cache.tier.CachedRegionHelper;
    +import org.apache.geode.internal.security.SecurityService;
    +
    +/**
    + * Creates instances of ServerConnection based on the connection mode provided.
    + */
    +public class ClientProtoclMessageHandlerLoader {
    --- End diff --
    
    Is this a duplicate that will be deleted?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133775690
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java ---
    @@ -76,7 +76,9 @@
      */
     public interface InternalCache extends Cache, Extensible<Cache>, CacheTime {
     
    -  InternalDistributedMember getMyId();
    +  default InternalDistributedMember getMyId() {
    --- End diff --
    
    so the default implementation will result in a potential NPE?!?!? I think this should NOT have a default and each InternalCache should implement this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by pivotal-amurmann <gi...@git.apache.org>.
Github user pivotal-amurmann commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134002721
  
    --- Diff: geode-core/src/test/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java ---
    @@ -392,7 +388,7 @@ public void testLocatorUpdateIntervalZero() throws Exception {
     
       private void startFakeLocator() throws UnknownHostException, IOException, InterruptedException {
         server = new TcpServer(port, InetAddress.getLocalHost(), null, null, handler, new FakeHelper(),
    -        Thread.currentThread().getThreadGroup(), "Tcp Server");
    +        Thread.currentThread().getThreadGroup(), "Tcp Server", null);
         server.start();
         Thread.sleep(500);
    --- End diff --
    
    It could, but we only really touched this file to account for the changed method signature


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133782267
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtoclMessageHandlerLoader.java ---
    @@ -0,0 +1,64 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import java.io.IOException;
    +import java.net.Socket;
    +import java.util.Iterator;
    +import java.util.ServiceLoader;
    +
    +import org.apache.geode.internal.cache.InternalCache;
    +import org.apache.geode.internal.cache.tier.Acceptor;
    +import org.apache.geode.internal.cache.tier.CachedRegionHelper;
    +import org.apache.geode.internal.security.SecurityService;
    +
    +/**
    + * Creates instances of ServerConnection based on the connection mode provided.
    --- End diff --
    
    I'm not sure that this comment is valid anymore... Something has to be said for not copy-pasting.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133779835
  
    --- Diff: geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/operations/GetAvailableServersOperationHandler.java ---
    @@ -50,51 +37,19 @@
       @Override
       public Result<ServerAPI.GetAvailableServersResponse> process(
           SerializationService serializationService, ServerAPI.GetAvailableServersRequest request,
    -      Cache cache) {
    -
    -    InternalDistributedSystem distributedSystem =
    -        (InternalDistributedSystem) cache.getDistributedSystem();
    -    Properties properties = distributedSystem.getProperties();
    -    String locatorsString = properties.getProperty(ConfigurationProperties.LOCATORS);
    -
    -    HashSet<DistributionLocatorId> locators = new HashSet();
    -    StringTokenizer stringTokenizer = new StringTokenizer(locatorsString, ",");
    -    while (stringTokenizer.hasMoreTokens()) {
    -      String locator = stringTokenizer.nextToken();
    -      if (StringUtils.isNotEmpty(locator)) {
    -        locators.add(new DistributionLocatorId(locator));
    -      }
    -    }
    +      ExecutionContext executionContext) throws InvalidExecutionContextException {
     
    -    TcpClient tcpClient = getTcpClient();
    -    for (DistributionLocatorId locator : locators) {
    -      try {
    -        return getGetAvailableServersFromLocator(tcpClient, locator.getHost());
    -      } catch (IOException | ClassNotFoundException e) {
    -        // try the next locator
    -      }
    -    }
    -    return Failure.of(ProtobufResponseUtilities.makeErrorResponse(
    -        ProtocolErrorCode.DATA_UNREACHABLE.codeValue, "Unable to find a locator"));
    -  }
    +    InternalLocator locator = executionContext.getLocator();
    +    ArrayList servers2 = locator.getServerLocatorAdvisee().getLoadSnapshot().getServers(null);
    --- End diff --
    
    I think if a 'null' is passed into the 'getServers' method, it should at least be explained that currently the server group functionality is not supported. Then a TODO could be added or a ticket can be raised to make sure this is not missed when server groups are supported


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by pivotal-amurmann <gi...@git.apache.org>.
Github user pivotal-amurmann commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r135087207
  
    --- Diff: geode-core/src/test/java/org/apache/geode/test/dunit/standalone/DUnitLauncher.java ---
    @@ -297,6 +297,8 @@ public Object call() throws IOException {
             // able to do so successfully anyway
             p.setProperty(DISABLE_AUTO_RECONNECT, "true");
     
    +        System.setProperty("geode.feature-protobuf-protocol", "true");
    --- End diff --
    
    Ideally we would not set this all the time. However, it would be a non-trivial effort to only set this when we need it and the feature flag should go away soon entirely.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by WireBaron <gi...@git.apache.org>.
Github user WireBaron commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133598026
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ExecutionContext.java ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import org.apache.geode.cache.Cache;
    +import org.apache.geode.distributed.internal.InternalLocator;
    +
    +public class ExecutionContext {
    --- End diff --
    
    Should this be marked @Experimental


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134006399
  
    --- Diff: geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java ---
    @@ -334,42 +342,46 @@ protected void run() {
        * fix for bug 33711 - client requests are spun off to another thread for processing. Requests are
        * synchronized in processGossip.
        */
    -  private void processRequest(final Socket sock) {
    +  private void processRequest(final Socket socket) {
         executor.execute(() -> {
           long startTime = DistributionStats.getStatTime();
           DataInputStream input = null;
           Object request, response;
           try {
     
    -        sock.setSoTimeout(READ_TIMEOUT);
    -        getSocketCreator().configureServerSSLSocket(sock);
    +        socket.setSoTimeout(READ_TIMEOUT);
    +        getSocketCreator().configureServerSSLSocket(socket);
     
             try {
    -          input = new DataInputStream(sock.getInputStream());
    +          input = new DataInputStream(socket.getInputStream());
             } catch (StreamCorruptedException e) {
               // Some garbage can be left on the socket stream
               // if a peer disappears at exactly the wrong moment.
               log.debug("Discarding illegal request from "
    -              + (sock.getInetAddress().getHostAddress() + ":" + sock.getPort()), e);
    +              + (socket.getInetAddress().getHostAddress() + ":" + socket.getPort()), e);
               return;
             }
    -        int gossipVersion = readGossipVersion(sock, input);
    +        int gossipVersion = readGossipVersion(socket, input);
     
             short versionOrdinal;
    +        if (gossipVersion == NON_GOSSIP_REQUEST_VERSION) {
    +          if (input.readUnsignedByte() == AcceptorImpl.PROTOBUF_CLIENT_SERVER_PROTOCOL
    +              && Boolean.getBoolean("geode.feature-protobuf-protocol")) {
    +            ClientProtocolMessageHandler messageHandler = ClientProtocolMessageHandlerLoader.load();
    --- End diff --
    
    Does it make sense to possibly pull out all the constants into a location that can be shared without cross-pollinating or referencing classes that aren't related


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by WireBaron <gi...@git.apache.org>.
Github user WireBaron commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133596604
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java ---
    @@ -76,7 +76,9 @@
      */
     public interface InternalCache extends Cache, Extensible<Cache>, CacheTime {
     
    -  InternalDistributedMember getMyId();
    +  default InternalDistributedMember getMyId() {
    --- End diff --
    
    I don't remember why this got done, is it still necessary?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133775860
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtoclMessageHandlerLoader.java ---
    @@ -0,0 +1,64 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import java.io.IOException;
    +import java.net.Socket;
    +import java.util.Iterator;
    +import java.util.ServiceLoader;
    +
    +import org.apache.geode.internal.cache.InternalCache;
    +import org.apache.geode.internal.cache.tier.Acceptor;
    +import org.apache.geode.internal.cache.tier.CachedRegionHelper;
    +import org.apache.geode.internal.security.SecurityService;
    +
    +/**
    + * Creates instances of ServerConnection based on the connection mode provided.
    + */
    +public class ClientProtoclMessageHandlerLoader {
    +  private static ClientProtocolMessageHandler protobufProtocolHandler;
    --- End diff --
    
    say no to statics


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133783618
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtoclMessageHandlerLoader.java ---
    @@ -0,0 +1,64 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import java.io.IOException;
    +import java.net.Socket;
    +import java.util.Iterator;
    +import java.util.ServiceLoader;
    +
    +import org.apache.geode.internal.cache.InternalCache;
    +import org.apache.geode.internal.cache.tier.Acceptor;
    --- End diff --
    
    Do we maybe need to remove some imports?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134006932
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ExecutionContext.java ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import org.apache.geode.cache.Cache;
    +import org.apache.geode.distributed.internal.InternalLocator;
    +
    +public class ExecutionContext {
    +  private Cache cache;
    +  private InternalLocator locator;
    +
    +  public ExecutionContext(Cache cache) {
    +    this.cache = cache;
    +  }
    +
    +  public ExecutionContext(InternalLocator locator) {
    +    this.locator = locator;
    +  }
    +
    +  // This throws if the cache isn't present because we know that non of the callers can take any
    +  // reasonable action if the cache is not present
    +  public Cache getCache() throws InvalidExecutionContextException {
    +    if (cache != null) {
    +      return cache;
    +    } else {
    +      throw new InvalidExecutionContextException(
    +          "Execution context's cache was accessed but isn't present. Did this happen on a locator? Operations on the locator should not try to operate on a cache");
    +    }
    +  }
    +
    +  // This throws if the locator isn't present because we know that non of the callers can take any
    +  // reasonable action if the locator is not present
    +  public InternalLocator getLocator() throws InvalidExecutionContextException {
    --- End diff --
    
    I think we should return a `Locator` rather than an `InternalLocator`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by galen-pivotal <gi...@git.apache.org>.
Github user galen-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134021523
  
    --- Diff: geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java ---
    @@ -334,107 +351,109 @@ protected void run() {
        * fix for bug 33711 - client requests are spun off to another thread for processing. Requests are
        * synchronized in processGossip.
        */
    -  private void processRequest(final Socket sock) {
    +  private void processRequest(final Socket socket) {
         executor.execute(() -> {
           long startTime = DistributionStats.getStatTime();
           DataInputStream input = null;
           Object request, response;
           try {
     
    -        sock.setSoTimeout(READ_TIMEOUT);
    -        getSocketCreator().configureServerSSLSocket(sock);
    +        socket.setSoTimeout(READ_TIMEOUT);
    +        getSocketCreator().configureServerSSLSocket(socket);
     
             try {
    -          input = new DataInputStream(sock.getInputStream());
    +          input = new DataInputStream(socket.getInputStream());
             } catch (StreamCorruptedException e) {
               // Some garbage can be left on the socket stream
               // if a peer disappears at exactly the wrong moment.
               log.debug("Discarding illegal request from "
    -              + (sock.getInetAddress().getHostAddress() + ":" + sock.getPort()), e);
    +              + (socket.getInetAddress().getHostAddress() + ":" + socket.getPort()), e);
               return;
             }
    -        int gossipVersion = readGossipVersion(sock, input);
    +        int gossipVersion = readGossipVersion(socket, input);
     
             short versionOrdinal;
    -        if (gossipVersion <= getCurrentGossipVersion()
    -            && GOSSIP_TO_GEMFIRE_VERSION_MAP.containsKey(gossipVersion)) {
    -          // Create a versioned stream to remember sender's GemFire version
    -          versionOrdinal = (short) GOSSIP_TO_GEMFIRE_VERSION_MAP.get(gossipVersion);
    -        } else {
    -          // Close the socket. We can not accept requests from a newer version
    -          try {
    -            sock.getOutputStream().write("unknown protocol version".getBytes());
    -            sock.getOutputStream().flush();
    -          } catch (IOException e) {
    -            log.debug(
    -                "exception in sending reply to process using unknown protocol " + gossipVersion, e);
    +        if (gossipVersion == NON_GOSSIP_REQUEST_VERSION) {
    +          if (input.readUnsignedByte() == PROTOBUF_CLIENT_SERVER_PROTOCOL
    +              && Boolean.getBoolean("geode.feature-protobuf-protocol")) {
    +            messageHandler.receiveMessage(input, socket.getOutputStream(),
    +                new MessageExecutionContext(internalLocator));
    +          } else {
    +            rejectUnknownProtocolConnection(socket, gossipVersion);
               }
    -          sock.close();
    -          return;
    -        }
    -        if (Version.GFE_71.compareTo(versionOrdinal) <= 0) {
    -          // Recent versions of TcpClient will send the version ordinal
    -          versionOrdinal = input.readShort();
    -        }
    -
    -        if (log.isDebugEnabled() && versionOrdinal != Version.CURRENT_ORDINAL) {
    -          log.debug("Locator reading request from " + sock.getInetAddress() + " with version "
    -              + Version.fromOrdinal(versionOrdinal, false));
    -        }
    -        input = new VersionedDataInputStream(input, Version.fromOrdinal(versionOrdinal, false));
    -        request = DataSerializer.readObject(input);
    -        if (log.isDebugEnabled()) {
    -          log.debug("Locator received request " + request + " from " + sock.getInetAddress());
    -        }
    -        if (request instanceof ShutdownRequest) {
    -          shuttingDown = true;
    -          // Don't call shutdown from within the worker thread, see java bug #6576792.
    -          // Closing the socket will cause our acceptor thread to shutdown the executor
    -          this.serverSocketPortAtClose = srv_sock.getLocalPort();
    -          srv_sock.close();
    -          response = new ShutdownResponse();
    -        } else if (request instanceof InfoRequest) {
    -          response = handleInfoRequest(request);
    -        } else if (request instanceof VersionRequest) {
    -          response = handleVersionRequest(request);
             } else {
    -          response = handler.processRequest(request);
    -        }
    -
    -        handler.endRequest(request, startTime);
    +          if (gossipVersion <= getCurrentGossipVersion()
    --- End diff --
    
    I think in my mind it would be clearer to front load the validation, i.e.
    ```java
            if (gossipVersion == NON_GOSSIP_REQUEST_VERSION) {
              if (input.readUnsignedByte() == PROTOBUF_CLIENT_SERVER_PROTOCOL
                  && Boolean.getBoolean("geode.feature-protobuf-protocol")) {
                messageHandler.receiveMessage(input, socket.getOutputStream(),
                    new MessageExecutionContext(internalLocator));
              } else {
                rejectUnknownProtocolConnection(socket, gossipVersion);
              }
            } else if (gossipVersion > getCurrentGossipVersion() || !GOSSIP_TO_GEMFIRE_VERSION_MAP.containsKey(gossipVersion)) {
                rejectUnknownProtocolConnection(socket, gossipVersion);
                return;
                
            } else {
                // Create a versioned stream to remember sender's GemFire version
                versionOrdinal = (short) GOSSIP_TO_GEMFIRE_VERSION_MAP.get(gossipVersion);
    
                ...
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by WireBaron <gi...@git.apache.org>.
Github user WireBaron commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133595959
  
    --- Diff: geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java ---
    @@ -120,6 +126,7 @@
       private InetAddress bind_address;
       private volatile boolean shuttingDown = false; // GemStoneAddition
       private final PoolStatHelper poolHelper;
    +  private InternalLocator internalLocator;
    --- End diff --
    
    Can this be final?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by galen-pivotal <gi...@git.apache.org>.
Github user galen-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134021675
  
    --- Diff: geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java ---
    @@ -334,107 +351,109 @@ protected void run() {
        * fix for bug 33711 - client requests are spun off to another thread for processing. Requests are
        * synchronized in processGossip.
        */
    -  private void processRequest(final Socket sock) {
    +  private void processRequest(final Socket socket) {
         executor.execute(() -> {
           long startTime = DistributionStats.getStatTime();
           DataInputStream input = null;
           Object request, response;
           try {
     
    -        sock.setSoTimeout(READ_TIMEOUT);
    -        getSocketCreator().configureServerSSLSocket(sock);
    +        socket.setSoTimeout(READ_TIMEOUT);
    +        getSocketCreator().configureServerSSLSocket(socket);
     
             try {
    -          input = new DataInputStream(sock.getInputStream());
    +          input = new DataInputStream(socket.getInputStream());
             } catch (StreamCorruptedException e) {
               // Some garbage can be left on the socket stream
               // if a peer disappears at exactly the wrong moment.
               log.debug("Discarding illegal request from "
    -              + (sock.getInetAddress().getHostAddress() + ":" + sock.getPort()), e);
    +              + (socket.getInetAddress().getHostAddress() + ":" + socket.getPort()), e);
               return;
             }
    -        int gossipVersion = readGossipVersion(sock, input);
    +        int gossipVersion = readGossipVersion(socket, input);
     
             short versionOrdinal;
    -        if (gossipVersion <= getCurrentGossipVersion()
    -            && GOSSIP_TO_GEMFIRE_VERSION_MAP.containsKey(gossipVersion)) {
    -          // Create a versioned stream to remember sender's GemFire version
    -          versionOrdinal = (short) GOSSIP_TO_GEMFIRE_VERSION_MAP.get(gossipVersion);
    -        } else {
    -          // Close the socket. We can not accept requests from a newer version
    -          try {
    -            sock.getOutputStream().write("unknown protocol version".getBytes());
    -            sock.getOutputStream().flush();
    -          } catch (IOException e) {
    -            log.debug(
    -                "exception in sending reply to process using unknown protocol " + gossipVersion, e);
    +        if (gossipVersion == NON_GOSSIP_REQUEST_VERSION) {
    +          if (input.readUnsignedByte() == PROTOBUF_CLIENT_SERVER_PROTOCOL
    +              && Boolean.getBoolean("geode.feature-protobuf-protocol")) {
    +            messageHandler.receiveMessage(input, socket.getOutputStream(),
    +                new MessageExecutionContext(internalLocator));
    +          } else {
    +            rejectUnknownProtocolConnection(socket, gossipVersion);
               }
    -          sock.close();
    -          return;
    -        }
    -        if (Version.GFE_71.compareTo(versionOrdinal) <= 0) {
    -          // Recent versions of TcpClient will send the version ordinal
    -          versionOrdinal = input.readShort();
    -        }
    -
    -        if (log.isDebugEnabled() && versionOrdinal != Version.CURRENT_ORDINAL) {
    -          log.debug("Locator reading request from " + sock.getInetAddress() + " with version "
    -              + Version.fromOrdinal(versionOrdinal, false));
    -        }
    -        input = new VersionedDataInputStream(input, Version.fromOrdinal(versionOrdinal, false));
    -        request = DataSerializer.readObject(input);
    -        if (log.isDebugEnabled()) {
    -          log.debug("Locator received request " + request + " from " + sock.getInetAddress());
    -        }
    -        if (request instanceof ShutdownRequest) {
    -          shuttingDown = true;
    -          // Don't call shutdown from within the worker thread, see java bug #6576792.
    -          // Closing the socket will cause our acceptor thread to shutdown the executor
    -          this.serverSocketPortAtClose = srv_sock.getLocalPort();
    -          srv_sock.close();
    -          response = new ShutdownResponse();
    -        } else if (request instanceof InfoRequest) {
    -          response = handleInfoRequest(request);
    -        } else if (request instanceof VersionRequest) {
    -          response = handleVersionRequest(request);
             } else {
    -          response = handler.processRequest(request);
    -        }
    -
    -        handler.endRequest(request, startTime);
    +          if (gossipVersion <= getCurrentGossipVersion()
    --- End diff --
    
    Also, we only early return in one rejection case; the end result is the same but it would be nice to be consistent.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by galen-pivotal <gi...@git.apache.org>.
Github user galen-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134024644
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/TcpServerFactory.java ---
    @@ -0,0 +1,39 @@
    +package org.apache.geode.internal.cache.tier.sockets;
    +
    +import java.net.InetAddress;
    +import java.util.Properties;
    +
    +import org.apache.geode.distributed.internal.DistributionConfigImpl;
    +import org.apache.geode.distributed.internal.InternalLocator;
    +import org.apache.geode.distributed.internal.PoolStatHelper;
    +import org.apache.geode.distributed.internal.tcpserver.TcpHandler;
    +import org.apache.geode.distributed.internal.tcpserver.TcpServer;
    +
    +public class TcpServerFactory {
    +  private ClientProtocolMessageHandler protocolHandler;
    +
    +  public TcpServerFactory() {
    +    initializeMessageHandler();
    +  }
    +
    +  public TcpServer makeTcpServer(int port, InetAddress bind_address, Properties sslConfig,
    +      DistributionConfigImpl cfg, TcpHandler handler, PoolStatHelper poolHelper,
    +      ThreadGroup threadGroup, String threadName, InternalLocator internalLocator) {
    +
    +    return new TcpServer(port, bind_address, sslConfig, cfg, handler, poolHelper, threadGroup,
    +        threadName, internalLocator, protocolHandler);
    +  }
    +
    +  public synchronized ClientProtocolMessageHandler initializeMessageHandler() {
    +    if (!Boolean.getBoolean("geode.feature-protobuf-protocol")) {
    +      return null;
    --- End diff --
    
    If we don't throw here, it will result in an NPE down the line when we try to receive a message from the null handler. I think it's better to throw here than return null -- see the similar `IOException` thrown by `ServerConnectionFactory`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by pivotal-amurmann <gi...@git.apache.org>.
Github user pivotal-amurmann commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133777949
  
    --- Diff: geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java ---
    @@ -334,42 +342,46 @@ protected void run() {
        * fix for bug 33711 - client requests are spun off to another thread for processing. Requests are
        * synchronized in processGossip.
        */
    -  private void processRequest(final Socket sock) {
    +  private void processRequest(final Socket socket) {
         executor.execute(() -> {
           long startTime = DistributionStats.getStatTime();
           DataInputStream input = null;
           Object request, response;
           try {
     
    -        sock.setSoTimeout(READ_TIMEOUT);
    -        getSocketCreator().configureServerSSLSocket(sock);
    +        socket.setSoTimeout(READ_TIMEOUT);
    +        getSocketCreator().configureServerSSLSocket(socket);
     
             try {
    -          input = new DataInputStream(sock.getInputStream());
    +          input = new DataInputStream(socket.getInputStream());
             } catch (StreamCorruptedException e) {
               // Some garbage can be left on the socket stream
               // if a peer disappears at exactly the wrong moment.
               log.debug("Discarding illegal request from "
    -              + (sock.getInetAddress().getHostAddress() + ":" + sock.getPort()), e);
    +              + (socket.getInetAddress().getHostAddress() + ":" + socket.getPort()), e);
               return;
             }
    -        int gossipVersion = readGossipVersion(sock, input);
    +        int gossipVersion = readGossipVersion(socket, input);
     
             short versionOrdinal;
    +        if (gossipVersion == NON_GOSSIP_REQUEST_VERSION) {
    +          if (input.readUnsignedByte() == AcceptorImpl.PROTOBUF_CLIENT_SERVER_PROTOCOL
    +              && Boolean.getBoolean("geode.feature-protobuf-protocol")) {
    +            ClientProtocolMessageHandler messageHandler = ClientProtocolMessageHandlerLoader.load();
    +            messageHandler.receiveMessage(input, socket.getOutputStream(),
    +                new ExecutionContext(internalLocator));
    --- End diff --
    
    after this we need to close the socket. --Hitesh


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by pivotal-amurmann <gi...@git.apache.org>.
Github user pivotal-amurmann commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133844781
  
    --- Diff: geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java ---
    @@ -334,42 +342,46 @@ protected void run() {
        * fix for bug 33711 - client requests are spun off to another thread for processing. Requests are
        * synchronized in processGossip.
        */
    -  private void processRequest(final Socket sock) {
    +  private void processRequest(final Socket socket) {
         executor.execute(() -> {
           long startTime = DistributionStats.getStatTime();
           DataInputStream input = null;
           Object request, response;
           try {
     
    -        sock.setSoTimeout(READ_TIMEOUT);
    -        getSocketCreator().configureServerSSLSocket(sock);
    +        socket.setSoTimeout(READ_TIMEOUT);
    +        getSocketCreator().configureServerSSLSocket(socket);
     
             try {
    -          input = new DataInputStream(sock.getInputStream());
    +          input = new DataInputStream(socket.getInputStream());
             } catch (StreamCorruptedException e) {
               // Some garbage can be left on the socket stream
               // if a peer disappears at exactly the wrong moment.
               log.debug("Discarding illegal request from "
    -              + (sock.getInetAddress().getHostAddress() + ":" + sock.getPort()), e);
    +              + (socket.getInetAddress().getHostAddress() + ":" + socket.getPort()), e);
               return;
             }
    -        int gossipVersion = readGossipVersion(sock, input);
    +        int gossipVersion = readGossipVersion(socket, input);
     
             short versionOrdinal;
    +        if (gossipVersion == NON_GOSSIP_REQUEST_VERSION) {
    +          if (input.readUnsignedByte() == AcceptorImpl.PROTOBUF_CLIENT_SERVER_PROTOCOL
    +              && Boolean.getBoolean("geode.feature-protobuf-protocol")) {
    +            ClientProtocolMessageHandler messageHandler = ClientProtocolMessageHandlerLoader.load();
    --- End diff --
    
    I would really like to have this constant only defined once. Agreed that AcceptorImpl is a bad spot for that. What would be a good spot?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133783516
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtoclMessageHandlerLoader.java ---
    @@ -0,0 +1,64 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import java.io.IOException;
    +import java.net.Socket;
    +import java.util.Iterator;
    +import java.util.ServiceLoader;
    +
    +import org.apache.geode.internal.cache.InternalCache;
    +import org.apache.geode.internal.cache.tier.Acceptor;
    +import org.apache.geode.internal.cache.tier.CachedRegionHelper;
    +import org.apache.geode.internal.security.SecurityService;
    +
    +/**
    + * Creates instances of ServerConnection based on the connection mode provided.
    + */
    +public class ClientProtoclMessageHandlerLoader {
    --- End diff --
    
    Do we not prefer 'Factories' over 'Loaders'?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by pivotal-amurmann <gi...@git.apache.org>.
Github user pivotal-amurmann commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133847176
  
    --- Diff: geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAvailableServersOperationHandlerJUnitTest.java ---
    @@ -32,75 +30,48 @@
     import org.junit.Test;
     import org.junit.experimental.categories.Category;
     
    -import java.io.IOException;
     import java.util.ArrayList;
    -import java.util.Properties;
     
     import static org.junit.Assert.assertEquals;
     import static org.junit.Assert.assertTrue;
     import static org.mockito.ArgumentMatchers.any;
    -import static org.mockito.ArgumentMatchers.anyBoolean;
    -import static org.mockito.ArgumentMatchers.anyInt;
     import static org.mockito.Mockito.mock;
     import static org.mockito.Mockito.when;
     
     @Category(UnitTest.class)
     public class GetAvailableServersOperationHandlerJUnitTest extends OperationHandlerJUnitTest {
    --- End diff --
    
    tried to remove it. It's quite painful. => probably the answer to your question is "yes"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134007487
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ExecutionContext.java ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    --- End diff --
    
    Is this class maybe in the wrong package?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by pivotal-amurmann <gi...@git.apache.org>.
Github user pivotal-amurmann commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133847775
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtoclMessageHandlerLoader.java ---
    @@ -0,0 +1,64 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import java.io.IOException;
    +import java.net.Socket;
    +import java.util.Iterator;
    +import java.util.ServiceLoader;
    +
    +import org.apache.geode.internal.cache.InternalCache;
    +import org.apache.geode.internal.cache.tier.Acceptor;
    +import org.apache.geode.internal.cache.tier.CachedRegionHelper;
    +import org.apache.geode.internal.security.SecurityService;
    +
    +/**
    + * Creates instances of ServerConnection based on the connection mode provided.
    + */
    +public class ClientProtoclMessageHandlerLoader {
    --- End diff --
    
    definitely. just trying to be consistent with code that was already there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by galen-pivotal <gi...@git.apache.org>.
Github user galen-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134023037
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactory.java ---
    @@ -49,20 +48,13 @@ private synchronized void initializeAuthenticatorsMap() {
       }
     
       private synchronized ClientProtocolMessageHandler initializeMessageHandler() {
    --- End diff --
    
    The return value of this method is being ignored. Perhaps it should return `void`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133781367
  
    --- Diff: geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAvailableServersOperationHandlerJUnitTest.java ---
    @@ -32,75 +30,48 @@
     import org.junit.Test;
     import org.junit.experimental.categories.Category;
     
    -import java.io.IOException;
     import java.util.ArrayList;
    -import java.util.Properties;
     
     import static org.junit.Assert.assertEquals;
     import static org.junit.Assert.assertTrue;
     import static org.mockito.ArgumentMatchers.any;
    -import static org.mockito.ArgumentMatchers.anyBoolean;
    -import static org.mockito.ArgumentMatchers.anyInt;
     import static org.mockito.Mockito.mock;
     import static org.mockito.Mockito.when;
     
     @Category(UnitTest.class)
     public class GetAvailableServersOperationHandlerJUnitTest extends OperationHandlerJUnitTest {
     
    -  private TcpClient mockTCPClient;
    +  public static final String HOSTNAME_1 = "hostname1";
    --- End diff --
    
    most likely these don't have to be 'public' AND 'static'. I believe 'private' and 'final' would be enough


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by WireBaron <gi...@git.apache.org>.
Github user WireBaron commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133597935
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ExecutionContext.java ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import org.apache.geode.cache.Cache;
    +import org.apache.geode.distributed.internal.InternalLocator;
    +
    +public class ExecutionContext {
    +  private Cache cache;
    +  private InternalLocator locator;
    +
    +  public ExecutionContext(Cache cache) {
    +    this.cache = cache;
    +  }
    +
    +  public ExecutionContext(InternalLocator locator) {
    +    this.locator = locator;
    +  }
    +
    +  // This throws if the cache isn't present because we know that non of the callers can take any
    +  // reasonable action if the cache is not present
    +  public Cache getCache() throws InvalidExecutionContextException {
    +    if (cache != null) {
    +      return cache;
    +    } else {
    +      throw new InvalidExecutionContextException(
    +          "Execution context's cache was accessed but isn't present. Did this happen on a locator? Operations on the locator should not try to operate on a cache");
    +    }
    +  }
    +
    +  // This throws if the locator isn't present because we know that non of the callers can take any
    +  // reasonable action if the locator is not present
    +  public InternalLocator getLocator() throws InvalidExecutionContextException {
    +    if (locator != null) {
    +      return locator;
    +    } else {
    +      throw new InvalidExecutionContextException(
    +          "Execution context's locator was accessed but isn't present. Did this happen on a server? Operations on the locator should not try to operate on a cache");
    --- End diff --
    
    Update comment "Operations on the server should not to try to operate on a locator.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by WireBaron <gi...@git.apache.org>.
Github user WireBaron commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133598920
  
    --- Diff: geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/operations/GetAvailableServersOperationHandler.java ---
    @@ -50,51 +37,19 @@
       @Override
       public Result<ServerAPI.GetAvailableServersResponse> process(
           SerializationService serializationService, ServerAPI.GetAvailableServersRequest request,
    -      Cache cache) {
    -
    -    InternalDistributedSystem distributedSystem =
    -        (InternalDistributedSystem) cache.getDistributedSystem();
    -    Properties properties = distributedSystem.getProperties();
    -    String locatorsString = properties.getProperty(ConfigurationProperties.LOCATORS);
    -
    -    HashSet<DistributionLocatorId> locators = new HashSet();
    -    StringTokenizer stringTokenizer = new StringTokenizer(locatorsString, ",");
    -    while (stringTokenizer.hasMoreTokens()) {
    -      String locator = stringTokenizer.nextToken();
    -      if (StringUtils.isNotEmpty(locator)) {
    -        locators.add(new DistributionLocatorId(locator));
    -      }
    -    }
    +      ExecutionContext executionContext) throws InvalidExecutionContextException {
     
    -    TcpClient tcpClient = getTcpClient();
    -    for (DistributionLocatorId locator : locators) {
    -      try {
    -        return getGetAvailableServersFromLocator(tcpClient, locator.getHost());
    -      } catch (IOException | ClassNotFoundException e) {
    -        // try the next locator
    -      }
    -    }
    -    return Failure.of(ProtobufResponseUtilities.makeErrorResponse(
    -        ProtocolErrorCode.DATA_UNREACHABLE.codeValue, "Unable to find a locator"));
    -  }
    +    InternalLocator locator = executionContext.getLocator();
    +    ArrayList servers2 = locator.getServerLocatorAdvisee().getLoadSnapshot().getServers(null);
    --- End diff --
    
    I see we didn't fix up this inane variable name I added, can you take care of that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133778667
  
    --- Diff: geode-core/src/test/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java ---
    @@ -392,7 +388,7 @@ public void testLocatorUpdateIntervalZero() throws Exception {
     
       private void startFakeLocator() throws UnknownHostException, IOException, InterruptedException {
         server = new TcpServer(port, InetAddress.getLocalHost(), null, null, handler, new FakeHelper(),
    -        Thread.currentThread().getThreadGroup(), "Tcp Server");
    +        Thread.currentThread().getThreadGroup(), "Tcp Server", null);
         server.start();
         Thread.sleep(500);
    --- End diff --
    
    Could this possibly be replaced with an Awaitility statement rather than a generic Thread.sleep which is more error prone


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134007044
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ExecutionContext.java ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import org.apache.geode.cache.Cache;
    +import org.apache.geode.distributed.internal.InternalLocator;
    +
    +public class ExecutionContext {
    +  private Cache cache;
    +  private InternalLocator locator;
    --- End diff --
    
    I think we should deal with the `Locator` interface rather than the concrete implementation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133775965
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtoclMessageHandlerLoader.java ---
    @@ -0,0 +1,64 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    +
    +import java.io.IOException;
    +import java.net.Socket;
    +import java.util.Iterator;
    +import java.util.ServiceLoader;
    +
    +import org.apache.geode.internal.cache.InternalCache;
    +import org.apache.geode.internal.cache.tier.Acceptor;
    +import org.apache.geode.internal.cache.tier.CachedRegionHelper;
    +import org.apache.geode.internal.security.SecurityService;
    +
    +/**
    + * Creates instances of ServerConnection based on the connection mode provided.
    + */
    +public class ClientProtoclMessageHandlerLoader {
    +  private static ClientProtocolMessageHandler protobufProtocolHandler;
    +  private static final Object protocolLoadLock = new Object();
    --- End diff --
    
    I think one can have a better implementation without statics and locking objects.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by WireBaron <gi...@git.apache.org>.
Github user WireBaron commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133596280
  
    --- Diff: geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java ---
    @@ -334,42 +342,46 @@ protected void run() {
        * fix for bug 33711 - client requests are spun off to another thread for processing. Requests are
        * synchronized in processGossip.
        */
    -  private void processRequest(final Socket sock) {
    +  private void processRequest(final Socket socket) {
         executor.execute(() -> {
           long startTime = DistributionStats.getStatTime();
           DataInputStream input = null;
           Object request, response;
           try {
     
    -        sock.setSoTimeout(READ_TIMEOUT);
    -        getSocketCreator().configureServerSSLSocket(sock);
    +        socket.setSoTimeout(READ_TIMEOUT);
    +        getSocketCreator().configureServerSSLSocket(socket);
     
             try {
    -          input = new DataInputStream(sock.getInputStream());
    +          input = new DataInputStream(socket.getInputStream());
             } catch (StreamCorruptedException e) {
               // Some garbage can be left on the socket stream
               // if a peer disappears at exactly the wrong moment.
               log.debug("Discarding illegal request from "
    -              + (sock.getInetAddress().getHostAddress() + ":" + sock.getPort()), e);
    +              + (socket.getInetAddress().getHostAddress() + ":" + socket.getPort()), e);
               return;
             }
    -        int gossipVersion = readGossipVersion(sock, input);
    +        int gossipVersion = readGossipVersion(socket, input);
     
             short versionOrdinal;
    +        if (gossipVersion == NON_GOSSIP_REQUEST_VERSION) {
    +          if (input.readUnsignedByte() == AcceptorImpl.PROTOBUF_CLIENT_SERVER_PROTOCOL
    +              && Boolean.getBoolean("geode.feature-protobuf-protocol")) {
    +            ClientProtocolMessageHandler messageHandler = ClientProtocolMessageHandlerLoader.load();
    --- End diff --
    
    Rather than introduce a new static object here, can we just have the TcpServer own a messageHandler?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133775201
  
    --- Diff: geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java ---
    @@ -334,42 +342,46 @@ protected void run() {
        * fix for bug 33711 - client requests are spun off to another thread for processing. Requests are
        * synchronized in processGossip.
        */
    -  private void processRequest(final Socket sock) {
    +  private void processRequest(final Socket socket) {
         executor.execute(() -> {
           long startTime = DistributionStats.getStatTime();
           DataInputStream input = null;
           Object request, response;
           try {
     
    -        sock.setSoTimeout(READ_TIMEOUT);
    -        getSocketCreator().configureServerSSLSocket(sock);
    +        socket.setSoTimeout(READ_TIMEOUT);
    +        getSocketCreator().configureServerSSLSocket(socket);
     
             try {
    -          input = new DataInputStream(sock.getInputStream());
    +          input = new DataInputStream(socket.getInputStream());
             } catch (StreamCorruptedException e) {
               // Some garbage can be left on the socket stream
               // if a peer disappears at exactly the wrong moment.
               log.debug("Discarding illegal request from "
    -              + (sock.getInetAddress().getHostAddress() + ":" + sock.getPort()), e);
    +              + (socket.getInetAddress().getHostAddress() + ":" + socket.getPort()), e);
               return;
             }
    -        int gossipVersion = readGossipVersion(sock, input);
    +        int gossipVersion = readGossipVersion(socket, input);
     
             short versionOrdinal;
    +        if (gossipVersion == NON_GOSSIP_REQUEST_VERSION) {
    +          if (input.readUnsignedByte() == AcceptorImpl.PROTOBUF_CLIENT_SERVER_PROTOCOL
    +              && Boolean.getBoolean("geode.feature-protobuf-protocol")) {
    +            ClientProtocolMessageHandler messageHandler = ClientProtocolMessageHandlerLoader.load();
    +            messageHandler.receiveMessage(input, socket.getOutputStream(),
    +                new ExecutionContext(internalLocator));
    +          } else {
    +            rejectUnknownProtocolConnection(socket, gossipVersion);
    +            return;
    +          }
    +        }
             if (gossipVersion <= getCurrentGossipVersion()
    --- End diff --
    
    why would this not be an 'else-if' construct. Or can the 'gossipVersion' be valid for both paths?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/geode/pull/716


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134514607
  
    --- Diff: geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/operations/GetAvailableServersOperationHandler.java ---
    @@ -50,51 +37,23 @@
       @Override
       public Result<ServerAPI.GetAvailableServersResponse> process(
           SerializationService serializationService, ServerAPI.GetAvailableServersRequest request,
    -      Cache cache) {
    -
    -    InternalDistributedSystem distributedSystem =
    -        (InternalDistributedSystem) cache.getDistributedSystem();
    -    Properties properties = distributedSystem.getProperties();
    -    String locatorsString = properties.getProperty(ConfigurationProperties.LOCATORS);
    +      MessageExecutionContext executionContext) throws InvalidExecutionContextException {
     
    -    HashSet<DistributionLocatorId> locators = new HashSet();
    -    StringTokenizer stringTokenizer = new StringTokenizer(locatorsString, ",");
    -    while (stringTokenizer.hasMoreTokens()) {
    -      String locator = stringTokenizer.nextToken();
    -      if (StringUtils.isNotEmpty(locator)) {
    -        locators.add(new DistributionLocatorId(locator));
    -      }
    +    InternalLocator locator = executionContext.getLocator();
    +    ArrayList serversFromSnapshot =
    --- End diff --
    
    I think the chances of the Locator changing on us would be really hard, as this code is running on a locator.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by kohlmu-pivotal <gi...@git.apache.org>.
Github user kohlmu-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r133774197
  
    --- Diff: geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java ---
    @@ -334,42 +342,46 @@ protected void run() {
        * fix for bug 33711 - client requests are spun off to another thread for processing. Requests are
        * synchronized in processGossip.
        */
    -  private void processRequest(final Socket sock) {
    +  private void processRequest(final Socket socket) {
         executor.execute(() -> {
           long startTime = DistributionStats.getStatTime();
           DataInputStream input = null;
           Object request, response;
           try {
     
    -        sock.setSoTimeout(READ_TIMEOUT);
    -        getSocketCreator().configureServerSSLSocket(sock);
    +        socket.setSoTimeout(READ_TIMEOUT);
    +        getSocketCreator().configureServerSSLSocket(socket);
     
             try {
    -          input = new DataInputStream(sock.getInputStream());
    +          input = new DataInputStream(socket.getInputStream());
             } catch (StreamCorruptedException e) {
               // Some garbage can be left on the socket stream
               // if a peer disappears at exactly the wrong moment.
               log.debug("Discarding illegal request from "
    -              + (sock.getInetAddress().getHostAddress() + ":" + sock.getPort()), e);
    +              + (socket.getInetAddress().getHostAddress() + ":" + socket.getPort()), e);
               return;
             }
    -        int gossipVersion = readGossipVersion(sock, input);
    +        int gossipVersion = readGossipVersion(socket, input);
     
             short versionOrdinal;
    +        if (gossipVersion == NON_GOSSIP_REQUEST_VERSION) {
    +          if (input.readUnsignedByte() == AcceptorImpl.PROTOBUF_CLIENT_SERVER_PROTOCOL
    +              && Boolean.getBoolean("geode.feature-protobuf-protocol")) {
    +            ClientProtocolMessageHandler messageHandler = ClientProtocolMessageHandlerLoader.load();
    --- End diff --
    
    Agreed. Is there a specific reason that TcpServer now needs to know about AcceptorImpl?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by galen-pivotal <gi...@git.apache.org>.
Github user galen-pivotal commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134026218
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/TcpServerFactory.java ---
    @@ -0,0 +1,39 @@
    +package org.apache.geode.internal.cache.tier.sockets;
    +
    +import java.net.InetAddress;
    +import java.util.Properties;
    +
    +import org.apache.geode.distributed.internal.DistributionConfigImpl;
    +import org.apache.geode.distributed.internal.InternalLocator;
    +import org.apache.geode.distributed.internal.PoolStatHelper;
    +import org.apache.geode.distributed.internal.tcpserver.TcpHandler;
    +import org.apache.geode.distributed.internal.tcpserver.TcpServer;
    +
    +public class TcpServerFactory {
    +  private ClientProtocolMessageHandler protocolHandler;
    +
    +  public TcpServerFactory() {
    +    initializeMessageHandler();
    +  }
    +
    +  public TcpServer makeTcpServer(int port, InetAddress bind_address, Properties sslConfig,
    +      DistributionConfigImpl cfg, TcpHandler handler, PoolStatHelper poolHelper,
    +      ThreadGroup threadGroup, String threadName, InternalLocator internalLocator) {
    +
    +    return new TcpServer(port, bind_address, sslConfig, cfg, handler, poolHelper, threadGroup,
    +        threadName, internalLocator, protocolHandler);
    +  }
    +
    +  public synchronized ClientProtocolMessageHandler initializeMessageHandler() {
    +    if (!Boolean.getBoolean("geode.feature-protobuf-protocol")) {
    +      return null;
    --- End diff --
    
    Ah, I see. `TcpServer` gates on the feature flag in the one place where it's used. And we create the same class regardless of old or new protocol version. 
    
    Could the factory determine whether the new protocol is enabled and make a `TcpServer` that either has the new protocol enabled or not? This way we're only checking config in one place.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] geode pull request #716: GEODE-3406: Locator accepts Protobuf requests

Posted by pivotal-amurmann <gi...@git.apache.org>.
Github user pivotal-amurmann commented on a diff in the pull request:

    https://github.com/apache/geode/pull/716#discussion_r134053063
  
    --- Diff: geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/InvalidExecutionContextException.java ---
    @@ -0,0 +1,33 @@
    +/*
    + * 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.geode.internal.cache.tier.sockets;
    --- End diff --
    
    It gets thrown by the execution context which gets passed in from TcpServer and GenericProtocolServerConnection


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---