You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@quickstep.apache.org by cramja <gi...@git.apache.org> on 2017/04/08 14:19:47 UTC

[GitHub] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

GitHub user cramja opened a pull request:

    https://github.com/apache/incubator-quickstep/pull/230

    Initial SingleNodeClient implementation

    This is the first part of a couple of PRs which will implement basic server functionality in QS. The basic usage of the client is:
    ```
    $ GLOG_logtostderr=1 ./quickstep_client --port 3001 < my_query.sql
    
    W0408 09:10:28.233783 3400758208 QuickstepClient.cpp:74] RPC call failed with code 14 and message: Connect Failed
    RPC failed
    ```
    The client uses gRPC libraries from the tmb and thus makes some changes to cli/CMakelists to include necessary libraries.
    
    This will not add additional functionality on top of the the existing CLI, it merely allows quickstep to run as a separate process.
    
    The next PR will have appropriate changes to the cli to handle network interactions.


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

    $ git pull https://github.com/cramja/incubator-quickstep initial-sns-client

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

    https://github.com/apache/incubator-quickstep/pull/230.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 #230
    
----
commit 77bdb74d3eb53af491ef0bb5629d54208835b3c9
Author: cramja <ma...@gmail.com>
Date:   2017-04-08T14:10:57Z

    Initial SingleNodeClient 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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524040
  
    --- Diff: cli/QuickstepClient.cpp ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.
    + **/
    +
    +#include <chrono>
    +#include <cstddef>
    +#include <cstdio>
    +#include <exception>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <utility>
    +#include <vector>
    --- End diff --
    
    Please run `iwyu` for all new source code files.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524006
  
    --- Diff: cli/CMakeLists.txt ---
    @@ -44,6 +44,20 @@ configure_file (
       "${CMAKE_CURRENT_BINARY_DIR}/CliConfig.h"
     )
     
    +# Compile the protos for Single Node Server mode.
    +if(ENABLE_SINGLE_NODE_SERVER)
    +  # We will need some of the TMBs libraries
    +  set(CMAKE_MODULE_PATH "${PROJECT_SOURCE_DIR}/third_party/src/tmb/cmake")
    +
    +  find_package(Grpc++ REQUIRED)
    +  include_directories(${GRPCPLUSPLUS_INCLUDE_DIRS})
    --- End diff --
    
    We should move above to the root `cmake` file. Also, it should look like
    
    ```
      set(CMAKE_MODULE_PATH "$CMAKE_MODULE_PATH"
                            "${PROJECT_SOURCE_DIR}/third_party/src/tmb/cmake")
    
      find_package(Grpc++ REQUIRED)
      include_directories(${GRPCPLUSPLUS_INCLUDE_DIRS})
    ```


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524074
  
    --- Diff: cli/QuickstepClient.cpp ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.
    + **/
    +
    +#include <chrono>
    +#include <cstddef>
    +#include <cstdio>
    +#include <exception>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <utility>
    +#include <vector>
    +
    +#include "cli/SingleNodeServer.grpc.pb.h"
    +#include "cli/SingleNodeServer.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +#include "grpc++/grpc++.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  if (value >= 0 && value < 32768)
    +    return true;
    +  printf("Invalid value for --%s: %d\n", flagname, (int) value);
    --- End diff --
    
    Use `std::printf`.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524143
  
    --- Diff: cli/QuickstepClient.cpp ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.
    + **/
    +
    +#include <chrono>
    +#include <cstddef>
    +#include <cstdio>
    +#include <exception>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <utility>
    +#include <vector>
    +
    +#include "cli/SingleNodeServer.grpc.pb.h"
    +#include "cli/SingleNodeServer.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +#include "grpc++/grpc++.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  if (value >= 0 && value < 32768)
    +    return true;
    +  printf("Invalid value for --%s: %d\n", flagname, (int) value);
    +  return false;
    +}
    +
    +DEFINE_int32(port, 3000, "Listens for TCP connection on this port.");
    +DEFINE_validator(port, &ValidatePort);
    +
    +namespace quickstep {
    +
    +class QuickstepClient {
    +public:
    +  QuickstepClient(std::shared_ptr<Channel> channel)
    --- End diff --
    
    Use `const reference` to avoid a deep copy of a smart pointer.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110542764
  
    --- Diff: cli/QuickstepClient.cpp ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.
    + **/
    +
    +#include <chrono>
    +#include <cstddef>
    +#include <cstdio>
    +#include <exception>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <utility>
    +#include <vector>
    --- End diff --
    
    I removed some of the obviously not needed includes. However, these tools are imprecise, and I find that their recommendations don't always work, or agree with one another. I.e. Clion IDE gives me recommendations as well but they are different from iwyu. 


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524205
  
    --- Diff: cli/QuickstepClient.cpp ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.
    + **/
    +
    +#include <chrono>
    +#include <cstddef>
    +#include <cstdio>
    +#include <exception>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <utility>
    +#include <vector>
    +
    +#include "cli/SingleNodeServer.grpc.pb.h"
    +#include "cli/SingleNodeServer.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +#include "grpc++/grpc++.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  if (value >= 0 && value < 32768)
    +    return true;
    +  printf("Invalid value for --%s: %d\n", flagname, (int) value);
    +  return false;
    +}
    +
    +DEFINE_int32(port, 3000, "Listens for TCP connection on this port.");
    +DEFINE_validator(port, &ValidatePort);
    +
    +namespace quickstep {
    +
    +class QuickstepClient {
    +public:
    +  QuickstepClient(std::shared_ptr<Channel> channel)
    +    : stub_(SingleNodeServerRequest::NewStub(channel)) {}
    +
    +  /**
    +   * Assembles the client's payload, sends it and presents the response back from the server.
    +   * @param user_query A SQL statement or command to be executed on the server.
    +   * @return The text of the server's response.
    +   */
    +  std::string SendQuery(const std::string &user_query) {
    +    QueryRequest request;
    +    request.set_query(user_query);
    +    QueryResponse response;
    +    ClientContext context;
    +
    +    Status status = stub_->SendQuery(&context, request, &response);
    +
    +    if (status.ok()) {
    +      return HandleQueryResponse(response);
    +    } else {
    +      LOG(WARNING) << "RPC call failed with code " << status.error_code()
    +                   << " and message: " << status.error_message();
    +      return "RPC failed";
    +    }
    +  }
    +
    +private:
    +  /**
    +   * Handle a valid response from the server.
    +   * @param response A valid query response.
    +   * @return The response string.
    +   */
    +  std::string HandleQueryResponse(QueryResponse const &response) {
    --- End diff --
    
    Use `std::string HandleQueryResponse(const QueryResponse & response) const {`.
    
    In fact, it could be a static, non-member method..


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524222
  
    --- Diff: cli/QuickstepClient.cpp ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.
    + **/
    +
    +#include <chrono>
    +#include <cstddef>
    +#include <cstdio>
    +#include <exception>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <utility>
    +#include <vector>
    +
    +#include "cli/SingleNodeServer.grpc.pb.h"
    +#include "cli/SingleNodeServer.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +#include "grpc++/grpc++.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  if (value >= 0 && value < 32768)
    +    return true;
    +  printf("Invalid value for --%s: %d\n", flagname, (int) value);
    +  return false;
    +}
    +
    +DEFINE_int32(port, 3000, "Listens for TCP connection on this port.");
    +DEFINE_validator(port, &ValidatePort);
    +
    +namespace quickstep {
    +
    +class QuickstepClient {
    +public:
    +  QuickstepClient(std::shared_ptr<Channel> channel)
    +    : stub_(SingleNodeServerRequest::NewStub(channel)) {}
    +
    +  /**
    +   * Assembles the client's payload, sends it and presents the response back from the server.
    +   * @param user_query A SQL statement or command to be executed on the server.
    +   * @return The text of the server's response.
    +   */
    +  std::string SendQuery(const std::string &user_query) {
    +    QueryRequest request;
    +    request.set_query(user_query);
    +    QueryResponse response;
    +    ClientContext context;
    +
    +    Status status = stub_->SendQuery(&context, request, &response);
    +
    +    if (status.ok()) {
    +      return HandleQueryResponse(response);
    +    } else {
    +      LOG(WARNING) << "RPC call failed with code " << status.error_code()
    +                   << " and message: " << status.error_message();
    +      return "RPC failed";
    +    }
    +  }
    +
    +private:
    +  /**
    +   * Handle a valid response from the server.
    +   * @param response A valid query response.
    +   * @return The response string.
    +   */
    +  std::string HandleQueryResponse(QueryResponse const &response) {
    +    if(response.error()) {
    +      return response.error_result();
    +    }
    +    return response.query_result();
    +  }
    +
    +  std::unique_ptr<SingleNodeServerRequest::Stub> stub_;
    +};
    +}
    --- End diff --
    
    Code style: add an empty line and change to `}  // quickstep`.


---
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] incubator-quickstep issue #230: Initial SingleNodeClient implementation

Posted by cramja <gi...@git.apache.org>.
Github user cramja commented on the issue:

    https://github.com/apache/incubator-quickstep/pull/230
  
    Oh shoot, so no Travis for dist?


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524371
  
    --- Diff: cli/SingleNodeServer.proto ---
    @@ -0,0 +1,34 @@
    +// 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.
    +
    +syntax = "proto3";
    +
    +package quickstep;
    +
    +service SingleNodeServerRequest {
    --- End diff --
    
    Recommend to rename to `SingleNodeServer`, so that the service name is the same as the file name.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110542406
  
    --- Diff: cli/CMakeLists.txt ---
    @@ -44,6 +44,20 @@ configure_file (
       "${CMAKE_CURRENT_BINARY_DIR}/CliConfig.h"
     )
     
    +# Compile the protos for Single Node Server mode.
    +if(ENABLE_SINGLE_NODE_SERVER)
    +  # We will need some of the TMBs libraries
    +  set(CMAKE_MODULE_PATH "${PROJECT_SOURCE_DIR}/third_party/src/tmb/cmake")
    +
    +  find_package(Grpc++ REQUIRED)
    +  include_directories(${GRPCPLUSPLUS_INCLUDE_DIRS})
    --- End diff --
    
    Also good catch on keeping the orig module path
    
    Note that cmake likes brackets:
    ```
    set(CMAKE_MODULE_PATH
          ${CMAKE_MODULE_PATH}
          "${PROJECT_SOURCE_DIR}/third_party/src/tmb/cmake")
    ```
    
    works while
    
    ```
    set(CMAKE_MODULE_PATH
          $CMAKE_MODULE_PATH
          "${PROJECT_SOURCE_DIR}/third_party/src/tmb/cmake")
    ```
    treats the middle argument like a string literal


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524066
  
    --- Diff: cli/QuickstepClient.cpp ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.
    + **/
    +
    +#include <chrono>
    +#include <cstddef>
    +#include <cstdio>
    +#include <exception>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <utility>
    +#include <vector>
    +
    +#include "cli/SingleNodeServer.grpc.pb.h"
    +#include "cli/SingleNodeServer.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +#include "grpc++/grpc++.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  if (value >= 0 && value < 32768)
    --- End diff --
    
    A valid value should be `(0, 65536)`, not including both boundaries.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110560109
  
    --- Diff: cli/SingleNodeServer.proto ---
    @@ -0,0 +1,34 @@
    +// 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.
    +
    +syntax = "proto3";
    +
    +package quickstep;
    +
    +service SingleNodeServerRequest {
    --- End diff --
    
    Then, change the file name as well.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110523885
  
    --- Diff: CMakeLists.txt ---
    @@ -696,7 +697,8 @@ set(ENABLE_ZOOKEEPER OFF CACHE BOOL "Enable Zookeeper TMB")
     add_subdirectory("${THIRD_PARTY_SOURCE_DIR}/tmb" "${CMAKE_CURRENT_BINARY_DIR}/third_party/tmb")
     include_directories(${TMB_INCLUDE_DIRS})
     
    -if (ENABLE_DISTRIBUTED)
    +# Single node server requires this for gRPC headers
    +if (ENABLE_DISTRIBUTED OR ENABLE_SINGLE_NODE_SERVER)
    --- End diff --
    
    Reset the change, as `gRPC` does not come from `TMB`.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110560093
  
    --- Diff: CMakeLists.txt ---
    @@ -145,6 +145,7 @@ if (ENABLE_VECTOR_PREDICATE_SHORT_CIRCUIT)
       )
     endif()
     
    +option(ENABLE_SINGLE_NODE_SERVER "Enables server mode in single node setting" ON)
    --- End diff --
    
    Yes, for now `Quickstep` does not declare the `grpc` dependency. So, by default, we don't build any binaries that link to `grpc`.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110523933
  
    --- Diff: CMakeLists.txt ---
    @@ -792,3 +794,13 @@ if (ENABLE_DISTRIBUTED)
                             ${GFLAGS_LIB_NAME}
                             ${GRPCPLUSPLUS_LIBRARIES})
     endif(ENABLE_DISTRIBUTED)
    +
    +if(ENABLE_SINGLE_NODE_SERVER)
    +  add_executable (quickstep_client cli/QuickstepClient.cpp)
    +  target_link_libraries(quickstep_client
    +            ${GFLAGS_LIB_NAME}
    +            glog
    +            ${GRPCPLUSPLUS_LIBRARIES}
    +            ${PROTOBUF3_LIBRARIES}
    +            quickstep_cli_SingleNodeServer_proto)
    --- End diff --
    
    All linking libraries should be aligned with `(`, and sort in the alphabet order.
    
    ```
                 glog
                 quickstep_cli_SingleNodeServer_proto
                 ${GFLAGS_LIB_NAME}
                 ${GRPCPLUSPLUS_LIBRARIES})
    ```


---
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] incubator-quickstep issue #230: Initial SingleNodeClient implementation

Posted by cramja <gi...@git.apache.org>.
Github user cramja commented on the issue:

    https://github.com/apache/incubator-quickstep/pull/230
  
    @zuyu I'm going to close this PR and submit another one with the tests as I think the content has changed sufficiently enough to have another PR


---
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] incubator-quickstep issue #230: Initial SingleNodeClient implementation

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on the issue:

    https://github.com/apache/incubator-quickstep/pull/230
  
    Unfortunately, for now no CIs for the distributed version due to both `Protobuf 3` and `gRPC` dependencies. But we definitely should have!
    
    One thing to add is that ccache could speed up the build time to avoid time-out, but I have no luck to set it up correctly.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110523774
  
    --- Diff: CMakeLists.txt ---
    @@ -145,6 +145,7 @@ if (ENABLE_VECTOR_PREDICATE_SHORT_CIRCUIT)
       )
     endif()
     
    +option(ENABLE_SINGLE_NODE_SERVER "Enables server mode in single node setting" ON)
    --- End diff --
    
    This should by default be `off`.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524263
  
    --- Diff: cli/QuickstepClient.cpp ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.
    + **/
    +
    +#include <chrono>
    +#include <cstddef>
    +#include <cstdio>
    +#include <exception>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <utility>
    +#include <vector>
    +
    +#include "cli/SingleNodeServer.grpc.pb.h"
    +#include "cli/SingleNodeServer.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +#include "grpc++/grpc++.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  if (value >= 0 && value < 32768)
    +    return true;
    +  printf("Invalid value for --%s: %d\n", flagname, (int) value);
    +  return false;
    +}
    +
    +DEFINE_int32(port, 3000, "Listens for TCP connection on this port.");
    +DEFINE_validator(port, &ValidatePort);
    +
    +namespace quickstep {
    +
    +class QuickstepClient {
    +public:
    +  QuickstepClient(std::shared_ptr<Channel> channel)
    +    : stub_(SingleNodeServerRequest::NewStub(channel)) {}
    +
    +  /**
    +   * Assembles the client's payload, sends it and presents the response back from the server.
    +   * @param user_query A SQL statement or command to be executed on the server.
    +   * @return The text of the server's response.
    +   */
    +  std::string SendQuery(const std::string &user_query) {
    +    QueryRequest request;
    +    request.set_query(user_query);
    +    QueryResponse response;
    +    ClientContext context;
    +
    +    Status status = stub_->SendQuery(&context, request, &response);
    +
    +    if (status.ok()) {
    +      return HandleQueryResponse(response);
    +    } else {
    +      LOG(WARNING) << "RPC call failed with code " << status.error_code()
    +                   << " and message: " << status.error_message();
    +      return "RPC failed";
    +    }
    +  }
    +
    +private:
    +  /**
    +   * Handle a valid response from the server.
    +   * @param response A valid query response.
    +   * @return The response string.
    +   */
    +  std::string HandleQueryResponse(QueryResponse const &response) {
    +    if(response.error()) {
    +      return response.error_result();
    +    }
    +    return response.query_result();
    +  }
    +
    +  std::unique_ptr<SingleNodeServerRequest::Stub> stub_;
    +};
    +}
    +
    +int main(int argc, char** argv) {
    +  google::InitGoogleLogging(argv[0]);
    +  gflags::ParseCommandLineFlags(&argc, &argv, true);
    +
    +  // Attempts to send a single query retrieved from stdin to the Quickstep Server.
    +  quickstep::QuickstepClient qs_client(
    +    grpc::CreateChannel("localhost:" + std::to_string(FLAGS_port),
    +    grpc::InsecureChannelCredentials()));
    --- End diff --
    
    Both lines above should add two more whitespace indentations.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524355
  
    --- Diff: cli/SingleNodeServer.proto ---
    @@ -0,0 +1,34 @@
    +// 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.
    +
    +syntax = "proto3";
    +
    +package quickstep;
    +
    +service SingleNodeServerRequest {
    +    rpc SendQuery (QueryRequest) returns (QueryResponse) {}
    +}
    +
    +message QueryRequest {
    +    string query = 1;
    +}
    +
    +message QueryResponse {
    +    bool error = 1;
    +    string query_result = 2;
    +    string error_result = 3;
    +}
    --- End diff --
    
    Please reduce to two whitespace indentations, and add `EOL`.


---
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] incubator-quickstep issue #230: Initial SingleNodeClient implementation

Posted by cramja <gi...@git.apache.org>.
Github user cramja commented on the issue:

    https://github.com/apache/incubator-quickstep/pull/230
  
    @zuyu Thank you for the review
    
    Here's a possibility based on your overall comment: I could add the IOBase and its subclass GrpcWrapper to this PR. The IOBase abstracts around all IO (including LineReaders). The GrpcWrapper is a wrapper around server side interactions. Then the unit test would test communication between GrpcWrapper and the Client.
    
    I was thinking to add 1 new executable to Quickstep, that being the dumb client. All it really is is an interface for a user to send a query through. It just handles send and receive, therefore does no parsing/etc. That can come later if we want.
    
    Then in another PR the Cli exe can be modified to have the `--mode=server` option as you suggested.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524033
  
    --- Diff: cli/QuickstepClient.cpp ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.
    + **/
    +
    +#include <chrono>
    +#include <cstddef>
    +#include <cstdio>
    +#include <exception>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <utility>
    +#include <vector>
    +
    +#include "cli/SingleNodeServer.grpc.pb.h"
    +#include "cli/SingleNodeServer.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +#include "grpc++/grpc++.h"
    --- End diff --
    
    Should be `<grpc++/grpc++.h>`.


---
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] incubator-quickstep issue #230: Initial SingleNodeClient implementation

Posted by cramja <gi...@git.apache.org>.
Github user cramja commented on the issue:

    https://github.com/apache/incubator-quickstep/pull/230
  
    @zuyu is there something special we specify in Travis to get Proto3?


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524089
  
    --- Diff: cli/QuickstepClient.cpp ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.
    + **/
    +
    +#include <chrono>
    +#include <cstddef>
    +#include <cstdio>
    +#include <exception>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <utility>
    +#include <vector>
    +
    +#include "cli/SingleNodeServer.grpc.pb.h"
    +#include "cli/SingleNodeServer.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +#include "grpc++/grpc++.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  if (value >= 0 && value < 32768)
    +    return true;
    +  printf("Invalid value for --%s: %d\n", flagname, (int) value);
    +  return false;
    +}
    +
    +DEFINE_int32(port, 3000, "Listens for TCP connection on this port.");
    +DEFINE_validator(port, &ValidatePort);
    +
    +namespace quickstep {
    +
    +class QuickstepClient {
    +public:
    --- End diff --
    
    Add one whitespace indentation. Similarly for `private:` below.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110542269
  
    --- Diff: cli/CMakeLists.txt ---
    @@ -44,6 +44,20 @@ configure_file (
       "${CMAKE_CURRENT_BINARY_DIR}/CliConfig.h"
     )
     
    +# Compile the protos for Single Node Server mode.
    +if(ENABLE_SINGLE_NODE_SERVER)
    +  # We will need some of the TMBs libraries
    +  set(CMAKE_MODULE_PATH "${PROJECT_SOURCE_DIR}/third_party/src/tmb/cmake")
    +
    +  find_package(Grpc++ REQUIRED)
    +  include_directories(${GRPCPLUSPLUS_INCLUDE_DIRS})
    --- End diff --
    
    Seems to me like leaving it in `/cli/cmakelists.txt` should be fine because the module is being used by sources in `/cli/` but not necessarily by sources in `/`.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524125
  
    --- Diff: cli/QuickstepClient.cpp ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.
    + **/
    +
    +#include <chrono>
    +#include <cstddef>
    +#include <cstdio>
    +#include <exception>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <utility>
    +#include <vector>
    +
    +#include "cli/SingleNodeServer.grpc.pb.h"
    +#include "cli/SingleNodeServer.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +#include "grpc++/grpc++.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  if (value >= 0 && value < 32768)
    +    return true;
    +  printf("Invalid value for --%s: %d\n", flagname, (int) value);
    +  return false;
    +}
    +
    +DEFINE_int32(port, 3000, "Listens for TCP connection on this port.");
    +DEFINE_validator(port, &ValidatePort);
    +
    +namespace quickstep {
    +
    +class QuickstepClient {
    +public:
    +  QuickstepClient(std::shared_ptr<Channel> channel)
    +    : stub_(SingleNodeServerRequest::NewStub(channel)) {}
    --- End diff --
    
    Add two more whitespace indentation.


---
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] incubator-quickstep issue #230: Initial SingleNodeClient implementation

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on the issue:

    https://github.com/apache/incubator-quickstep/pull/230
  
    For this PR, we could add two new classes: `client` and `server`, and a unit test to test the communication path between both. No need changes to any existing source code files.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110542459
  
    --- Diff: cli/CMakeLists.txt ---
    @@ -44,6 +44,20 @@ configure_file (
       "${CMAKE_CURRENT_BINARY_DIR}/CliConfig.h"
     )
     
    +# Compile the protos for Single Node Server mode.
    +if(ENABLE_SINGLE_NODE_SERVER)
    +  # We will need some of the TMBs libraries
    +  set(CMAKE_MODULE_PATH "${PROJECT_SOURCE_DIR}/third_party/src/tmb/cmake")
    +
    +  find_package(Grpc++ REQUIRED)
    +  include_directories(${GRPCPLUSPLUS_INCLUDE_DIRS})
    --- End diff --
    
    Also not that because of CMAKE scoping, the module will not be added to the parent scope's module path:
    
    `/cmakelists.txt`:
    ```
    message("before module path: ${CMAKE_MODULE_PATH}")
    add_subdirectory(cli)
    message("after module path: ${CMAKE_MODULE_PATH}")
    ```
    output:
    ```
    before module path: /Users/cramja/workspace/quickstep/incubator-quickstep/cmake
    after module path: /Users/cramja/workspace/quickstep/incubator-quickstep/cmake
    ```


---
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] incubator-quickstep issue #230: Initial SingleNodeClient implementation

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on the issue:

    https://github.com/apache/incubator-quickstep/pull/230
  
    There are several ways to get `Protobuf 3` and `gRPC` in Travis, but unfortunately all may result in time-out.
     * PR #223 upgrades git submodule in the third_party.
     * PR #220 downloads `Protobuf 3` source code and builds locally.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110542826
  
    --- Diff: cli/SingleNodeServer.proto ---
    @@ -0,0 +1,34 @@
    +// 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.
    +
    +syntax = "proto3";
    +
    +package quickstep;
    +
    +service SingleNodeServerRequest {
    --- End diff --
    
    maybe `SingleNodeServerService` ?


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110542523
  
    --- Diff: CMakeLists.txt ---
    @@ -145,6 +145,7 @@ if (ENABLE_VECTOR_PREDICATE_SHORT_CIRCUIT)
       )
     endif()
     
    +option(ENABLE_SINGLE_NODE_SERVER "Enables server mode in single node setting" ON)
    --- End diff --
    
    Does it need to be for any reason other than travis?


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524344
  
    --- Diff: cli/QuickstepClient.cpp ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.
    + **/
    +
    +#include <chrono>
    +#include <cstddef>
    +#include <cstdio>
    +#include <exception>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <utility>
    +#include <vector>
    +
    +#include "cli/SingleNodeServer.grpc.pb.h"
    +#include "cli/SingleNodeServer.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +#include "grpc++/grpc++.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  if (value >= 0 && value < 32768)
    +    return true;
    +  printf("Invalid value for --%s: %d\n", flagname, (int) value);
    +  return false;
    +}
    +
    +DEFINE_int32(port, 3000, "Listens for TCP connection on this port.");
    +DEFINE_validator(port, &ValidatePort);
    +
    +namespace quickstep {
    +
    +class QuickstepClient {
    +public:
    +  QuickstepClient(std::shared_ptr<Channel> channel)
    +    : stub_(SingleNodeServerRequest::NewStub(channel)) {}
    +
    +  /**
    +   * Assembles the client's payload, sends it and presents the response back from the server.
    +   * @param user_query A SQL statement or command to be executed on the server.
    +   * @return The text of the server's response.
    +   */
    +  std::string SendQuery(const std::string &user_query) {
    +    QueryRequest request;
    +    request.set_query(user_query);
    +    QueryResponse response;
    +    ClientContext context;
    +
    +    Status status = stub_->SendQuery(&context, request, &response);
    +
    +    if (status.ok()) {
    +      return HandleQueryResponse(response);
    +    } else {
    +      LOG(WARNING) << "RPC call failed with code " << status.error_code()
    +                   << " and message: " << status.error_message();
    +      return "RPC failed";
    +    }
    +  }
    +
    +private:
    +  /**
    +   * Handle a valid response from the server.
    +   * @param response A valid query response.
    +   * @return The response string.
    +   */
    +  std::string HandleQueryResponse(QueryResponse const &response) {
    +    if(response.error()) {
    +      return response.error_result();
    +    }
    +    return response.query_result();
    +  }
    +
    +  std::unique_ptr<SingleNodeServerRequest::Stub> stub_;
    +};
    +}
    +
    +int main(int argc, char** argv) {
    +  google::InitGoogleLogging(argv[0]);
    +  gflags::ParseCommandLineFlags(&argc, &argv, true);
    +
    +  // Attempts to send a single query retrieved from stdin to the Quickstep Server.
    +  quickstep::QuickstepClient qs_client(
    +    grpc::CreateChannel("localhost:" + std::to_string(FLAGS_port),
    +    grpc::InsecureChannelCredentials()));
    +  std::string user_query;
    +  std::cin >> user_query;
    --- End diff --
    
    We really need some tests. This does not work, even for a simply `SELECT-one` query, as it can not handle an input with whitespace.


---
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] incubator-quickstep pull request #230: Initial SingleNodeClient implementati...

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

    https://github.com/apache/incubator-quickstep/pull/230#discussion_r110524021
  
    --- Diff: cli/CMakeLists.txt ---
    @@ -150,7 +170,15 @@ if(USE_LINENOISE)
     else()
       target_link_libraries(quickstep_cli_LineReader
                             quickstep_utility_Macros)
    +
     endif()
    +
    +if(ENABLE_SINGLE_NODE_SERVER)
    +  target_link_libraries(quickstep_cli_SingleNodeServer_proto
    +                        ${PROTOBUF_LIBRARY}
    --- End diff --
    
    Should be `${PROTOBUF3_LIBRARIES}`.


---
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.
---