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/17 23:00:49 UTC

[GitHub] incubator-quickstep pull request #232: Adds network cli interface.

GitHub user cramja opened a pull request:

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

    Adds network cli interface.

    # Adds Network CLI functionality to Single Node
    
    **Functional Changes include:**
    Then, run quickstep in `network` mode (defaults to `local`)
    
    `quickstep_cli_shell --mode=network`
    
    Now quickstep boots a server, you can interact using the client executable...
    
    `quickstep_client < my_query.sql`
    
    **Code Changes include:**
    * Adds a generic IO interface
      * Hides Linereader from CLI behind class `LocalIO`
      * New `NetworkIO` also hidden here
    * Adds tests for NetworkIO
    * Adds simple Client executable 
    * Cmake flag to disable (`ENABLE_NETWORK_CLI=false`)

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

    $ git pull https://github.com/cramja/incubator-quickstep implements-network-cli

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

    https://github.com/apache/incubator-quickstep/pull/232.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 #232
    
----
commit 28195c008d33328c72e20ded106d3771158cda37
Author: cramja <ma...@gmail.com>
Date:   2017-04-17T22:58:26Z

    Adds network cli interface.

----


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819449
  
    --- Diff: cli/CMakeLists.txt ---
    @@ -15,7 +15,6 @@
     # specific language governing permissions and limitations
     # under the License.
     
    -include_directories(${CMAKE_CURRENT_BINARY_DIR})
    --- End diff --
    
    Why removing 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] incubator-quickstep pull request #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819502
  
    --- Diff: cli/LocalIO.hpp ---
    @@ -0,0 +1,88 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LOCAL_IO_HPP_
    +#define QUICKSTEP_CLI_LOCAL_IO_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "cli/CliConfig.h"
    +#include "cli/IOInterface.hpp"
    +#include "cli/LineReader.hpp"
    +#ifdef QUICKSTEP_USE_LINENOISE
    +#include "cli/LineReaderLineNoise.hpp"
    +typedef quickstep::LineReaderLineNoise LineReaderImpl;
    +#else
    +#include "cli/LineReaderDumb.hpp"
    +typedef quickstep::LineReaderDumb LineReaderImpl;
    +#endif
    +#include "utility/Macros.hpp"
    +
    +namespace quickstep {
    +
    +class LocalIOHandle : public IOHandle {
    + public:
    +  LocalIOHandle() : IOHandle(), command_("") {}
    --- End diff --
    
    Overkill: change to `LocalIOHandle() {}`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819579
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,286 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <cstdio>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/Macros.hpp"
    +#include "utility/MemStream.hpp"
    +#include "utility/ThreadSafeQueue.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
    + * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
    + */
    +class RequestState {
    + public:
    +  explicit RequestState(QueryRequest const *request)
    --- End diff --
    
    Change to `const QueryRequest *requst`.
    
    Also, could we use const reference instead of const pointer here?


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112842641
  
    --- Diff: cli/Flags.cpp ---
    @@ -87,4 +87,25 @@ DEFINE_bool(preload_buffer_pool, false,
                 "accepting queries (should also set --buffer_pool_slots to be "
                 "large enough to accomodate the entire database).");
     
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  int const min = 0, max = 65536;
    +  if (value > min && value < max) {
    +    return true;
    +  }
    +  std::cout << "Invalid value for --" << flagname << ": " << value
    +            << "\nUse ports between " << min << " and "
    +            << max << std::endl;
    +  return false;
    +}
    +
    +DEFINE_int32(cli_network_port, 3000,
    +             "Listens for TCP connections on this port when network mode is enabled. "
    +             "This is only used if the cli is set to use the network mode (--mode=network).");
    +DEFINE_validator(cli_network_port, &ValidatePort);
    +
    +DEFINE_string(cli_network_ip, "localhost",
    --- End diff --
    
    `192.0.0.1` does not make sense. You need to review the network basis.
    
    In `CliClientMain`, you could just [declare the flag](https://github.com/apache/incubator-quickstep/blob/master/cli/QuickstepCli.cpp#L148).


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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

    https://github.com/apache/incubator-quickstep/pull/232#discussion_r112248871
  
    --- Diff: cli/Flags.cpp ---
    @@ -87,4 +87,15 @@ DEFINE_bool(preload_buffer_pool, false,
                 "accepting queries (should also set --buffer_pool_slots to be "
                 "large enough to accomodate the entire database).");
     
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  if (value > 0 && value < 65536) {
    +    return true;
    +  }
    +  std::cout << "Invalid value for --" << flagname << ": " << std::to_string(value) << std::endl;
    --- End diff --
    
    Should the error message also include the valid range of values? That might be helpful. 


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112548112
  
    --- Diff: cli/tests/CMakeLists.txt ---
    @@ -90,3 +101,18 @@ if (ENABLE_DISTRIBUTED)
                             ${GFLAGS_LIB_NAME}
                             ${LIBS})
     endif(ENABLE_DISTRIBUTED)
    +if (ENABLE_SINGLE_NODE_SERVER)
    --- End diff --
    
    Please add an empty line above.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112797758
  
    --- Diff: CMakeLists.txt ---
    @@ -145,6 +145,7 @@ if (ENABLE_VECTOR_PREDICATE_SHORT_CIRCUIT)
       )
     endif()
     
    +option(ENABLE_NETWORK_CLI "Allows use of the network cli" ON)
    --- End diff --
    
    The problem is, if we check in this PR, by default we will build the network-based client for the single-node version, no matter whether the machine has `grpc` installed or not. And typically, it will fail due to no `grpc` found.


---
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 #232: Adds network cli interface.

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

    https://github.com/apache/incubator-quickstep/pull/232
  
    @hbdeshmukh I created a ticket `QUICKSTEP-87`; maybe you can show me how to link to this PR later.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112842721
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,286 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <cstdio>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/Macros.hpp"
    +#include "utility/MemStream.hpp"
    +#include "utility/ThreadSafeQueue.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
    + * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
    + */
    +class RequestState {
    + public:
    +  explicit RequestState(QueryRequest const *request)
    +      : response_ready_(false),
    +        canceled_(false),
    +        request_(request),
    +        condition_(mutex_.createConditionVariable()) {}
    +
    +  /**
    +   * @brief Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   * @param stdout_str Stdout from Quickstep.
    +   * @param stderr_str Stderr from Quickstep.
    +   */
    +  void responseReady(std::string stdout_str, std::string stderr_str) {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    response_message_.set_query_result(stdout_str);
    +    response_message_.set_error_result(stderr_str);
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @brief The producer thread blocks until Quickstep signals that it has finished.
    +   * @note Quickstep may either produce a query response or cancel. Both these actions must notify the condition.
    +   */
    +  void waitForResponse() {
    +    while (!response_ready_)
    +      condition_->await();
    +  }
    +
    +  /**
    +   * @brief Notifies the producer that its request will not be served by Quickstep.
    +   */
    +  void cancel() {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    canceled_ = true;
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @return The producer's query for Quickstep to process.
    +   */
    +  std::string getRequest() const {
    +    return request_->query();
    +  }
    +
    +  /**
    +   * @return The response message from Quickstep.
    +   */
    +  QueryResponse getResponse() const {
    +    DCHECK(response_ready_);
    +    return response_message_;
    +  }
    +
    +  /**
    +   * @return True if query was canceled.
    +   */
    +  bool getCanceled() const {
    +    DCHECK(response_ready_);
    +    return canceled_;
    +  }
    +
    + private:
    +  bool response_ready_;
    +  bool canceled_;
    +  QueryRequest const * request_;
    +  QueryResponse response_message_;
    +  Mutex mutex_;
    +  ConditionVariable *condition_;  // note: owned by the mutex.
    +
    +  DISALLOW_COPY_AND_ASSIGN(RequestState);
    +};
    +
    +}  // namespace networkio_internal
    +
    +using networkio_internal::RequestState;
    +
    +/**
    + * @brief Contains the callback methods which the gRPC service defines.
    + * When a request is made of gRPC, a gRPC worker thread is spun up and enters one of the callback methods
    + * (eg SendQuery). This thread keeps the network connection open while Quickstep processes the query. Concurrency
    + * control between the gRPC worker thread, and the Quickstep thread is controlled by a RequestState object which is
    + * created for each interaction.
    + */
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +      : running_(true) {}
    +
    +  /**
    +   * @brief Handles gRPC request.
    +   * Sets the buffer in the RequestState, places the request on a queue, then waits for a response. The response shall
    +   * be triggered by a
    +   */
    +  Status SendQuery(grpc::ServerContext *context,
    +                   const QueryRequest *request,
    +                   QueryResponse *response) override {
    +    std::unique_ptr<RequestState> request_state;
    +    // Check to see if the gRPC service has been shutdown.
    +    {
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      if (!running_) {
    +        return Status::CANCELLED;
    +      }
    +      // While we have a service lock, we add to the Queue. Note that we keep the service lock to protect ourselves from
    +      // a race condition in the kill() method.
    +      request_state.reset(new RequestState(request));
    +
    +      // Pushing to the queue will notify consumers.
    +      request_queue_.push(request_state.get());
    +    }
    +    DCHECK(request_state);
    +
    +    // We have pushed to the request queue, so all there is to do now is wait for Quickstep to process the request.
    +    request_state->waitForResponse();
    +    if (request_state->getCanceled()) {
    +      return Status::CANCELLED;
    +    }
    +    *response = request_state->getResponse();
    +    return Status::OK;
    +  }
    +
    +  /**
    +   * @brief The consumer thread waits for a request to materialize.
    +   * @return A non-owned RequestState.
    +   */
    +  RequestState* waitForRequest() {
    +    return request_queue_.popOne();
    +  }
    +
    +  /**
    +   * @brief Stops accepting further requests and cancels all pending requests.
    +   */
    +  void kill() {
    +    {
    +      // This action guarantees that no further requests are added to the queue.
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      running_ = false;
    +    }
    +    // Go through each pending request, and cancel them.
    +    while (!request_queue_.empty()) {
    +      request_queue_.popOne()->cancel();
    +    }
    +  }
    +
    + private:
    +  Mutex service_mtx_;
    +  bool running_;
    +  ThreadSafeQueue<RequestState*> request_queue_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkCliServiceImpl);
    +};
    +
    +class NetworkIOHandle : public IOHandle {
    + public:
    +  explicit NetworkIOHandle(RequestState* state)
    +      : request_state_(state) {}
    +
    +  ~NetworkIOHandle() override {
    +    // All the commands from the last network interaction have completed, return our response.
    +    // This signals to the producer thread that the interaction is complete.
    +    request_state_->responseReady(out_stream_.str(), err_stream_.str());
    +  }
    +
    +  FILE* out() override {
    +    return out_stream_.file();
    +  }
    +
    +  FILE* err() override {
    +    return err_stream_.file();
    +  }
    +
    +  std::string getCommand() override {
    +    return request_state_->getRequest();
    +  }
    +
    + private:
    +  MemStream out_stream_, err_stream_;
    +  RequestState* request_state_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkIOHandle);
    +};
    +
    +/**
    + * A network interface that uses gRPC to accept commands.
    + */
    +class NetworkIO : public IOInterface {
    + public:
    +  NetworkIO() {
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(GetAddress(), grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    LOG(INFO) << "Listening on " << GetAddress();
    +  }
    +
    +  ~NetworkIO() {
    +    service_.kill();
    +    server_->Shutdown();
    +    server_->Wait();
    +  }
    +
    +  IOHandle* getNextIOHandle() override {
    +    return new NetworkIOHandle(service_.waitForRequest());
    +  }
    +
    +  /**
    +   * @return The underlying service which interacts with gRPC.
    +   */
    +  NetworkCliServiceImpl& getService() {
    --- End diff --
    
    If it is only used by a test, move this method to private, and add that test as a friend.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819652
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,286 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <cstdio>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/Macros.hpp"
    +#include "utility/MemStream.hpp"
    +#include "utility/ThreadSafeQueue.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
    + * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
    + */
    +class RequestState {
    + public:
    +  explicit RequestState(QueryRequest const *request)
    +      : response_ready_(false),
    +        canceled_(false),
    +        request_(request),
    +        condition_(mutex_.createConditionVariable()) {}
    +
    +  /**
    +   * @brief Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   * @param stdout_str Stdout from Quickstep.
    +   * @param stderr_str Stderr from Quickstep.
    +   */
    +  void responseReady(std::string stdout_str, std::string stderr_str) {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    response_message_.set_query_result(stdout_str);
    +    response_message_.set_error_result(stderr_str);
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @brief The producer thread blocks until Quickstep signals that it has finished.
    +   * @note Quickstep may either produce a query response or cancel. Both these actions must notify the condition.
    +   */
    +  void waitForResponse() {
    +    while (!response_ready_)
    +      condition_->await();
    +  }
    +
    +  /**
    +   * @brief Notifies the producer that its request will not be served by Quickstep.
    +   */
    +  void cancel() {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    canceled_ = true;
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @return The producer's query for Quickstep to process.
    +   */
    +  std::string getRequest() const {
    +    return request_->query();
    +  }
    +
    +  /**
    +   * @return The response message from Quickstep.
    +   */
    +  QueryResponse getResponse() const {
    +    DCHECK(response_ready_);
    +    return response_message_;
    +  }
    +
    +  /**
    +   * @return True if query was canceled.
    +   */
    +  bool getCanceled() const {
    +    DCHECK(response_ready_);
    +    return canceled_;
    +  }
    +
    + private:
    +  bool response_ready_;
    +  bool canceled_;
    +  QueryRequest const * request_;
    +  QueryResponse response_message_;
    +  Mutex mutex_;
    +  ConditionVariable *condition_;  // note: owned by the mutex.
    +
    +  DISALLOW_COPY_AND_ASSIGN(RequestState);
    +};
    +
    +}  // namespace networkio_internal
    +
    +using networkio_internal::RequestState;
    +
    +/**
    + * @brief Contains the callback methods which the gRPC service defines.
    + * When a request is made of gRPC, a gRPC worker thread is spun up and enters one of the callback methods
    + * (eg SendQuery). This thread keeps the network connection open while Quickstep processes the query. Concurrency
    + * control between the gRPC worker thread, and the Quickstep thread is controlled by a RequestState object which is
    + * created for each interaction.
    + */
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +      : running_(true) {}
    +
    +  /**
    +   * @brief Handles gRPC request.
    +   * Sets the buffer in the RequestState, places the request on a queue, then waits for a response. The response shall
    +   * be triggered by a
    +   */
    +  Status SendQuery(grpc::ServerContext *context,
    +                   const QueryRequest *request,
    +                   QueryResponse *response) override {
    +    std::unique_ptr<RequestState> request_state;
    +    // Check to see if the gRPC service has been shutdown.
    +    {
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      if (!running_) {
    +        return Status::CANCELLED;
    +      }
    +      // While we have a service lock, we add to the Queue. Note that we keep the service lock to protect ourselves from
    +      // a race condition in the kill() method.
    +      request_state.reset(new RequestState(request));
    +
    +      // Pushing to the queue will notify consumers.
    +      request_queue_.push(request_state.get());
    +    }
    +    DCHECK(request_state);
    +
    +    // We have pushed to the request queue, so all there is to do now is wait for Quickstep to process the request.
    +    request_state->waitForResponse();
    +    if (request_state->getCanceled()) {
    +      return Status::CANCELLED;
    +    }
    +    *response = request_state->getResponse();
    +    return Status::OK;
    +  }
    +
    +  /**
    +   * @brief The consumer thread waits for a request to materialize.
    +   * @return A non-owned RequestState.
    +   */
    +  RequestState* waitForRequest() {
    +    return request_queue_.popOne();
    +  }
    +
    +  /**
    +   * @brief Stops accepting further requests and cancels all pending requests.
    +   */
    +  void kill() {
    +    {
    +      // This action guarantees that no further requests are added to the queue.
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      running_ = false;
    +    }
    +    // Go through each pending request, and cancel them.
    +    while (!request_queue_.empty()) {
    +      request_queue_.popOne()->cancel();
    +    }
    +  }
    +
    + private:
    +  Mutex service_mtx_;
    +  bool running_;
    +  ThreadSafeQueue<RequestState*> request_queue_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkCliServiceImpl);
    +};
    +
    +class NetworkIOHandle : public IOHandle {
    + public:
    +  explicit NetworkIOHandle(RequestState* state)
    +      : request_state_(state) {}
    +
    +  ~NetworkIOHandle() override {
    +    // All the commands from the last network interaction have completed, return our response.
    +    // This signals to the producer thread that the interaction is complete.
    +    request_state_->responseReady(out_stream_.str(), err_stream_.str());
    +  }
    +
    +  FILE* out() override {
    +    return out_stream_.file();
    +  }
    +
    +  FILE* err() override {
    +    return err_stream_.file();
    +  }
    +
    +  std::string getCommand() override {
    +    return request_state_->getRequest();
    +  }
    +
    + private:
    +  MemStream out_stream_, err_stream_;
    +  RequestState* request_state_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkIOHandle);
    +};
    +
    +/**
    + * A network interface that uses gRPC to accept commands.
    + */
    +class NetworkIO : public IOInterface {
    + public:
    +  NetworkIO() {
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(GetAddress(), grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    LOG(INFO) << "Listening on " << GetAddress();
    +  }
    +
    +  ~NetworkIO() {
    +    service_.kill();
    +    server_->Shutdown();
    +    server_->Wait();
    +  }
    +
    +  IOHandle* getNextIOHandle() override {
    +    return new NetworkIOHandle(service_.waitForRequest());
    +  }
    +
    +  /**
    +   * @return The underlying service which interacts with gRPC.
    +   */
    +  NetworkCliServiceImpl& getService() {
    --- End diff --
    
    We should never return a reference. Instead, a const reference, or a 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 issue #232: QUICKSTEP-87 Adds network cli interface.

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

    https://github.com/apache/incubator-quickstep/pull/232
  
    @zuyu Thank you for review. I addressed the minor stuff and will look into Async when I have more time this coming week.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112406932
  
    --- Diff: cli/LocalIO.hpp ---
    @@ -0,0 +1,74 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LOCAL_IO_HPP_
    +#define QUICKSTEP_CLI_LOCAL_IO_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "cli/CliConfig.h"
    +#include "cli/IOInterface.hpp"
    +#include "cli/LineReader.hpp"
    +
    +#ifdef QUICKSTEP_USE_LINENOISE
    +#include "cli/LineReaderLineNoise.hpp"
    +typedef quickstep::LineReaderLineNoise LineReaderImpl;
    +#else
    +#include "cli/LineReaderDumb.hpp"
    +typedef quickstep::LineReaderDumb LineReaderImpl;
    +#endif
    +
    +namespace quickstep {
    +
    +class LocalIO : public IOInterface {
    + public:
    +  LocalIO() : IOInterface(), line_reader_("quickstep> ", "      ...> ") {}
    --- End diff --
    
    Suggest to refactor as following:
    
    ```
      ... 
      line_reader("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 #232: QUICKSTEP-87 Adds network cli interface.

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

    https://github.com/apache/incubator-quickstep/pull/232
  
    I should also mention that @hbdeshmukh and I had a discussion yesterday about refactoring the CLI main method. I was unhappy with how cluttered seeming it has become, and this PR only makes it slightly more cluttered. I came up with a solution which I have been working in a WIP branch. It uses RIIA to close down any IO-related state (eg network connections) by creating an `IOHandle` for every cli user interaction. Because I think this change is a big improvement over, and complimentary to the change which is present here, I will submit it as a commit on top of this commit in the 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 #232: QUICKSTEP-87 Adds network cli interface.

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

    https://github.com/apache/incubator-quickstep/pull/232
  
    I mean, if this class is always a base class, move the constructor in
    protected session.
    On Mon, Apr 24, 2017 at 12:58 PM Marc S <no...@github.com> wrote:
    
    > *@cramja* commented on this pull request.
    > ------------------------------
    >
    > In cli/IOInterface.hpp
    > <https://github.com/apache/incubator-quickstep/pull/232#discussion_r113041261>
    > :
    >
    > > + * specific language governing permissions and limitations
    > + * under the License.
    > + **/
    > +
    > +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    > +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    > +
    > +#include <string>
    > +
    > +/**
    > + * Virtual base defines a generic, file-based interface around IO.
    > + */
    > +class IOInterface {
    > + public:
    > +  IOInterface() {}
    > +
    >
    > I removed the constructor but ran into a compiler error:
    >
    > /Users/cramja/workspace/quickstep/incubator-quickstep/cli/NetworkIO.cpp:50:12: error: constructor for 'quickstep::NetworkIO' must explicitly initialize the base class 'quickstep::IOInterface' which does not have a default constructor
    > NetworkIO::NetworkIO() {
    >            ^
    > /Users/cramja/workspace/quickstep/incubator-quickstep/cli/IOInterface.hpp:62:7: note: 'quickstep::IOInterface' declared here
    > class IOInterface {
    >       ^
    >
    > On doing the suggestion in the output, that too gave a compiler error.
    >
    > \u2014
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub
    > <https://github.com/apache/incubator-quickstep/pull/232#discussion_r113041261>,
    > or mute the thread
    > <https://github.com/notifications/unsubscribe-auth/ACJDxGtWVm0ZhTjvMb9w0_2D7-hWuOdeks5rzP7OgaJpZM4M_q9a>
    > .
    >



---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112409077
  
    --- Diff: cli/NetworkCliClient.hpp ---
    @@ -0,0 +1,91 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +#define QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +/**
    + * A simple wrapper class used to do CLI interactions with QuickstepCLI via the gRPC interface.
    + */
    +class NetworkCliClient {
    + public:
    +  explicit NetworkCliClient(std::shared_ptr<Channel> const &channel)
    +    : stub_(NetworkCli::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 Query(const std::string &user_query) {
    +    QueryRequest request;
    +    request.set_query(user_query);
    +    QueryResponse response;
    +
    +    Status status = SendQuery(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";
    +    }
    +  }
    +
    +  Status SendQuery(QueryRequest request, QueryResponse* response) {
    --- End diff --
    
    Change to `const QueryRequest &request, QueryResponse *response`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112686604
  
    --- Diff: CMakeLists.txt ---
    @@ -145,6 +145,7 @@ if (ENABLE_VECTOR_PREDICATE_SHORT_CIRCUIT)
       )
     endif()
     
    +option(ENABLE_NETWORK_CLI "Allows use of the network cli" ON)
    --- End diff --
    
    Yes, I remembered. However, this was only because of Travis and I'd therefore prefer to keep this available by default. So I disabled it in the `.travis.yml` file.


---
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 #232: QUICKSTEP-87 Adds network cli interface.

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

    https://github.com/apache/incubator-quickstep/pull/232
  
    Thanks @hbdeshmukh , fixed validate_cmakelists issue.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112825369
  
    --- Diff: cli/CMakeLists.txt ---
    @@ -176,5 +249,14 @@ target_link_libraries(quickstep_cli
                           quickstep_cli_DropRelation
                           quickstep_cli_Flags
                           quickstep_cli_InputParserUtil
    +                      quickstep_cli_IOInterface
    --- End diff --
    
    Alphabet order: move this line above `quickstep_cli_InputParserUtil`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819541
  
    --- Diff: cli/LocalIO.hpp ---
    @@ -0,0 +1,88 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LOCAL_IO_HPP_
    +#define QUICKSTEP_CLI_LOCAL_IO_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "cli/CliConfig.h"
    +#include "cli/IOInterface.hpp"
    +#include "cli/LineReader.hpp"
    +#ifdef QUICKSTEP_USE_LINENOISE
    +#include "cli/LineReaderLineNoise.hpp"
    +typedef quickstep::LineReaderLineNoise LineReaderImpl;
    +#else
    +#include "cli/LineReaderDumb.hpp"
    +typedef quickstep::LineReaderDumb LineReaderImpl;
    +#endif
    +#include "utility/Macros.hpp"
    +
    +namespace quickstep {
    +
    +class LocalIOHandle : public IOHandle {
    + public:
    +  LocalIOHandle() : IOHandle(), command_("") {}
    +
    +  explicit LocalIOHandle(std::string const & command)
    +      : IOHandle(),
    +        command_(command) {}
    +
    +  ~LocalIOHandle() {}
    +
    +  FILE *out() override {
    +    return stdout;
    +  }
    +
    +  FILE *err() override {
    +    return stderr;
    +  }
    +
    +  std::string getCommand() override {
    +    return command_;
    +  }
    +
    + private:
    +  std::string command_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(LocalIOHandle);
    +};
    +
    +/**
    + * IO class for getting commands from stdin via an interactive line reader.
    + */
    +class LocalIO : public IOInterface {
    + public:
    +  LocalIO() : line_reader_("quickstep> ",
    +                           "      ...> ") {}
    +
    --- End diff --
    
    We need a `override` distructor.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819491
  
    --- Diff: cli/LocalIO.hpp ---
    @@ -0,0 +1,88 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LOCAL_IO_HPP_
    +#define QUICKSTEP_CLI_LOCAL_IO_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "cli/CliConfig.h"
    +#include "cli/IOInterface.hpp"
    +#include "cli/LineReader.hpp"
    +#ifdef QUICKSTEP_USE_LINENOISE
    +#include "cli/LineReaderLineNoise.hpp"
    +typedef quickstep::LineReaderLineNoise LineReaderImpl;
    +#else
    +#include "cli/LineReaderDumb.hpp"
    +typedef quickstep::LineReaderDumb LineReaderImpl;
    +#endif
    +#include "utility/Macros.hpp"
    +
    +namespace quickstep {
    +
    +class LocalIOHandle : public IOHandle {
    + public:
    +  LocalIOHandle() : IOHandle(), command_("") {}
    +
    +  explicit LocalIOHandle(std::string const & command)
    --- End diff --
    
    Change to `const std::string &command`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112410750
  
    --- Diff: cli/Flags.hpp ---
    @@ -45,6 +45,8 @@ DECLARE_string(storage_path);
     
     DECLARE_bool(preload_buffer_pool);
     
    +DECLARE_int32(port);
    --- End diff --
    
    Please move to `cli/NetworkIO.hpp`, and change name to sth like `cli_network_port`, because this flag is only used for network cli, but not shared between the single-node and the distributed 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] incubator-quickstep pull request #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r113040305
  
    --- Diff: CMakeLists.txt ---
    @@ -145,6 +145,7 @@ if (ENABLE_VECTOR_PREDICATE_SHORT_CIRCUIT)
       )
     endif()
     
    +option(ENABLE_NETWORK_CLI "Allows use of the network cli" ON)
    --- End diff --
    
    That makes sense. I will change it.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819487
  
    --- Diff: cli/LocalIO.hpp ---
    @@ -0,0 +1,88 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LOCAL_IO_HPP_
    +#define QUICKSTEP_CLI_LOCAL_IO_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "cli/CliConfig.h"
    +#include "cli/IOInterface.hpp"
    +#include "cli/LineReader.hpp"
    +#ifdef QUICKSTEP_USE_LINENOISE
    +#include "cli/LineReaderLineNoise.hpp"
    +typedef quickstep::LineReaderLineNoise LineReaderImpl;
    +#else
    +#include "cli/LineReaderDumb.hpp"
    +typedef quickstep::LineReaderDumb LineReaderImpl;
    +#endif
    +#include "utility/Macros.hpp"
    +
    +namespace quickstep {
    +
    +class LocalIOHandle : public IOHandle {
    + public:
    +  LocalIOHandle() : IOHandle(), command_("") {}
    +
    +  explicit LocalIOHandle(std::string const & command)
    +      : IOHandle(),
    +        command_(command) {}
    +
    +  ~LocalIOHandle() {}
    --- End diff --
    
    Add `override`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112405717
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,62 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <string>
    +
    +/**
    + * Virtual base defines a generic, file-based interface around IO.
    + */
    +class IOInterface {
    + public:
    +  IOInterface() {}
    +
    --- End diff --
    
    As a base class, we need a `virtual` distructor. On the other hand, the default constructor in Line `30` will always be provided by the compiler.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112547038
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,227 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/MemStream.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer (requestor/requestee)
    + * thread.
    + */
    +class RequestState {
    + public:
    +  RequestState() :
    +    request_ready_(false),
    +    response_ready_(false),
    +    request_buffer_(""),
    +    mutex_(),
    +    condition_() {}
    +
    +  /**
    +   * Notifies waiter that a piece of work has been created and added to the buffer.
    +   * @param to_consume The arguments for the consuming thread.
    +   */
    +  void requestReady(std::string to_consume) {
    +    request_ready_ = true;
    +    response_ready_ = false;
    +    request_buffer_ = to_consume;
    +    condition_.notify_one();
    +  }
    +
    +  /**
    +   * Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   */
    +  void responseReady() {
    +    request_ready_ = false;
    +    response_ready_ = true;
    +    condition_.notify_one();
    +  }
    +
    +  bool request_ready_;
    +  bool response_ready_;
    +  std::string request_buffer_;
    +  QueryResponse response_message_;
    +  std::mutex mutex_;
    +  std::condition_variable condition_;
    +};
    +}  // namespace networkio_internal
    +
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +    : NetworkCli::Service(),
    +      worker_exclusive_mtx_(),
    +      running_(true),
    +      request_state_() { }
    +
    +  /**
    +   * Handles gRPC request. Sets the buffer in the RequestState, notifies the main thread, then waits for a response.
    +   * @param context
    +   * @param request
    +   * @param response
    +   * @return
    +   */
    +  Status SendQuery(grpc::ServerContext *context,
    +                   const QueryRequest *request,
    +                   QueryResponse *response) override {
    +    auto service_ex_lock = enter();
    +    if (!service_ex_lock) {
    +      return Status::CANCELLED;
    +    }
    +
    +    // Service thread sets a notification for the main thread.
    +    // Because of service thread exclusivity, we have rights to this data structure.
    +    request_state_.requestReady(request->query());
    +
    +    // Service thread - main thread critical section:
    +    // Service thread waits for the buffered message response from the main thread. The main thread will set
    +    // consumer_ready_ when it is finished and released its exclusive lock on the communication data structure.
    +    std::unique_lock<std::mutex> lock(request_state_.mutex_);
    +    while (!request_state_.response_ready_)
    +      request_state_.condition_.wait(lock);
    +
    +    *response = request_state_.response_message_;
    +
    +    return Status::OK;
    +  }
    +
    +  void kill() {
    +    running_ = false;
    +  }
    +
    +  networkio_internal::RequestState& getRequestState() {
    +    return request_state_;
    +  }
    +
    + private:
    +  /**
    +   * When a worker enters, it gains exclusive access to the main thread. That is, no other worker of this service
    +   * is allowed to interact with main thread.
    +   * @return A lock which grants the worker mutual exclusion.
    +   */
    +  std::unique_lock<std::mutex> enter() {
    +    std::unique_lock<std::mutex> lock(worker_exclusive_mtx_);
    +    if (!running_)
    +      lock.unlock();
    +    return lock;
    +  }
    +
    +  std::mutex worker_exclusive_mtx_;
    +  bool running_;
    +  networkio_internal::RequestState request_state_;
    +};
    +
    +class NetworkIO : public IOInterface {
    + public:
    +  NetworkIO()
    +      : IOInterface(),
    +        out_stream_(),
    +        err_stream_(),
    +        server_(nullptr),
    +        service_() {
    --- End diff --
    
    No needs for explicitly calling default constructors.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112787479
  
    --- Diff: cli/NetworkCliClient.hpp ---
    @@ -0,0 +1,91 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +#define QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +/**
    + * A simple wrapper class used to do CLI interactions with QuickstepCLI via the gRPC interface.
    + */
    +class NetworkCliClient {
    + public:
    +  explicit NetworkCliClient(std::shared_ptr<Channel> const &channel)
    --- End diff --
    
    FYI the style guide says this is a matter of preference
    
    > That said, while we encourage putting const first, we do not require it. But be consistent with the code around you!
    
    My opinion is that it should always follow the target object because then it will always 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] incubator-quickstep pull request #232: QUICKSTEP-87 Adds network cli interfa...

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

    https://github.com/apache/incubator-quickstep/pull/232#discussion_r112249915
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,61 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <string>
    +
    +/**
    + * Virtual base defines a generic, file-based interface around IO.
    + */
    +class IOInterface {
    + public:
    +  IOInterface() {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  virtual FILE* out() = 0;
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  virtual FILE* err() = 0;
    +
    +  /**
    +   * Requests a complete SQL command. This call may block until user input is given.
    +   * @note When the command is complete, commandComplete() should be called so that certain implementations can clean
    +   *    up state related to sending the command.
    +   * @return A string containing a quickstep command.
    +   */
    +  virtual std::string nextCommand() = 0;
    +
    +  /**
    +   * Notifies the IO system that the previously acquired command is complete.
    +   */
    +  virtual void commandComplete() {}
    --- End diff --
    
    How about ``notifyCommandComplete`` as a function name? In general, I try to name functions as a verb, so that their name can express what they are doing. 


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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

    https://github.com/apache/incubator-quickstep/pull/232#discussion_r112250621
  
    --- Diff: cli/LineReaderBuffered.hpp ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +#define QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +
    +#include "cli/LineReader.hpp"
    +
    +namespace quickstep {
    +
    +class LineReaderBuffered : public LineReader {
    + public:
    +  /**
    +   * A line reader which accepts a string buffer.
    +   * This does no IO and ignores any prompts given to the constructor.
    +   * @param default_prompt Ignored.
    +   * @param continue_prompt Ignored.
    --- End diff --
    
    I don't quite understand the doxygen comments for the constructor arguments. Can you please elaborate? 


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112826572
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,286 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <cstdio>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/Macros.hpp"
    +#include "utility/MemStream.hpp"
    +#include "utility/ThreadSafeQueue.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
    + * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
    + */
    +class RequestState {
    + public:
    +  explicit RequestState(QueryRequest const *request)
    +      : response_ready_(false),
    +        canceled_(false),
    +        request_(request),
    +        condition_(mutex_.createConditionVariable()) {}
    +
    +  /**
    +   * @brief Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   * @param stdout_str Stdout from Quickstep.
    +   * @param stderr_str Stderr from Quickstep.
    +   */
    +  void responseReady(std::string stdout_str, std::string stderr_str) {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    response_message_.set_query_result(stdout_str);
    +    response_message_.set_error_result(stderr_str);
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @brief The producer thread blocks until Quickstep signals that it has finished.
    +   * @note Quickstep may either produce a query response or cancel. Both these actions must notify the condition.
    +   */
    +  void waitForResponse() {
    +    while (!response_ready_)
    +      condition_->await();
    +  }
    +
    +  /**
    +   * @brief Notifies the producer that its request will not be served by Quickstep.
    +   */
    +  void cancel() {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    canceled_ = true;
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @return The producer's query for Quickstep to process.
    +   */
    +  std::string getRequest() const {
    +    return request_->query();
    +  }
    +
    +  /**
    +   * @return The response message from Quickstep.
    +   */
    +  QueryResponse getResponse() const {
    +    DCHECK(response_ready_);
    +    return response_message_;
    +  }
    +
    +  /**
    +   * @return True if query was canceled.
    +   */
    +  bool getCanceled() const {
    +    DCHECK(response_ready_);
    +    return canceled_;
    +  }
    +
    + private:
    +  bool response_ready_;
    +  bool canceled_;
    +  QueryRequest const * request_;
    +  QueryResponse response_message_;
    +  Mutex mutex_;
    +  ConditionVariable *condition_;  // note: owned by the mutex.
    +
    +  DISALLOW_COPY_AND_ASSIGN(RequestState);
    +};
    +
    +}  // namespace networkio_internal
    +
    +using networkio_internal::RequestState;
    +
    +/**
    + * @brief Contains the callback methods which the gRPC service defines.
    + * When a request is made of gRPC, a gRPC worker thread is spun up and enters one of the callback methods
    + * (eg SendQuery). This thread keeps the network connection open while Quickstep processes the query. Concurrency
    + * control between the gRPC worker thread, and the Quickstep thread is controlled by a RequestState object which is
    + * created for each interaction.
    + */
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +      : running_(true) {}
    +
    +  /**
    +   * @brief Handles gRPC request.
    +   * Sets the buffer in the RequestState, places the request on a queue, then waits for a response. The response shall
    +   * be triggered by a
    +   */
    +  Status SendQuery(grpc::ServerContext *context,
    +                   const QueryRequest *request,
    +                   QueryResponse *response) override {
    +    std::unique_ptr<RequestState> request_state;
    +    // Check to see if the gRPC service has been shutdown.
    +    {
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      if (!running_) {
    +        return Status::CANCELLED;
    +      }
    +      // While we have a service lock, we add to the Queue. Note that we keep the service lock to protect ourselves from
    +      // a race condition in the kill() method.
    +      request_state.reset(new RequestState(request));
    +
    +      // Pushing to the queue will notify consumers.
    +      request_queue_.push(request_state.get());
    +    }
    +    DCHECK(request_state);
    +
    +    // We have pushed to the request queue, so all there is to do now is wait for Quickstep to process the request.
    +    request_state->waitForResponse();
    +    if (request_state->getCanceled()) {
    +      return Status::CANCELLED;
    +    }
    +    *response = request_state->getResponse();
    +    return Status::OK;
    +  }
    +
    +  /**
    +   * @brief The consumer thread waits for a request to materialize.
    +   * @return A non-owned RequestState.
    +   */
    +  RequestState* waitForRequest() {
    +    return request_queue_.popOne();
    +  }
    +
    +  /**
    +   * @brief Stops accepting further requests and cancels all pending requests.
    +   */
    +  void kill() {
    +    {
    +      // This action guarantees that no further requests are added to the queue.
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      running_ = false;
    +    }
    +    // Go through each pending request, and cancel them.
    +    while (!request_queue_.empty()) {
    +      request_queue_.popOne()->cancel();
    +    }
    +  }
    +
    + private:
    +  Mutex service_mtx_;
    +  bool running_;
    +  ThreadSafeQueue<RequestState*> request_queue_;
    --- End diff --
    
    I suggest to follow [the official async server example](https://github.com/grpc/grpc/blob/master/examples/cpp/helloworld/greeter_async_server.cc#L54) so that we don't need mutexes and locks for multiple concurrent requests.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112550617
  
    --- Diff: cli/tests/NetworkIO_unittest.cpp ---
    @@ -0,0 +1,182 @@
    +/**
    + * 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 <cstddef>
    +#include <cstdio>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +#include "cli/NetworkIO.hpp"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "threading/Thread.hpp"
    +
    +#include "glog/logging.h"
    +#include "gtest/gtest.h"
    +
    +using std::unique_ptr;
    +
    +namespace quickstep {
    +
    +static std::string const kQueryRequest = "O Captain! My Captain!";
    +static std::string const kQueryResponse = "Our fearful trip is done,";
    +
    +/**
    + * Contains a server instance for testing.
    + */
    +class TestNetworkIO {
    + public:
    +  TestNetworkIO()
    +      : service_(),
    +        server_address_("localhost:" + std::to_string(FLAGS_port)),
    +        server_(nullptr) {
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(server_address_, grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    CHECK(server_) << "Failed to start server";
    +    LOG(INFO) << "TestSingleNodeServer listening on " << server_address_;
    +  }
    +
    +  // Gets a message from the Service worker.
    +  std::string getSentMessage() {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    +    std::unique_lock<std::mutex> lock(requestState.mutex_);
    +    while (!requestState.request_ready_)
    +      requestState.condition_.wait(lock);
    +
    +    EXPECT_EQ(requestState.request_ready_, true);
    +    EXPECT_EQ(requestState.response_ready_, false);
    +
    +    return requestState.request_buffer_;
    +  }
    +
    +  // Sets the response message of the Service worker. Alerts it that the request is ready.
    +  void setResponse(std::string response) {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    +    requestState.response_message_.set_query_result(response);
    +
    +    EXPECT_EQ(requestState.response_ready_, false);
    +
    +    requestState.responseReady();
    +  }
    +
    +  NetworkCliServiceImpl& getService() {
    +    return service_;
    +  }
    +
    +  ~TestNetworkIO() {
    +    service_.kill();
    +    server_->Shutdown();
    +    server_->Wait();
    +  }
    +
    + private:
    +  NetworkCliServiceImpl service_;
    +  std::string server_address_;
    +  std::unique_ptr<grpc::Server> server_;
    +};
    +
    +/**
    + * Tests that killing the service will cancel requests.
    + */
    +TEST(NetworkIOTest, TestShutdown) {
    +  TestNetworkIO server;
    +
    +  server.getService().kill();
    +
    +  NetworkCliClient client(
    +    grpc::CreateChannel("localhost:" + std::to_string(FLAGS_port),
    +                        grpc::InsecureChannelCredentials()));
    +  QueryRequest request;
    +  request.set_query(kQueryRequest);
    +  QueryResponse response;
    +  Status status = client.SendQuery(request, &response);
    +  ASSERT_EQ(status.error_code(), grpc::CANCELLED);
    +}
    +
    +/**
    + * Tests a simple call and response to the Service.
    + */
    +TEST(NetworkIOTest, TestNetworkIOCommandInteraction) {
    +  NetworkIO networkIO;
    +  std::string const query_stmt = kQueryRequest + ";" + kQueryRequest;
    +
    +  // This thread will handle the response from the client in a similar way as the quickstep cli will.
    +  std::thread server_handler([&networkIO, &query_stmt]() {
    +    std::string command = networkIO.getNextCommand();
    +    EXPECT_EQ(command, query_stmt);
    +
    +    // Set some output for the main test thread, return.
    +    fprintf(networkIO.out(), "%s", kQueryResponse.c_str());
    +    networkIO.notifyCommandComplete();
    +  });
    +
    +  NetworkCliClient client(
    +    grpc::CreateChannel("localhost:" + std::to_string(FLAGS_port),
    +                        grpc::InsecureChannelCredentials()));
    +  QueryRequest request;
    +  request.set_query(query_stmt);
    +  QueryResponse response;
    +  Status status = client.SendQuery(request, &response);
    +  ASSERT_TRUE(status.ok());
    +  EXPECT_EQ(response.query_result(), kQueryResponse);
    +  EXPECT_EQ(response.error_result(), std::string(""));
    +
    +  server_handler.join();
    +}
    +
    +TEST(NetworkIOTest, TestLineReaderBuffered) {
    +  // The buffered line reader is used exclusively by the NetworkIO's client.
    +  LineReaderBuffered linereader;
    +  EXPECT_TRUE(linereader.bufferEmpty());
    +
    +  std::string stmt = "select * from foo;";
    +  std::string stmts = stmt + "select 1; select 2; quit;";
    --- End diff --
    
    Suggest to change to `single_statement` and `multiple_statements`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819627
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,286 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <cstdio>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/Macros.hpp"
    +#include "utility/MemStream.hpp"
    +#include "utility/ThreadSafeQueue.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
    + * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
    + */
    +class RequestState {
    + public:
    +  explicit RequestState(QueryRequest const *request)
    +      : response_ready_(false),
    +        canceled_(false),
    +        request_(request),
    +        condition_(mutex_.createConditionVariable()) {}
    +
    +  /**
    +   * @brief Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   * @param stdout_str Stdout from Quickstep.
    +   * @param stderr_str Stderr from Quickstep.
    +   */
    +  void responseReady(std::string stdout_str, std::string stderr_str) {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    response_message_.set_query_result(stdout_str);
    +    response_message_.set_error_result(stderr_str);
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @brief The producer thread blocks until Quickstep signals that it has finished.
    +   * @note Quickstep may either produce a query response or cancel. Both these actions must notify the condition.
    +   */
    +  void waitForResponse() {
    +    while (!response_ready_)
    +      condition_->await();
    +  }
    +
    +  /**
    +   * @brief Notifies the producer that its request will not be served by Quickstep.
    +   */
    +  void cancel() {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    canceled_ = true;
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @return The producer's query for Quickstep to process.
    +   */
    +  std::string getRequest() const {
    +    return request_->query();
    +  }
    +
    +  /**
    +   * @return The response message from Quickstep.
    +   */
    +  QueryResponse getResponse() const {
    +    DCHECK(response_ready_);
    +    return response_message_;
    +  }
    +
    +  /**
    +   * @return True if query was canceled.
    +   */
    +  bool getCanceled() const {
    +    DCHECK(response_ready_);
    +    return canceled_;
    +  }
    +
    + private:
    +  bool response_ready_;
    +  bool canceled_;
    +  QueryRequest const * request_;
    +  QueryResponse response_message_;
    +  Mutex mutex_;
    +  ConditionVariable *condition_;  // note: owned by the mutex.
    +
    +  DISALLOW_COPY_AND_ASSIGN(RequestState);
    +};
    +
    +}  // namespace networkio_internal
    +
    +using networkio_internal::RequestState;
    +
    +/**
    + * @brief Contains the callback methods which the gRPC service defines.
    + * When a request is made of gRPC, a gRPC worker thread is spun up and enters one of the callback methods
    + * (eg SendQuery). This thread keeps the network connection open while Quickstep processes the query. Concurrency
    + * control between the gRPC worker thread, and the Quickstep thread is controlled by a RequestState object which is
    + * created for each interaction.
    + */
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +      : running_(true) {}
    +
    +  /**
    +   * @brief Handles gRPC request.
    +   * Sets the buffer in the RequestState, places the request on a queue, then waits for a response. The response shall
    +   * be triggered by a
    +   */
    +  Status SendQuery(grpc::ServerContext *context,
    +                   const QueryRequest *request,
    +                   QueryResponse *response) override {
    +    std::unique_ptr<RequestState> request_state;
    +    // Check to see if the gRPC service has been shutdown.
    +    {
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      if (!running_) {
    +        return Status::CANCELLED;
    +      }
    +      // While we have a service lock, we add to the Queue. Note that we keep the service lock to protect ourselves from
    +      // a race condition in the kill() method.
    +      request_state.reset(new RequestState(request));
    +
    +      // Pushing to the queue will notify consumers.
    +      request_queue_.push(request_state.get());
    +    }
    +    DCHECK(request_state);
    +
    +    // We have pushed to the request queue, so all there is to do now is wait for Quickstep to process the request.
    +    request_state->waitForResponse();
    +    if (request_state->getCanceled()) {
    +      return Status::CANCELLED;
    +    }
    +    *response = request_state->getResponse();
    +    return Status::OK;
    +  }
    +
    +  /**
    +   * @brief The consumer thread waits for a request to materialize.
    +   * @return A non-owned RequestState.
    +   */
    +  RequestState* waitForRequest() {
    +    return request_queue_.popOne();
    +  }
    +
    +  /**
    +   * @brief Stops accepting further requests and cancels all pending requests.
    +   */
    +  void kill() {
    +    {
    +      // This action guarantees that no further requests are added to the queue.
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      running_ = false;
    +    }
    +    // Go through each pending request, and cancel them.
    +    while (!request_queue_.empty()) {
    +      request_queue_.popOne()->cancel();
    +    }
    +  }
    +
    + private:
    +  Mutex service_mtx_;
    +  bool running_;
    +  ThreadSafeQueue<RequestState*> request_queue_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkCliServiceImpl);
    +};
    +
    +class NetworkIOHandle : public IOHandle {
    --- End diff --
    
    Mark `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] incubator-quickstep pull request #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112413621
  
    --- Diff: cli/tests/CMakeLists.txt ---
    @@ -32,6 +32,17 @@ if (ENABLE_DISTRIBUTED)
                      "${PROJECT_SOURCE_DIR}/utility/textbased_test/TextBasedTest.hpp")
     endif(ENABLE_DISTRIBUTED)
     
    +if (ENABLE_SINGLE_NODE_SERVER)
    --- End diff --
    
    Which would set this variable?


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112404137
  
    --- Diff: cli/CMakeLists.txt ---
    @@ -150,6 +189,34 @@ if(USE_LINENOISE)
     else()
       target_link_libraries(quickstep_cli_LineReader
                             quickstep_utility_Macros)
    +
    --- End diff --
    
    Remove this empty line.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112403533
  
    --- Diff: CMakeLists.txt ---
    @@ -145,6 +145,7 @@ if (ENABLE_VECTOR_PREDICATE_SHORT_CIRCUIT)
       )
     endif()
     
    +option(ENABLE_NETWORK_CLI "Allows use of the network cli" ON)
    --- End diff --
    
    As [stated in the previous PR](https://github.com/apache/incubator-quickstep/pull/230#discussion_r110523774), this should be `OFF` by default, as it requires `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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112549415
  
    --- Diff: cli/tests/NetworkIO_unittest.cpp ---
    @@ -0,0 +1,182 @@
    +/**
    + * 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 <cstddef>
    +#include <cstdio>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +#include "cli/NetworkIO.hpp"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "threading/Thread.hpp"
    +
    +#include "glog/logging.h"
    +#include "gtest/gtest.h"
    +
    +using std::unique_ptr;
    +
    +namespace quickstep {
    +
    +static std::string const kQueryRequest = "O Captain! My Captain!";
    +static std::string const kQueryResponse = "Our fearful trip is done,";
    +
    +/**
    + * Contains a server instance for testing.
    + */
    +class TestNetworkIO {
    + public:
    +  TestNetworkIO()
    +      : service_(),
    +        server_address_("localhost:" + std::to_string(FLAGS_port)),
    +        server_(nullptr) {
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(server_address_, grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    CHECK(server_) << "Failed to start server";
    +    LOG(INFO) << "TestSingleNodeServer listening on " << server_address_;
    +  }
    +
    +  // Gets a message from the Service worker.
    +  std::string getSentMessage() {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    +    std::unique_lock<std::mutex> lock(requestState.mutex_);
    +    while (!requestState.request_ready_)
    +      requestState.condition_.wait(lock);
    +
    +    EXPECT_EQ(requestState.request_ready_, true);
    +    EXPECT_EQ(requestState.response_ready_, false);
    +
    +    return requestState.request_buffer_;
    +  }
    +
    +  // Sets the response message of the Service worker. Alerts it that the request is ready.
    +  void setResponse(std::string response) {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    +    requestState.response_message_.set_query_result(response);
    +
    +    EXPECT_EQ(requestState.response_ready_, false);
    +
    +    requestState.responseReady();
    +  }
    +
    +  NetworkCliServiceImpl& getService() {
    +    return service_;
    +  }
    +
    +  ~TestNetworkIO() {
    +    service_.kill();
    +    server_->Shutdown();
    +    server_->Wait();
    +  }
    +
    + private:
    +  NetworkCliServiceImpl service_;
    +  std::string server_address_;
    +  std::unique_ptr<grpc::Server> server_;
    +};
    +
    +/**
    + * Tests that killing the service will cancel requests.
    + */
    +TEST(NetworkIOTest, TestShutdown) {
    +  TestNetworkIO server;
    +
    +  server.getService().kill();
    +
    +  NetworkCliClient client(
    +    grpc::CreateChannel("localhost:" + std::to_string(FLAGS_port),
    +                        grpc::InsecureChannelCredentials()));
    +  QueryRequest request;
    +  request.set_query(kQueryRequest);
    +  QueryResponse response;
    +  Status status = client.SendQuery(request, &response);
    +  ASSERT_EQ(status.error_code(), grpc::CANCELLED);
    +}
    +
    +/**
    + * Tests a simple call and response to the Service.
    + */
    +TEST(NetworkIOTest, TestNetworkIOCommandInteraction) {
    +  NetworkIO networkIO;
    +  std::string const query_stmt = kQueryRequest + ";" + kQueryRequest;
    +
    +  // This thread will handle the response from the client in a similar way as the quickstep cli will.
    +  std::thread server_handler([&networkIO, &query_stmt]() {
    +    std::string command = networkIO.getNextCommand();
    +    EXPECT_EQ(command, query_stmt);
    +
    +    // Set some output for the main test thread, return.
    +    fprintf(networkIO.out(), "%s", kQueryResponse.c_str());
    +    networkIO.notifyCommandComplete();
    +  });
    +
    +  NetworkCliClient client(
    +    grpc::CreateChannel("localhost:" + std::to_string(FLAGS_port),
    --- End diff --
    
    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 #232: QUICKSTEP-87 Adds network cli interfa...

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

    https://github.com/apache/incubator-quickstep/pull/232#discussion_r112250845
  
    --- Diff: cli/LineReaderBuffered.hpp ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +#define QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +
    +#include "cli/LineReader.hpp"
    +
    +namespace quickstep {
    +
    +class LineReaderBuffered : public LineReader {
    --- End diff --
    
    Does ``BufferedLineReader`` sound a better 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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112547359
  
    --- Diff: cli/QuickstepCli.cpp ---
    @@ -28,26 +28,23 @@
     #include <utility>
     #include <vector>
     
    -#include "cli/CliConfig.h"  // For QUICKSTEP_USE_LINENOISE, QUICKSTEP_ENABLE_GOOGLE_PROFILER.
    +#include "cli/CliConfig.h"  // For QUICKSTEP_ENABLE_GOOGLE_PROFILER.
    --- End diff --
    
    Add `QUICKSTEP_ENABLE_NETWORK_CLI`.


---
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 #232: QUICKSTEP-87 Adds network cli interface.

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

    https://github.com/apache/incubator-quickstep/pull/232
  
    After @hbdeshmukh and I's code review, we created:
    * QUICKSTEP-90
    * QUICKSTEP-91
    in response to some of the issues we found. @zuyu 90 will address the async upgrade.
    
    Additionally, we changed the locks in `NetworkIO.hpp` to be quickstep locks, not `std` locks.


---
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 #232: QUICKSTEP-87 Adds network cli interface.

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

    https://github.com/apache/incubator-quickstep/pull/232
  
    @zuyu Thank you for review. Regarding the question about mutexes and cond_var in the `NetworkCliServiceImpl`, we disallow multiple gRPC threads from submitting requests at the same time.
    
    As for the suggestion, I think it's okay to split those up. I didn't do this in the draft because it creates very small test 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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112548998
  
    --- Diff: cli/tests/NetworkIO_unittest.cpp ---
    @@ -0,0 +1,182 @@
    +/**
    + * 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 <cstddef>
    +#include <cstdio>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +#include "cli/NetworkIO.hpp"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "threading/Thread.hpp"
    +
    +#include "glog/logging.h"
    +#include "gtest/gtest.h"
    +
    +using std::unique_ptr;
    +
    +namespace quickstep {
    +
    +static std::string const kQueryRequest = "O Captain! My Captain!";
    +static std::string const kQueryResponse = "Our fearful trip is done,";
    +
    +/**
    + * Contains a server instance for testing.
    + */
    +class TestNetworkIO {
    + public:
    +  TestNetworkIO()
    +      : service_(),
    +        server_address_("localhost:" + std::to_string(FLAGS_port)),
    +        server_(nullptr) {
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(server_address_, grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    CHECK(server_) << "Failed to start server";
    +    LOG(INFO) << "TestSingleNodeServer listening on " << server_address_;
    +  }
    +
    +  // Gets a message from the Service worker.
    +  std::string getSentMessage() {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    +    std::unique_lock<std::mutex> lock(requestState.mutex_);
    +    while (!requestState.request_ready_)
    +      requestState.condition_.wait(lock);
    +
    +    EXPECT_EQ(requestState.request_ready_, true);
    +    EXPECT_EQ(requestState.response_ready_, false);
    +
    +    return requestState.request_buffer_;
    +  }
    +
    +  // Sets the response message of the Service worker. Alerts it that the request is ready.
    +  void setResponse(std::string response) {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    +    requestState.response_message_.set_query_result(response);
    +
    +    EXPECT_EQ(requestState.response_ready_, false);
    +
    +    requestState.responseReady();
    +  }
    +
    +  NetworkCliServiceImpl& getService() {
    +    return service_;
    +  }
    +
    +  ~TestNetworkIO() {
    +    service_.kill();
    +    server_->Shutdown();
    +    server_->Wait();
    +  }
    +
    + private:
    +  NetworkCliServiceImpl service_;
    +  std::string server_address_;
    +  std::unique_ptr<grpc::Server> server_;
    +};
    +
    +/**
    + * Tests that killing the service will cancel requests.
    + */
    +TEST(NetworkIOTest, TestShutdown) {
    +  TestNetworkIO server;
    +
    +  server.getService().kill();
    +
    +  NetworkCliClient client(
    +    grpc::CreateChannel("localhost:" + std::to_string(FLAGS_port),
    --- End diff --
    
    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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112788365
  
    --- Diff: cli/tests/CMakeLists.txt ---
    @@ -32,6 +32,17 @@ if (ENABLE_DISTRIBUTED)
                      "${PROJECT_SOURCE_DIR}/utility/textbased_test/TextBasedTest.hpp")
     endif(ENABLE_DISTRIBUTED)
     
    +if (ENABLE_SINGLE_NODE_SERVER)
    --- End diff --
    
    Corrected to `ENABLE_NETWORK_CLI`


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112790553
  
    --- Diff: cli/NetworkCliClient.hpp ---
    @@ -0,0 +1,91 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +#define QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +/**
    + * A simple wrapper class used to do CLI interactions with QuickstepCLI via the gRPC interface.
    + */
    +class NetworkCliClient {
    + public:
    +  explicit NetworkCliClient(std::shared_ptr<Channel> const &channel)
    --- End diff --
    
    By consistency, we should be aligned with all usages in the codebase. Thus, `const Foo &foo`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112407922
  
    --- Diff: cli/NetworkCliClient.hpp ---
    @@ -0,0 +1,91 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +#define QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +/**
    + * A simple wrapper class used to do CLI interactions with QuickstepCLI via the gRPC interface.
    + */
    +class NetworkCliClient {
    + public:
    +  explicit NetworkCliClient(std::shared_ptr<Channel> const &channel)
    --- End diff --
    
    Change to `const std::shared_ptr<Channel> &channel`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112836758
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,286 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <cstdio>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/Macros.hpp"
    +#include "utility/MemStream.hpp"
    +#include "utility/ThreadSafeQueue.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
    + * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
    + */
    +class RequestState {
    + public:
    +  explicit RequestState(QueryRequest const *request)
    +      : response_ready_(false),
    +        canceled_(false),
    +        request_(request),
    +        condition_(mutex_.createConditionVariable()) {}
    +
    +  /**
    +   * @brief Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   * @param stdout_str Stdout from Quickstep.
    +   * @param stderr_str Stderr from Quickstep.
    +   */
    +  void responseReady(std::string stdout_str, std::string stderr_str) {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    response_message_.set_query_result(stdout_str);
    +    response_message_.set_error_result(stderr_str);
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @brief The producer thread blocks until Quickstep signals that it has finished.
    +   * @note Quickstep may either produce a query response or cancel. Both these actions must notify the condition.
    +   */
    +  void waitForResponse() {
    +    while (!response_ready_)
    +      condition_->await();
    +  }
    +
    +  /**
    +   * @brief Notifies the producer that its request will not be served by Quickstep.
    +   */
    +  void cancel() {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    canceled_ = true;
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @return The producer's query for Quickstep to process.
    +   */
    +  std::string getRequest() const {
    +    return request_->query();
    +  }
    +
    +  /**
    +   * @return The response message from Quickstep.
    +   */
    +  QueryResponse getResponse() const {
    +    DCHECK(response_ready_);
    +    return response_message_;
    +  }
    +
    +  /**
    +   * @return True if query was canceled.
    +   */
    +  bool getCanceled() const {
    +    DCHECK(response_ready_);
    +    return canceled_;
    +  }
    +
    + private:
    +  bool response_ready_;
    +  bool canceled_;
    +  QueryRequest const * request_;
    +  QueryResponse response_message_;
    +  Mutex mutex_;
    +  ConditionVariable *condition_;  // note: owned by the mutex.
    +
    +  DISALLOW_COPY_AND_ASSIGN(RequestState);
    +};
    +
    +}  // namespace networkio_internal
    +
    +using networkio_internal::RequestState;
    +
    +/**
    + * @brief Contains the callback methods which the gRPC service defines.
    + * When a request is made of gRPC, a gRPC worker thread is spun up and enters one of the callback methods
    + * (eg SendQuery). This thread keeps the network connection open while Quickstep processes the query. Concurrency
    + * control between the gRPC worker thread, and the Quickstep thread is controlled by a RequestState object which is
    + * created for each interaction.
    + */
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +      : running_(true) {}
    +
    +  /**
    +   * @brief Handles gRPC request.
    +   * Sets the buffer in the RequestState, places the request on a queue, then waits for a response. The response shall
    +   * be triggered by a
    +   */
    +  Status SendQuery(grpc::ServerContext *context,
    +                   const QueryRequest *request,
    +                   QueryResponse *response) override {
    +    std::unique_ptr<RequestState> request_state;
    +    // Check to see if the gRPC service has been shutdown.
    +    {
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      if (!running_) {
    +        return Status::CANCELLED;
    +      }
    +      // While we have a service lock, we add to the Queue. Note that we keep the service lock to protect ourselves from
    +      // a race condition in the kill() method.
    +      request_state.reset(new RequestState(request));
    +
    +      // Pushing to the queue will notify consumers.
    +      request_queue_.push(request_state.get());
    +    }
    +    DCHECK(request_state);
    +
    +    // We have pushed to the request queue, so all there is to do now is wait for Quickstep to process the request.
    +    request_state->waitForResponse();
    +    if (request_state->getCanceled()) {
    +      return Status::CANCELLED;
    +    }
    +    *response = request_state->getResponse();
    +    return Status::OK;
    +  }
    +
    +  /**
    +   * @brief The consumer thread waits for a request to materialize.
    +   * @return A non-owned RequestState.
    +   */
    +  RequestState* waitForRequest() {
    +    return request_queue_.popOne();
    +  }
    +
    +  /**
    +   * @brief Stops accepting further requests and cancels all pending requests.
    +   */
    +  void kill() {
    +    {
    +      // This action guarantees that no further requests are added to the queue.
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      running_ = false;
    +    }
    +    // Go through each pending request, and cancel them.
    +    while (!request_queue_.empty()) {
    +      request_queue_.popOne()->cancel();
    +    }
    +  }
    +
    + private:
    +  Mutex service_mtx_;
    +  bool running_;
    +  ThreadSafeQueue<RequestState*> request_queue_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkCliServiceImpl);
    +};
    +
    +class NetworkIOHandle : public IOHandle {
    --- End diff --
    
    True, both the `Handle` and `IO` implementations could 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] incubator-quickstep pull request #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112416384
  
    --- Diff: cli/LineReaderBuffered.hpp ---
    @@ -0,0 +1,73 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +#define QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +
    +#include <string>
    +
    +#include "cli/LineReader.hpp"
    --- End diff --
    
    Add the header file for `DISALLOW_COPY_AND_ASSIGN`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112417791
  
    --- Diff: cli/tests/NetworkIO_unittest.cpp ---
    @@ -0,0 +1,182 @@
    +/**
    + * 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 <cstddef>
    +#include <cstdio>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +#include "cli/NetworkIO.hpp"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "threading/Thread.hpp"
    +
    +#include "glog/logging.h"
    +#include "gtest/gtest.h"
    +
    +using std::unique_ptr;
    +
    +namespace quickstep {
    +
    +static std::string const kQueryRequest = "O Captain! My Captain!";
    +static std::string const kQueryResponse = "Our fearful trip is done,";
    +
    +/**
    + * Contains a server instance for testing.
    + */
    +class TestNetworkIO {
    + public:
    +  TestNetworkIO()
    +      : service_(),
    +        server_address_("localhost:" + std::to_string(FLAGS_port)),
    +        server_(nullptr) {
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(server_address_, grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    CHECK(server_) << "Failed to start server";
    +    LOG(INFO) << "TestSingleNodeServer listening on " << server_address_;
    +  }
    +
    +  // Gets a message from the Service worker.
    +  std::string getSentMessage() {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    +    std::unique_lock<std::mutex> lock(requestState.mutex_);
    +    while (!requestState.request_ready_)
    +      requestState.condition_.wait(lock);
    +
    +    EXPECT_EQ(requestState.request_ready_, true);
    +    EXPECT_EQ(requestState.response_ready_, false);
    +
    +    return requestState.request_buffer_;
    +  }
    +
    +  // Sets the response message of the Service worker. Alerts it that the request is ready.
    +  void setResponse(std::string response) {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    +    requestState.response_message_.set_query_result(response);
    +
    +    EXPECT_EQ(requestState.response_ready_, false);
    +
    +    requestState.responseReady();
    +  }
    +
    +  NetworkCliServiceImpl& getService() {
    +    return service_;
    +  }
    +
    +  ~TestNetworkIO() {
    +    service_.kill();
    +    server_->Shutdown();
    +    server_->Wait();
    +  }
    +
    + private:
    +  NetworkCliServiceImpl service_;
    +  std::string server_address_;
    +  std::unique_ptr<grpc::Server> server_;
    +};
    +
    +/**
    + * Tests that killing the service will cancel requests.
    + */
    +TEST(NetworkIOTest, TestShutdown) {
    +  TestNetworkIO server;
    +
    +  server.getService().kill();
    +
    +  NetworkCliClient client(
    +    grpc::CreateChannel("localhost:" + std::to_string(FLAGS_port),
    +                        grpc::InsecureChannelCredentials()));
    +  QueryRequest request;
    +  request.set_query(kQueryRequest);
    +  QueryResponse response;
    +  Status status = client.SendQuery(request, &response);
    +  ASSERT_EQ(status.error_code(), grpc::CANCELLED);
    +}
    +
    +/**
    + * Tests a simple call and response to the Service.
    + */
    +TEST(NetworkIOTest, TestNetworkIOCommandInteraction) {
    +  NetworkIO networkIO;
    +  std::string const query_stmt = kQueryRequest + ";" + kQueryRequest;
    +
    +  // This thread will handle the response from the client in a similar way as the quickstep cli will.
    +  std::thread server_handler([&networkIO, &query_stmt]() {
    +    std::string command = networkIO.getNextCommand();
    +    EXPECT_EQ(command, query_stmt);
    +
    +    // Set some output for the main test thread, return.
    +    fprintf(networkIO.out(), "%s", kQueryResponse.c_str());
    +    networkIO.notifyCommandComplete();
    +  });
    +
    +  NetworkCliClient client(
    +    grpc::CreateChannel("localhost:" + std::to_string(FLAGS_port),
    +                        grpc::InsecureChannelCredentials()));
    +  QueryRequest request;
    +  request.set_query(query_stmt);
    +  QueryResponse response;
    +  Status status = client.SendQuery(request, &response);
    +  ASSERT_TRUE(status.ok());
    +  EXPECT_EQ(response.query_result(), kQueryResponse);
    +  EXPECT_EQ(response.error_result(), std::string(""));
    +
    +  server_handler.join();
    +}
    +
    +TEST(NetworkIOTest, TestLineReaderBuffered) {
    +  // The buffered line reader is used exclusively by the NetworkIO's client.
    +  LineReaderBuffered linereader;
    +  EXPECT_TRUE(linereader.bufferEmpty());
    +
    +  std::string stmt = "select * from foo;";
    +  std::string stmts = stmt + "select 1; select 2; quit;";
    +  auto const num_stmts = std::count(stmts.begin(), stmts.end(), ';');
    +  linereader.setBuffer(stmts);
    +  ASSERT_FALSE(linereader.bufferEmpty());
    +
    +  std::vector<std::string> stmts_vec;
    +  int parsed_commands;
    +  for (parsed_commands = 0;
    +      parsed_commands < num_stmts + 1 && !linereader.bufferEmpty();
    +      parsed_commands++) {
    +    std::string command = linereader.getNextCommand();
    +    if (command != "") {
    +      stmts_vec.push_back(command);
    +    }
    +  }
    +
    +  EXPECT_EQ(stmt, stmts_vec.front());
    +  EXPECT_EQ(num_stmts, stmts_vec.size());
    +  EXPECT_TRUE(linereader.bufferEmpty());
    +}
    +
    +}  // namespace quickstep
    +
    +int main(int argc, char** argv) {
    +  google::InitGoogleLogging(argv[0]);
    +  ::testing::InitGoogleTest(&argc, argv);
    --- End diff --
    
    We need to set up `gflags`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112417113
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,227 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/MemStream.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer (requestor/requestee)
    + * thread.
    + */
    +class RequestState {
    + public:
    +  RequestState() :
    +    request_ready_(false),
    --- End diff --
    
    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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112414736
  
    --- Diff: cli/CMakeLists.txt ---
    @@ -39,18 +38,39 @@ if (ENABLE_GOOGLE_PROFILER)
       set(QUICKSTEP_ENABLE_GOOGLE_PROFILER TRUE)
     endif()
     
    +if (ENABLE_NETWORK_CLI)
    +  set(QUICKSTEP_ENABLE_NETWORK_CLI TRUE)
    +endif()
    +
     configure_file (
       "${CMAKE_CURRENT_SOURCE_DIR}/CliConfig.h.in"
       "${CMAKE_CURRENT_BINARY_DIR}/CliConfig.h"
     )
     
    +# Compile the protos for Single Node Server mode.
    +if(ENABLE_NETWORK_CLI)
    --- End diff --
    
    Could we be consistent regarding `if(ENABLE_NETWORK_CLI)` vs `if (ENABLE_NETWORK_CLI)`?
    
    Same thing 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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819645
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,286 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <cstdio>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/Macros.hpp"
    +#include "utility/MemStream.hpp"
    +#include "utility/ThreadSafeQueue.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
    + * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
    + */
    +class RequestState {
    + public:
    +  explicit RequestState(QueryRequest const *request)
    +      : response_ready_(false),
    +        canceled_(false),
    +        request_(request),
    +        condition_(mutex_.createConditionVariable()) {}
    +
    +  /**
    +   * @brief Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   * @param stdout_str Stdout from Quickstep.
    +   * @param stderr_str Stderr from Quickstep.
    +   */
    +  void responseReady(std::string stdout_str, std::string stderr_str) {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    response_message_.set_query_result(stdout_str);
    +    response_message_.set_error_result(stderr_str);
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @brief The producer thread blocks until Quickstep signals that it has finished.
    +   * @note Quickstep may either produce a query response or cancel. Both these actions must notify the condition.
    +   */
    +  void waitForResponse() {
    +    while (!response_ready_)
    +      condition_->await();
    +  }
    +
    +  /**
    +   * @brief Notifies the producer that its request will not be served by Quickstep.
    +   */
    +  void cancel() {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    canceled_ = true;
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @return The producer's query for Quickstep to process.
    +   */
    +  std::string getRequest() const {
    +    return request_->query();
    +  }
    +
    +  /**
    +   * @return The response message from Quickstep.
    +   */
    +  QueryResponse getResponse() const {
    +    DCHECK(response_ready_);
    +    return response_message_;
    +  }
    +
    +  /**
    +   * @return True if query was canceled.
    +   */
    +  bool getCanceled() const {
    +    DCHECK(response_ready_);
    +    return canceled_;
    +  }
    +
    + private:
    +  bool response_ready_;
    +  bool canceled_;
    +  QueryRequest const * request_;
    +  QueryResponse response_message_;
    +  Mutex mutex_;
    +  ConditionVariable *condition_;  // note: owned by the mutex.
    +
    +  DISALLOW_COPY_AND_ASSIGN(RequestState);
    +};
    +
    +}  // namespace networkio_internal
    +
    +using networkio_internal::RequestState;
    +
    +/**
    + * @brief Contains the callback methods which the gRPC service defines.
    + * When a request is made of gRPC, a gRPC worker thread is spun up and enters one of the callback methods
    + * (eg SendQuery). This thread keeps the network connection open while Quickstep processes the query. Concurrency
    + * control between the gRPC worker thread, and the Quickstep thread is controlled by a RequestState object which is
    + * created for each interaction.
    + */
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +      : running_(true) {}
    +
    +  /**
    +   * @brief Handles gRPC request.
    +   * Sets the buffer in the RequestState, places the request on a queue, then waits for a response. The response shall
    +   * be triggered by a
    +   */
    +  Status SendQuery(grpc::ServerContext *context,
    +                   const QueryRequest *request,
    +                   QueryResponse *response) override {
    +    std::unique_ptr<RequestState> request_state;
    +    // Check to see if the gRPC service has been shutdown.
    +    {
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      if (!running_) {
    +        return Status::CANCELLED;
    +      }
    +      // While we have a service lock, we add to the Queue. Note that we keep the service lock to protect ourselves from
    +      // a race condition in the kill() method.
    +      request_state.reset(new RequestState(request));
    +
    +      // Pushing to the queue will notify consumers.
    +      request_queue_.push(request_state.get());
    +    }
    +    DCHECK(request_state);
    +
    +    // We have pushed to the request queue, so all there is to do now is wait for Quickstep to process the request.
    +    request_state->waitForResponse();
    +    if (request_state->getCanceled()) {
    +      return Status::CANCELLED;
    +    }
    +    *response = request_state->getResponse();
    +    return Status::OK;
    +  }
    +
    +  /**
    +   * @brief The consumer thread waits for a request to materialize.
    +   * @return A non-owned RequestState.
    +   */
    +  RequestState* waitForRequest() {
    +    return request_queue_.popOne();
    +  }
    +
    +  /**
    +   * @brief Stops accepting further requests and cancels all pending requests.
    +   */
    +  void kill() {
    +    {
    +      // This action guarantees that no further requests are added to the queue.
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      running_ = false;
    +    }
    +    // Go through each pending request, and cancel them.
    +    while (!request_queue_.empty()) {
    +      request_queue_.popOne()->cancel();
    +    }
    +  }
    +
    + private:
    +  Mutex service_mtx_;
    +  bool running_;
    +  ThreadSafeQueue<RequestState*> request_queue_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkCliServiceImpl);
    +};
    +
    +class NetworkIOHandle : public IOHandle {
    + public:
    +  explicit NetworkIOHandle(RequestState* state)
    +      : request_state_(state) {}
    +
    +  ~NetworkIOHandle() override {
    +    // All the commands from the last network interaction have completed, return our response.
    +    // This signals to the producer thread that the interaction is complete.
    +    request_state_->responseReady(out_stream_.str(), err_stream_.str());
    +  }
    +
    +  FILE* out() override {
    +    return out_stream_.file();
    +  }
    +
    +  FILE* err() override {
    +    return err_stream_.file();
    +  }
    +
    +  std::string getCommand() override {
    +    return request_state_->getRequest();
    +  }
    +
    + private:
    +  MemStream out_stream_, err_stream_;
    +  RequestState* request_state_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkIOHandle);
    +};
    +
    +/**
    + * A network interface that uses gRPC to accept commands.
    + */
    +class NetworkIO : public IOInterface {
    + public:
    +  NetworkIO() {
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(GetAddress(), grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    LOG(INFO) << "Listening on " << GetAddress();
    --- End diff --
    
    Suggest to move the implementation to the source file.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112323095
  
    --- Diff: cli/LineReaderBuffered.cpp ---
    @@ -0,0 +1,67 @@
    +/**
    + * 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 "cli/LineReaderBuffered.hpp"
    +#include <iostream>
    +
    +namespace quickstep {
    +
    +LineReaderBuffered::LineReaderBuffered(const std::string &default_prompt,
    +                                       const std::string &continue_prompt)
    +  : LineReader("", ""),
    +    buffer_empty_(true) { }
    +
    +LineReaderBuffered::LineReaderBuffered() : LineReader("", ""), buffer_empty_(true) { }
    +
    +std::string LineReaderBuffered::getLineInternal(const bool continuing) {
    +  // This method is called when the leftover_ string is depleted.
    +  buffer_empty_ = true;
    +  return "";
    +}
    +
    +}
    +
    +
    +/*
    +TEST(NetworkIOTest, TestLineReaderBuffered) {
    +  // The buffered line reader is used exclusively by the NetworkIO.
    +  LineReaderBuffered linereader;
    +  EXPECT_TRUE(linereader.bufferEmpty());
    +
    +  std::string stmt_1 = "select * from foo;";
    +  std::string stmts = stmt_1 + "select 1; select 2; quit;";
    +  linereader.setBuffer(stmts);
    +  ASSERT_FALSE(linereader.bufferEmpty());
    +
    +  std::vector<std::string> stmts_vec;
    +  int parsed_commands;
    +  for(parsed_commands = 0;
    +      parsed_commands < 10 && !linereader.bufferEmpty();
    +      parsed_commands++) {
    +    std::string command = linereader.getNextCommand();
    +    if (command != "") {
    +      stmts_vec.push_back(command);
    +    }
    +  }
    +
    +  EXPECT_EQ(stmt_1, stmts_vec.front());
    +  EXPECT_EQ(4, stmts_vec.size());
    +  EXPECT_TRUE(linereader.bufferEmpty());
    +}
    +*/
    --- End diff --
    
    Ah, I was stashing this the old fashion way... removed



---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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

    https://github.com/apache/incubator-quickstep/pull/232#discussion_r112249378
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,61 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <string>
    +
    +/**
    + * Virtual base defines a generic, file-based interface around IO.
    + */
    +class IOInterface {
    + public:
    +  IOInterface() {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  virtual FILE* out() = 0;
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  virtual FILE* err() = 0;
    +
    +  /**
    +   * Requests a complete SQL command. This call may block until user input is given.
    +   * @note When the command is complete, commandComplete() should be called so that certain implementations can clean
    +   *    up state related to sending the command.
    +   * @return A string containing a quickstep command.
    +   */
    +  virtual std::string nextCommand() = 0;
    +
    +  /**
    +   * Notifies the IO system that the previously acquired command is complete.
    --- End diff --
    
    Missing the @ brief annotation


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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

    https://github.com/apache/incubator-quickstep/pull/232#discussion_r112250274
  
    --- Diff: cli/LineReaderBuffered.cpp ---
    @@ -0,0 +1,67 @@
    +/**
    + * 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 "cli/LineReaderBuffered.hpp"
    +#include <iostream>
    +
    +namespace quickstep {
    +
    +LineReaderBuffered::LineReaderBuffered(const std::string &default_prompt,
    +                                       const std::string &continue_prompt)
    +  : LineReader("", ""),
    +    buffer_empty_(true) { }
    +
    +LineReaderBuffered::LineReaderBuffered() : LineReader("", ""), buffer_empty_(true) { }
    +
    +std::string LineReaderBuffered::getLineInternal(const bool continuing) {
    +  // This method is called when the leftover_ string is depleted.
    +  buffer_empty_ = true;
    +  return "";
    +}
    +
    +}
    +
    +
    +/*
    +TEST(NetworkIOTest, TestLineReaderBuffered) {
    +  // The buffered line reader is used exclusively by the NetworkIO.
    +  LineReaderBuffered linereader;
    +  EXPECT_TRUE(linereader.bufferEmpty());
    +
    +  std::string stmt_1 = "select * from foo;";
    +  std::string stmts = stmt_1 + "select 1; select 2; quit;";
    +  linereader.setBuffer(stmts);
    +  ASSERT_FALSE(linereader.bufferEmpty());
    +
    +  std::vector<std::string> stmts_vec;
    +  int parsed_commands;
    +  for(parsed_commands = 0;
    +      parsed_commands < 10 && !linereader.bufferEmpty();
    +      parsed_commands++) {
    +    std::string command = linereader.getNextCommand();
    +    if (command != "") {
    +      stmts_vec.push_back(command);
    +    }
    +  }
    +
    +  EXPECT_EQ(stmt_1, stmts_vec.front());
    +  EXPECT_EQ(4, stmts_vec.size());
    +  EXPECT_TRUE(linereader.bufferEmpty());
    +}
    +*/
    --- End diff --
    
    Can you either uncomment the dead code or remove it? 


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112836065
  
    --- Diff: cli/CMakeLists.txt ---
    @@ -15,7 +15,6 @@
     # specific language governing permissions and limitations
     # under the License.
     
    -include_directories(${CMAKE_CURRENT_BINARY_DIR})
    --- End diff --
    
    As I was checking over the commit, I saw this and thought I added it so that the compiled grpc files would be included. On looking at the history, it was not my addition- I'll put it back.
    
    But it raises the question: why is it there to begin with?


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112406128
  
    --- Diff: cli/LineReaderBuffered.cpp ---
    @@ -0,0 +1,39 @@
    +/**
    + * 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 "cli/LineReaderBuffered.hpp"
    +
    +#include <iostream>
    +
    +namespace quickstep {
    +
    +LineReaderBuffered::LineReaderBuffered(const std::string &default_prompt,
    +                                       const std::string &continue_prompt)
    +  : LineReader("", ""),
    --- End diff --
    
    Add two more whitespace indentations.
    
    Also, it should be `LineReader(default_prompt, continue_prompt)`.
    
    Finally, this constructor is not used.


---
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 #232: Adds network cli interface.

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

    https://github.com/apache/incubator-quickstep/pull/232
  
    Hi @cramja 
    
    Considering that this is a fairly big feature, can you please create a JIRA ticket and link this PR with it?


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112407440
  
    --- Diff: cli/LocalIO.hpp ---
    @@ -0,0 +1,74 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LOCAL_IO_HPP_
    +#define QUICKSTEP_CLI_LOCAL_IO_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "cli/CliConfig.h"
    +#include "cli/IOInterface.hpp"
    +#include "cli/LineReader.hpp"
    +
    +#ifdef QUICKSTEP_USE_LINENOISE
    +#include "cli/LineReaderLineNoise.hpp"
    +typedef quickstep::LineReaderLineNoise LineReaderImpl;
    +#else
    +#include "cli/LineReaderDumb.hpp"
    +typedef quickstep::LineReaderDumb LineReaderImpl;
    +#endif
    +
    +namespace quickstep {
    +
    +class LocalIO : public IOInterface {
    + public:
    +  LocalIO() : IOInterface(), line_reader_("quickstep> ", "      ...> ") {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  FILE *out() override {
    +    return stdout;
    +  }
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  FILE *err() override {
    +    return stderr;
    +  }
    +
    +  /**
    +   * Requests a complete SQL command. This call may block until user input is given.
    +   * @note When the command is complete, commandComplete() should be called so that certain implementations can clean
    +   *    up state related to sending the command.
    +   * @return A string containing a quickstep command.
    +   */
    +  std::string getNextCommand() override {
    +    return line_reader_.getNextCommand();
    +  }
    +
    + private:
    +  LineReaderImpl line_reader_;
    +};
    --- End diff --
    
    Add `DISALLOW_COPY_AND_ASSIGN`, include and link the header.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112325082
  
    --- Diff: cli/LineReaderBuffered.hpp ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +#define QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +
    +#include "cli/LineReader.hpp"
    +
    +namespace quickstep {
    +
    +class LineReaderBuffered : public LineReader {
    + public:
    +  /**
    +   * A line reader which accepts a string buffer.
    +   * This does no IO and ignores any prompts given to the constructor.
    +   * @param default_prompt Ignored.
    +   * @param continue_prompt Ignored.
    --- End diff --
    
    how about:
    ```
      /**
       * @brief A line reader which accepts a string buffer.
       * Other line readers are ment to support some form of user interaction. This linereader does not and is intended for
       * programmer use- it does not print anything to stdout. Therefore it ignores any prompt strings given to the
       * inherited constructor.
       * @param default_prompt Not used by this line reader, but required by interface.
       * @param continue_prompt Not used by this line reader, but required by interface.
       */
    ```


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112407627
  
    --- Diff: cli/NetworkCliClient.hpp ---
    @@ -0,0 +1,91 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +#define QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +/**
    + * A simple wrapper class used to do CLI interactions with QuickstepCLI via the gRPC interface.
    + */
    +class NetworkCliClient {
    + public:
    +  explicit NetworkCliClient(std::shared_ptr<Channel> const &channel)
    +    : stub_(NetworkCli::NewStub(channel)) {}
    --- End diff --
    
    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 #232: QUICKSTEP-87 Adds network cli interfa...

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

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


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112408163
  
    --- Diff: cli/NetworkCliClient.hpp ---
    @@ -0,0 +1,91 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +#define QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +/**
    + * A simple wrapper class used to do CLI interactions with QuickstepCLI via the gRPC interface.
    + */
    +class NetworkCliClient {
    + public:
    +  explicit NetworkCliClient(std::shared_ptr<Channel> const &channel)
    +    : stub_(NetworkCli::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 Query(const std::string &user_query) {
    +    QueryRequest request;
    +    request.set_query(user_query);
    +    QueryResponse response;
    +
    +    Status status = SendQuery(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";
    +    }
    +  }
    +
    +  Status SendQuery(QueryRequest request, QueryResponse* response) {
    +    ClientContext context;
    +    return stub_->SendQuery(&context, request, response);
    +  }
    +
    + private:
    +  /**
    +   * Handle a valid response from the server.
    +   * @param response A valid query response.
    +   * @return The response string.
    +   */
    +  std::string HandleQueryResponse(QueryResponse const &response) const {
    +    return response.query_result() + response.error_result();
    +  }
    +
    +  std::unique_ptr<NetworkCli::Stub> stub_;
    +};
    --- End diff --
    
    Add `DISALLOW_COPY_AND_ASSIGN`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819589
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,286 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <cstdio>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/Macros.hpp"
    +#include "utility/MemStream.hpp"
    +#include "utility/ThreadSafeQueue.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
    + * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
    + */
    +class RequestState {
    + public:
    +  explicit RequestState(QueryRequest const *request)
    +      : response_ready_(false),
    +        canceled_(false),
    +        request_(request),
    +        condition_(mutex_.createConditionVariable()) {}
    +
    +  /**
    +   * @brief Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   * @param stdout_str Stdout from Quickstep.
    +   * @param stderr_str Stderr from Quickstep.
    +   */
    +  void responseReady(std::string stdout_str, std::string stderr_str) {
    --- End diff --
    
    Use `const std::string &` for both input arguments to avoid deep copies.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819443
  
    --- Diff: cli/Flags.cpp ---
    @@ -87,4 +87,25 @@ DEFINE_bool(preload_buffer_pool, false,
                 "accepting queries (should also set --buffer_pool_slots to be "
                 "large enough to accomodate the entire database).");
     
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  int const min = 0, max = 65536;
    +  if (value > min && value < max) {
    +    return true;
    +  }
    +  std::cout << "Invalid value for --" << flagname << ": " << value
    +            << "\nUse ports between " << min << " and "
    +            << max << std::endl;
    +  return false;
    +}
    +
    +DEFINE_int32(cli_network_port, 3000,
    +             "Listens for TCP connections on this port when network mode is enabled. "
    +             "This is only used if the cli is set to use the network mode (--mode=network).");
    +DEFINE_validator(cli_network_port, &ValidatePort);
    +
    +DEFINE_string(cli_network_ip, "localhost",
    --- End diff --
    
    Change to `0.0.0.0` from `localhost`, as the latter is hostname, not ip.
    
    Also, move these two new flags in `NetworkIO` file, as they are not used by any other 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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112418012
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,227 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/MemStream.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer (requestor/requestee)
    + * thread.
    + */
    +class RequestState {
    + public:
    +  RequestState() :
    +    request_ready_(false),
    +    response_ready_(false),
    +    request_buffer_(""),
    +    mutex_(),
    +    condition_() {}
    +
    +  /**
    +   * Notifies waiter that a piece of work has been created and added to the buffer.
    +   * @param to_consume The arguments for the consuming thread.
    +   */
    +  void requestReady(std::string to_consume) {
    --- End diff --
    
    Change to `const std::string &to_consume`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112836342
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,82 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "utility/Macros.hpp"
    +
    +namespace quickstep {
    +
    +/**
    + * An individual IO interaction with Quickstep.
    + */
    +class IOHandle {
    + public:
    +  IOHandle() {}
    +
    +  /**
    +   * @note Destructor should handle clean up of any IO state.
    +   */
    +  virtual ~IOHandle() {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  virtual FILE *out() = 0;
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  virtual FILE *err() = 0;
    +
    +  virtual std::string getCommand() = 0;
    --- End diff --
    
    Err, I was thinking of `virtual IOHandle* getNextIOHandle()`. Your change makes sense.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819559
  
    --- Diff: cli/NetworkCliClientMain.cpp ---
    @@ -0,0 +1,62 @@
    +/**
    + * 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 <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <istream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +#include "cli/NetworkIO.hpp"
    +
    +
    +#include "gflags/gflags.h"
    +
    +using quickstep::LineReaderBuffered;
    +using quickstep::NetworkCliClient;
    +
    +int main(int argc, char **argv) {
    +  google::InitGoogleLogging(argv[0]);
    +  gflags::ParseCommandLineFlags(&argc, &argv, true);
    +
    --- End diff --
    
    We need to initialize `grpc` by calling `grpc_init`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112548566
  
    --- Diff: cli/tests/NetworkIO_unittest.cpp ---
    @@ -0,0 +1,182 @@
    +/**
    + * 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 <cstddef>
    +#include <cstdio>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +#include "cli/NetworkIO.hpp"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "threading/Thread.hpp"
    +
    +#include "glog/logging.h"
    +#include "gtest/gtest.h"
    +
    +using std::unique_ptr;
    +
    +namespace quickstep {
    +
    +static std::string const kQueryRequest = "O Captain! My Captain!";
    +static std::string const kQueryResponse = "Our fearful trip is done,";
    +
    +/**
    + * Contains a server instance for testing.
    + */
    +class TestNetworkIO {
    + public:
    +  TestNetworkIO()
    +      : service_(),
    +        server_address_("localhost:" + std::to_string(FLAGS_port)),
    +        server_(nullptr) {
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(server_address_, grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    CHECK(server_) << "Failed to start server";
    +    LOG(INFO) << "TestSingleNodeServer listening on " << server_address_;
    +  }
    +
    +  // Gets a message from the Service worker.
    +  std::string getSentMessage() {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    --- End diff --
    
    Change to `networkio_internal::RequestState &request_state`.
    
    Ditto in `setResponse`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112417078
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,227 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/MemStream.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    --- End diff --
    
    No need for this empty line.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112788031
  
    --- Diff: cli/NetworkCliClient.hpp ---
    @@ -0,0 +1,91 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +#define QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +/**
    + * A simple wrapper class used to do CLI interactions with QuickstepCLI via the gRPC interface.
    + */
    +class NetworkCliClient {
    + public:
    +  explicit NetworkCliClient(std::shared_ptr<Channel> const &channel)
    --- End diff --
    
    That said, I will change it here because elsewhere in this file, the const-first pattern is followed.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819466
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,82 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "utility/Macros.hpp"
    +
    +namespace quickstep {
    +
    +/**
    + * An individual IO interaction with Quickstep.
    + */
    +class IOHandle {
    + public:
    +  IOHandle() {}
    +
    +  /**
    +   * @note Destructor should handle clean up of any IO state.
    +   */
    +  virtual ~IOHandle() {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  virtual FILE *out() = 0;
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  virtual FILE *err() = 0;
    +
    +  virtual std::string getCommand() = 0;
    +
    + private:
    +  DISALLOW_COPY_AND_ASSIGN(IOHandle);
    +};
    +
    +/**
    + * Virtual base defines a generic, file-based interface around IO. One IO interaction (eg a SQL query) will be assigned
    + * an IOHandle. On destruction of the IOHandle, the IO interaction has finished.
    + */
    +class IOInterface {
    + public:
    +  IOInterface() {}
    +
    +  /**
    +   * @note Destructing the IOInterface should close any outstanding IO state (eg an open port).
    +   */
    +  virtual ~IOInterface() {}
    +
    +  /**
    +   * @brief Retrieves the next IOHandle. Blocks if no IO ready.
    +   * @return An IOHandle.
    +   */
    +  virtual IOHandle* getNextIOHandle() = 0;
    + private:
    --- End diff --
    
    Add an empty line above.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112404491
  
    --- Diff: cli/Flags.cpp ---
    @@ -87,4 +87,18 @@ DEFINE_bool(preload_buffer_pool, false,
                 "accepting queries (should also set --buffer_pool_slots to be "
                 "large enough to accomodate the entire database).");
     
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  int const min = 0, max = 65536;
    +  if (value > min && value < max) {
    +    return true;
    +  }
    +  std::cout << "Invalid value for --" << flagname << ": " << std::to_string(value)
    --- End diff --
    
    No needs for `to_string`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112411238
  
    --- Diff: cli/NetworkCliClientMain.cpp ---
    @@ -0,0 +1,62 @@
    +/**
    + * 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 <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <istream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +
    +#include "gflags/gflags.h"
    +
    +DECLARE_int32(port);
    +
    +using quickstep::LineReaderBuffered;
    +using quickstep::NetworkCliClient;
    +
    +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.
    +  NetworkCliClient qs_client(
    +    grpc::CreateChannel("localhost:" + std::to_string(quickstep::FLAGS_port),
    --- End diff --
    
    Add two more whitespace indentation, and we need one more flag for the ip address, instead of `localhost`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112405059
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,62 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <string>
    +
    +/**
    + * Virtual base defines a generic, file-based interface around IO.
    + */
    +class IOInterface {
    + public:
    +  IOInterface() {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  virtual FILE* out() = 0;
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  virtual FILE* err() = 0;
    +
    +  /**
    +   * @brief Requests a complete SQL command.
    +   * This call may block until user input is given.
    +   * @note When the command is complete, commandComplete() should be called so that certain implementations can clean
    +   *    up state related to sending the command.
    +   * @return A string containing a quickstep command.
    +   */
    +  virtual std::string getNextCommand() = 0;
    +
    +  /**
    +   * @brief Notifies the IO system that the previously acquired command is complete.
    +   */
    +  virtual void notifyCommandComplete() {}
    +
    +  /**
    +   * @brief Notifies the IO system that quickstep is shutting down.
    +   */
    +  virtual void notifyShutdown() {}
    --- End diff --
    
    We need a `private` session with `DISALLOW_COPY_AND_ASSIGN(IOInterface);`, and include / link the `macro` header.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112324457
  
    --- Diff: cli/LineReaderBuffered.cpp ---
    @@ -0,0 +1,67 @@
    +/**
    + * 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 "cli/LineReaderBuffered.hpp"
    +#include <iostream>
    +
    +namespace quickstep {
    +
    +LineReaderBuffered::LineReaderBuffered(const std::string &default_prompt,
    +                                       const std::string &continue_prompt)
    +  : LineReader("", ""),
    +    buffer_empty_(true) { }
    +
    +LineReaderBuffered::LineReaderBuffered() : LineReader("", ""), buffer_empty_(true) { }
    +
    +std::string LineReaderBuffered::getLineInternal(const bool continuing) {
    +  // This method is called when the leftover_ string is depleted.
    +  buffer_empty_ = true;
    +  return "";
    +}
    +
    +}
    +
    +
    +/*
    +TEST(NetworkIOTest, TestLineReaderBuffered) {
    +  // The buffered line reader is used exclusively by the NetworkIO.
    +  LineReaderBuffered linereader;
    +  EXPECT_TRUE(linereader.bufferEmpty());
    +
    +  std::string stmt_1 = "select * from foo;";
    +  std::string stmts = stmt_1 + "select 1; select 2; quit;";
    +  linereader.setBuffer(stmts);
    +  ASSERT_FALSE(linereader.bufferEmpty());
    +
    +  std::vector<std::string> stmts_vec;
    +  int parsed_commands;
    +  for(parsed_commands = 0;
    +      parsed_commands < 10 && !linereader.bufferEmpty();
    +      parsed_commands++) {
    +    std::string command = linereader.getNextCommand();
    +    if (command != "") {
    +      stmts_vec.push_back(command);
    +    }
    +  }
    +
    +  EXPECT_EQ(stmt_1, stmts_vec.front());
    +  EXPECT_EQ(4, stmts_vec.size());
    +  EXPECT_TRUE(linereader.bufferEmpty());
    +}
    +*/
    --- End diff --
    
    ..well, actually I moved the commented out comments back into the unit test. Originally I had written this LRBuffered class, then removed it, placing the files somewhere safe. When I moved them back in, I forgot that I stashed the test in the file.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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

    https://github.com/apache/incubator-quickstep/pull/232#discussion_r112249998
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,61 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <string>
    +
    +/**
    + * Virtual base defines a generic, file-based interface around IO.
    + */
    +class IOInterface {
    + public:
    +  IOInterface() {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  virtual FILE* out() = 0;
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  virtual FILE* err() = 0;
    +
    +  /**
    +   * Requests a complete SQL command. This call may block until user input is given.
    +   * @note When the command is complete, commandComplete() should be called so that certain implementations can clean
    +   *    up state related to sending the command.
    +   * @return A string containing a quickstep command.
    +   */
    +  virtual std::string nextCommand() = 0;
    +
    +  /**
    +   * Notifies the IO system that the previously acquired command is complete.
    +   */
    +  virtual void commandComplete() {}
    +
    +  /**
    +   * Notifies the IO system that quickstep is shutting down.
    +   */
    +  virtual void shutdown() {}
    --- End diff --
    
    Similar to earlier, how about ``notifyShutdown``?


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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

    https://github.com/apache/incubator-quickstep/pull/232#discussion_r112249519
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,61 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <string>
    +
    +/**
    + * Virtual base defines a generic, file-based interface around IO.
    + */
    +class IOInterface {
    + public:
    +  IOInterface() {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  virtual FILE* out() = 0;
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  virtual FILE* err() = 0;
    +
    +  /**
    +   * Requests a complete SQL command. This call may block until user input is given.
    +   * @note When the command is complete, commandComplete() should be called so that certain implementations can clean
    +   *    up state related to sending the command.
    +   * @return A string containing a quickstep command.
    +   */
    +  virtual std::string nextCommand() = 0;
    --- End diff --
    
    Would ``getNextCommand`` be more intuitive? 


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112547224
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,227 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/MemStream.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer (requestor/requestee)
    + * thread.
    + */
    +class RequestState {
    + public:
    +  RequestState() :
    +    request_ready_(false),
    +    response_ready_(false),
    +    request_buffer_(""),
    +    mutex_(),
    +    condition_() {}
    +
    +  /**
    +   * Notifies waiter that a piece of work has been created and added to the buffer.
    +   * @param to_consume The arguments for the consuming thread.
    +   */
    +  void requestReady(std::string to_consume) {
    +    request_ready_ = true;
    +    response_ready_ = false;
    +    request_buffer_ = to_consume;
    +    condition_.notify_one();
    +  }
    +
    +  /**
    +   * Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   */
    +  void responseReady() {
    +    request_ready_ = false;
    +    response_ready_ = true;
    +    condition_.notify_one();
    +  }
    +
    +  bool request_ready_;
    +  bool response_ready_;
    +  std::string request_buffer_;
    +  QueryResponse response_message_;
    +  std::mutex mutex_;
    +  std::condition_variable condition_;
    +};
    +}  // namespace networkio_internal
    +
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +    : NetworkCli::Service(),
    +      worker_exclusive_mtx_(),
    +      running_(true),
    +      request_state_() { }
    +
    +  /**
    +   * Handles gRPC request. Sets the buffer in the RequestState, notifies the main thread, then waits for a response.
    +   * @param context
    +   * @param request
    +   * @param response
    +   * @return
    +   */
    +  Status SendQuery(grpc::ServerContext *context,
    +                   const QueryRequest *request,
    +                   QueryResponse *response) override {
    +    auto service_ex_lock = enter();
    +    if (!service_ex_lock) {
    +      return Status::CANCELLED;
    +    }
    +
    +    // Service thread sets a notification for the main thread.
    +    // Because of service thread exclusivity, we have rights to this data structure.
    +    request_state_.requestReady(request->query());
    +
    +    // Service thread - main thread critical section:
    +    // Service thread waits for the buffered message response from the main thread. The main thread will set
    +    // consumer_ready_ when it is finished and released its exclusive lock on the communication data structure.
    +    std::unique_lock<std::mutex> lock(request_state_.mutex_);
    +    while (!request_state_.response_ready_)
    +      request_state_.condition_.wait(lock);
    +
    +    *response = request_state_.response_message_;
    +
    +    return Status::OK;
    +  }
    +
    +  void kill() {
    +    running_ = false;
    +  }
    +
    +  networkio_internal::RequestState& getRequestState() {
    +    return request_state_;
    +  }
    +
    + private:
    +  /**
    +   * When a worker enters, it gains exclusive access to the main thread. That is, no other worker of this service
    +   * is allowed to interact with main thread.
    +   * @return A lock which grants the worker mutual exclusion.
    +   */
    +  std::unique_lock<std::mutex> enter() {
    +    std::unique_lock<std::mutex> lock(worker_exclusive_mtx_);
    +    if (!running_)
    +      lock.unlock();
    +    return lock;
    +  }
    +
    +  std::mutex worker_exclusive_mtx_;
    +  bool running_;
    +  networkio_internal::RequestState request_state_;
    +};
    +
    +class NetworkIO : public IOInterface {
    + public:
    +  NetworkIO()
    +      : IOInterface(),
    +        out_stream_(),
    +        err_stream_(),
    +        server_(nullptr),
    +        service_() {
    +    std::string server_address("0.0.0.0:" + std::to_string(FLAGS_port));
    +
    +    // Starts a server.
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(server_address, grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    LOG(INFO) << "Listening on port " << std::to_string(FLAGS_port);
    +  }
    +
    +  FILE* out() override {
    +    return out_stream_.file();
    +  }
    +
    +  FILE* err() override {
    +    return err_stream_.file();
    +  }
    +
    +  std::string getNextCommand() override {
    +    // critical section: wait for a command
    +    networkio_internal::RequestState& request_state = service_.getRequestState();
    --- End diff --
    
    Change to `networkio_internal::RequestState &request_state`.
    
    Ditto in `notifyCommandComplete`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819512
  
    --- Diff: cli/LocalIO.hpp ---
    @@ -0,0 +1,88 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LOCAL_IO_HPP_
    +#define QUICKSTEP_CLI_LOCAL_IO_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "cli/CliConfig.h"
    +#include "cli/IOInterface.hpp"
    +#include "cli/LineReader.hpp"
    +#ifdef QUICKSTEP_USE_LINENOISE
    +#include "cli/LineReaderLineNoise.hpp"
    +typedef quickstep::LineReaderLineNoise LineReaderImpl;
    +#else
    +#include "cli/LineReaderDumb.hpp"
    +typedef quickstep::LineReaderDumb LineReaderImpl;
    +#endif
    +#include "utility/Macros.hpp"
    +
    +namespace quickstep {
    +
    +class LocalIOHandle : public IOHandle {
    + public:
    +  LocalIOHandle() : IOHandle(), command_("") {}
    +
    +  explicit LocalIOHandle(std::string const & command)
    +      : IOHandle(),
    --- End diff --
    
    No needs for `IOHandle()`; implicitly called.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112417941
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,227 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/MemStream.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer (requestor/requestee)
    + * thread.
    + */
    +class RequestState {
    + public:
    +  RequestState() :
    +    request_ready_(false),
    +    response_ready_(false),
    +    request_buffer_(""),
    +    mutex_(),
    +    condition_() {}
    +
    +  /**
    +   * Notifies waiter that a piece of work has been created and added to the buffer.
    +   * @param to_consume The arguments for the consuming thread.
    +   */
    +  void requestReady(std::string to_consume) {
    +    request_ready_ = true;
    +    response_ready_ = false;
    +    request_buffer_ = to_consume;
    +    condition_.notify_one();
    +  }
    +
    +  /**
    +   * Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   */
    +  void responseReady() {
    +    request_ready_ = false;
    +    response_ready_ = true;
    +    condition_.notify_one();
    +  }
    +
    +  bool request_ready_;
    +  bool response_ready_;
    +  std::string request_buffer_;
    +  QueryResponse response_message_;
    +  std::mutex mutex_;
    +  std::condition_variable condition_;
    +};
    +}  // namespace networkio_internal
    +
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +    : NetworkCli::Service(),
    +      worker_exclusive_mtx_(),
    +      running_(true),
    +      request_state_() { }
    --- End diff --
    
    No needs for `worker_exclusive_mtx_` and `request_state_`. And remove the whitespace 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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112549890
  
    --- Diff: cli/tests/NetworkIO_unittest.cpp ---
    @@ -0,0 +1,182 @@
    +/**
    + * 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 <cstddef>
    +#include <cstdio>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +#include "cli/NetworkIO.hpp"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "threading/Thread.hpp"
    +
    +#include "glog/logging.h"
    +#include "gtest/gtest.h"
    +
    +using std::unique_ptr;
    +
    +namespace quickstep {
    +
    +static std::string const kQueryRequest = "O Captain! My Captain!";
    +static std::string const kQueryResponse = "Our fearful trip is done,";
    +
    +/**
    + * Contains a server instance for testing.
    + */
    +class TestNetworkIO {
    + public:
    +  TestNetworkIO()
    +      : service_(),
    +        server_address_("localhost:" + std::to_string(FLAGS_port)),
    +        server_(nullptr) {
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(server_address_, grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    CHECK(server_) << "Failed to start server";
    +    LOG(INFO) << "TestSingleNodeServer listening on " << server_address_;
    +  }
    +
    +  // Gets a message from the Service worker.
    +  std::string getSentMessage() {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    +    std::unique_lock<std::mutex> lock(requestState.mutex_);
    +    while (!requestState.request_ready_)
    +      requestState.condition_.wait(lock);
    +
    +    EXPECT_EQ(requestState.request_ready_, true);
    +    EXPECT_EQ(requestState.response_ready_, false);
    +
    +    return requestState.request_buffer_;
    +  }
    +
    +  // Sets the response message of the Service worker. Alerts it that the request is ready.
    +  void setResponse(std::string response) {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    +    requestState.response_message_.set_query_result(response);
    +
    +    EXPECT_EQ(requestState.response_ready_, false);
    +
    +    requestState.responseReady();
    +  }
    +
    +  NetworkCliServiceImpl& getService() {
    +    return service_;
    +  }
    +
    +  ~TestNetworkIO() {
    +    service_.kill();
    +    server_->Shutdown();
    +    server_->Wait();
    +  }
    +
    + private:
    +  NetworkCliServiceImpl service_;
    +  std::string server_address_;
    +  std::unique_ptr<grpc::Server> server_;
    +};
    +
    +/**
    + * Tests that killing the service will cancel requests.
    + */
    +TEST(NetworkIOTest, TestShutdown) {
    +  TestNetworkIO server;
    +
    +  server.getService().kill();
    +
    +  NetworkCliClient client(
    +    grpc::CreateChannel("localhost:" + std::to_string(FLAGS_port),
    +                        grpc::InsecureChannelCredentials()));
    +  QueryRequest request;
    +  request.set_query(kQueryRequest);
    +  QueryResponse response;
    +  Status status = client.SendQuery(request, &response);
    +  ASSERT_EQ(status.error_code(), grpc::CANCELLED);
    +}
    +
    +/**
    + * Tests a simple call and response to the Service.
    + */
    +TEST(NetworkIOTest, TestNetworkIOCommandInteraction) {
    +  NetworkIO networkIO;
    +  std::string const query_stmt = kQueryRequest + ";" + kQueryRequest;
    +
    +  // This thread will handle the response from the client in a similar way as the quickstep cli will.
    +  std::thread server_handler([&networkIO, &query_stmt]() {
    +    std::string command = networkIO.getNextCommand();
    +    EXPECT_EQ(command, query_stmt);
    +
    +    // Set some output for the main test thread, return.
    +    fprintf(networkIO.out(), "%s", kQueryResponse.c_str());
    +    networkIO.notifyCommandComplete();
    +  });
    +
    +  NetworkCliClient client(
    +    grpc::CreateChannel("localhost:" + std::to_string(FLAGS_port),
    +                        grpc::InsecureChannelCredentials()));
    +  QueryRequest request;
    +  request.set_query(query_stmt);
    +  QueryResponse response;
    +  Status status = client.SendQuery(request, &response);
    +  ASSERT_TRUE(status.ok());
    +  EXPECT_EQ(response.query_result(), kQueryResponse);
    +  EXPECT_EQ(response.error_result(), std::string(""));
    --- End diff --
    
    Change to
    
    ```
    EXPECT_EQ(kQueryResponse, response.query_result());
    EXPECT_TRUE(response.error_result().empty());
    ```


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819629
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,286 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <cstdio>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/Macros.hpp"
    +#include "utility/MemStream.hpp"
    +#include "utility/ThreadSafeQueue.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
    + * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
    + */
    +class RequestState {
    + public:
    +  explicit RequestState(QueryRequest const *request)
    +      : response_ready_(false),
    +        canceled_(false),
    +        request_(request),
    +        condition_(mutex_.createConditionVariable()) {}
    +
    +  /**
    +   * @brief Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   * @param stdout_str Stdout from Quickstep.
    +   * @param stderr_str Stderr from Quickstep.
    +   */
    +  void responseReady(std::string stdout_str, std::string stderr_str) {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    response_message_.set_query_result(stdout_str);
    +    response_message_.set_error_result(stderr_str);
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @brief The producer thread blocks until Quickstep signals that it has finished.
    +   * @note Quickstep may either produce a query response or cancel. Both these actions must notify the condition.
    +   */
    +  void waitForResponse() {
    +    while (!response_ready_)
    +      condition_->await();
    +  }
    +
    +  /**
    +   * @brief Notifies the producer that its request will not be served by Quickstep.
    +   */
    +  void cancel() {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    canceled_ = true;
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @return The producer's query for Quickstep to process.
    +   */
    +  std::string getRequest() const {
    +    return request_->query();
    +  }
    +
    +  /**
    +   * @return The response message from Quickstep.
    +   */
    +  QueryResponse getResponse() const {
    +    DCHECK(response_ready_);
    +    return response_message_;
    +  }
    +
    +  /**
    +   * @return True if query was canceled.
    +   */
    +  bool getCanceled() const {
    +    DCHECK(response_ready_);
    +    return canceled_;
    +  }
    +
    + private:
    +  bool response_ready_;
    +  bool canceled_;
    +  QueryRequest const * request_;
    +  QueryResponse response_message_;
    +  Mutex mutex_;
    +  ConditionVariable *condition_;  // note: owned by the mutex.
    +
    +  DISALLOW_COPY_AND_ASSIGN(RequestState);
    +};
    +
    +}  // namespace networkio_internal
    +
    +using networkio_internal::RequestState;
    +
    +/**
    + * @brief Contains the callback methods which the gRPC service defines.
    + * When a request is made of gRPC, a gRPC worker thread is spun up and enters one of the callback methods
    + * (eg SendQuery). This thread keeps the network connection open while Quickstep processes the query. Concurrency
    + * control between the gRPC worker thread, and the Quickstep thread is controlled by a RequestState object which is
    + * created for each interaction.
    + */
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +      : running_(true) {}
    +
    +  /**
    +   * @brief Handles gRPC request.
    +   * Sets the buffer in the RequestState, places the request on a queue, then waits for a response. The response shall
    +   * be triggered by a
    +   */
    +  Status SendQuery(grpc::ServerContext *context,
    +                   const QueryRequest *request,
    +                   QueryResponse *response) override {
    +    std::unique_ptr<RequestState> request_state;
    +    // Check to see if the gRPC service has been shutdown.
    +    {
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      if (!running_) {
    +        return Status::CANCELLED;
    +      }
    +      // While we have a service lock, we add to the Queue. Note that we keep the service lock to protect ourselves from
    +      // a race condition in the kill() method.
    +      request_state.reset(new RequestState(request));
    +
    +      // Pushing to the queue will notify consumers.
    +      request_queue_.push(request_state.get());
    +    }
    +    DCHECK(request_state);
    +
    +    // We have pushed to the request queue, so all there is to do now is wait for Quickstep to process the request.
    +    request_state->waitForResponse();
    +    if (request_state->getCanceled()) {
    +      return Status::CANCELLED;
    +    }
    +    *response = request_state->getResponse();
    +    return Status::OK;
    +  }
    +
    +  /**
    +   * @brief The consumer thread waits for a request to materialize.
    +   * @return A non-owned RequestState.
    +   */
    +  RequestState* waitForRequest() {
    +    return request_queue_.popOne();
    +  }
    +
    +  /**
    +   * @brief Stops accepting further requests and cancels all pending requests.
    +   */
    +  void kill() {
    +    {
    +      // This action guarantees that no further requests are added to the queue.
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      running_ = false;
    +    }
    +    // Go through each pending request, and cancel them.
    +    while (!request_queue_.empty()) {
    +      request_queue_.popOne()->cancel();
    +    }
    +  }
    +
    + private:
    +  Mutex service_mtx_;
    +  bool running_;
    +  ThreadSafeQueue<RequestState*> request_queue_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkCliServiceImpl);
    +};
    +
    +class NetworkIOHandle : public IOHandle {
    + public:
    +  explicit NetworkIOHandle(RequestState* state)
    +      : request_state_(state) {}
    +
    +  ~NetworkIOHandle() override {
    +    // All the commands from the last network interaction have completed, return our response.
    +    // This signals to the producer thread that the interaction is complete.
    +    request_state_->responseReady(out_stream_.str(), err_stream_.str());
    +  }
    +
    +  FILE* out() override {
    +    return out_stream_.file();
    +  }
    +
    +  FILE* err() override {
    +    return err_stream_.file();
    +  }
    +
    +  std::string getCommand() override {
    +    return request_state_->getRequest();
    +  }
    +
    + private:
    +  MemStream out_stream_, err_stream_;
    +  RequestState* request_state_;
    --- End diff --
    
    Change to `RequestState *request_state_;`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819472
  
    --- Diff: cli/LineReaderBuffered.cpp ---
    @@ -0,0 +1,39 @@
    +/**
    + * 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 "cli/LineReaderBuffered.hpp"
    +
    +#include <string>
    +
    +namespace quickstep {
    +
    +LineReaderBuffered::LineReaderBuffered(const std::string &default_prompt,
    +                                       const std::string &continue_prompt)
    +    : LineReader(default_prompt, continue_prompt),
    +      buffer_empty_(true) {}
    +
    +LineReaderBuffered::LineReaderBuffered() : LineReader("", ""), buffer_empty_(true) { }
    --- End diff --
    
    Remove the whitespace 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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112407047
  
    --- Diff: cli/LocalIO.hpp ---
    @@ -0,0 +1,74 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LOCAL_IO_HPP_
    +#define QUICKSTEP_CLI_LOCAL_IO_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "cli/CliConfig.h"
    +#include "cli/IOInterface.hpp"
    +#include "cli/LineReader.hpp"
    +
    +#ifdef QUICKSTEP_USE_LINENOISE
    +#include "cli/LineReaderLineNoise.hpp"
    +typedef quickstep::LineReaderLineNoise LineReaderImpl;
    +#else
    +#include "cli/LineReaderDumb.hpp"
    +typedef quickstep::LineReaderDumb LineReaderImpl;
    +#endif
    +
    +namespace quickstep {
    +
    +class LocalIO : public IOInterface {
    + public:
    +  LocalIO() : IOInterface(), line_reader_("quickstep> ", "      ...> ") {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    --- End diff --
    
    No duplicated comments for override methods. Remove all 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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112835877
  
    --- Diff: cli/Flags.cpp ---
    @@ -87,4 +87,25 @@ DEFINE_bool(preload_buffer_pool, false,
                 "accepting queries (should also set --buffer_pool_slots to be "
                 "large enough to accomodate the entire database).");
     
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  int const min = 0, max = 65536;
    +  if (value > min && value < max) {
    +    return true;
    +  }
    +  std::cout << "Invalid value for --" << flagname << ": " << value
    +            << "\nUse ports between " << min << " and "
    +            << max << std::endl;
    +  return false;
    +}
    +
    +DEFINE_int32(cli_network_port, 3000,
    +             "Listens for TCP connections on this port when network mode is enabled. "
    +             "This is only used if the cli is set to use the network mode (--mode=network).");
    +DEFINE_validator(cli_network_port, &ValidatePort);
    +
    +DEFINE_string(cli_network_ip, "localhost",
    --- End diff --
    
    Also the CliClientMain uses the flag so I think they are appropriate to be kept 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] incubator-quickstep pull request #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112417374
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,227 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/MemStream.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer (requestor/requestee)
    + * thread.
    + */
    +class RequestState {
    + public:
    +  RequestState() :
    +    request_ready_(false),
    +    response_ready_(false),
    +    request_buffer_(""),
    +    mutex_(),
    +    condition_() {}
    +
    +  /**
    +   * Notifies waiter that a piece of work has been created and added to the buffer.
    +   * @param to_consume The arguments for the consuming thread.
    +   */
    +  void requestReady(std::string to_consume) {
    +    request_ready_ = true;
    +    response_ready_ = false;
    +    request_buffer_ = to_consume;
    +    condition_.notify_one();
    +  }
    +
    +  /**
    +   * Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   */
    +  void responseReady() {
    +    request_ready_ = false;
    +    response_ready_ = true;
    +    condition_.notify_one();
    +  }
    +
    +  bool request_ready_;
    +  bool response_ready_;
    +  std::string request_buffer_;
    +  QueryResponse response_message_;
    +  std::mutex mutex_;
    +  std::condition_variable condition_;
    +};
    +}  // namespace networkio_internal
    +
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +    : NetworkCli::Service(),
    --- End diff --
    
    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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112414574
  
    --- Diff: CMakeLists.txt ---
    @@ -792,3 +797,14 @@ if (ENABLE_DISTRIBUTED)
                             ${GFLAGS_LIB_NAME}
                             ${GRPCPLUSPLUS_LIBRARIES})
     endif(ENABLE_DISTRIBUTED)
    +
    +if(ENABLE_NETWORK_CLI)
    +  add_executable (quickstep_client cli/NetworkCliClientMain.cpp)
    +  target_link_libraries(quickstep_client
    +            ${GFLAGS_LIB_NAME}
    --- End diff --
    
    We should align with `(` in the line above.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819636
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,286 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <cstdio>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/Macros.hpp"
    +#include "utility/MemStream.hpp"
    +#include "utility/ThreadSafeQueue.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
    + * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
    + */
    +class RequestState {
    + public:
    +  explicit RequestState(QueryRequest const *request)
    +      : response_ready_(false),
    +        canceled_(false),
    +        request_(request),
    +        condition_(mutex_.createConditionVariable()) {}
    +
    +  /**
    +   * @brief Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   * @param stdout_str Stdout from Quickstep.
    +   * @param stderr_str Stderr from Quickstep.
    +   */
    +  void responseReady(std::string stdout_str, std::string stderr_str) {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    response_message_.set_query_result(stdout_str);
    +    response_message_.set_error_result(stderr_str);
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @brief The producer thread blocks until Quickstep signals that it has finished.
    +   * @note Quickstep may either produce a query response or cancel. Both these actions must notify the condition.
    +   */
    +  void waitForResponse() {
    +    while (!response_ready_)
    +      condition_->await();
    +  }
    +
    +  /**
    +   * @brief Notifies the producer that its request will not be served by Quickstep.
    +   */
    +  void cancel() {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    canceled_ = true;
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @return The producer's query for Quickstep to process.
    +   */
    +  std::string getRequest() const {
    +    return request_->query();
    +  }
    +
    +  /**
    +   * @return The response message from Quickstep.
    +   */
    +  QueryResponse getResponse() const {
    +    DCHECK(response_ready_);
    +    return response_message_;
    +  }
    +
    +  /**
    +   * @return True if query was canceled.
    +   */
    +  bool getCanceled() const {
    +    DCHECK(response_ready_);
    +    return canceled_;
    +  }
    +
    + private:
    +  bool response_ready_;
    +  bool canceled_;
    +  QueryRequest const * request_;
    +  QueryResponse response_message_;
    +  Mutex mutex_;
    +  ConditionVariable *condition_;  // note: owned by the mutex.
    +
    +  DISALLOW_COPY_AND_ASSIGN(RequestState);
    +};
    +
    +}  // namespace networkio_internal
    +
    +using networkio_internal::RequestState;
    +
    +/**
    + * @brief Contains the callback methods which the gRPC service defines.
    + * When a request is made of gRPC, a gRPC worker thread is spun up and enters one of the callback methods
    + * (eg SendQuery). This thread keeps the network connection open while Quickstep processes the query. Concurrency
    + * control between the gRPC worker thread, and the Quickstep thread is controlled by a RequestState object which is
    + * created for each interaction.
    + */
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +      : running_(true) {}
    +
    +  /**
    +   * @brief Handles gRPC request.
    +   * Sets the buffer in the RequestState, places the request on a queue, then waits for a response. The response shall
    +   * be triggered by a
    +   */
    +  Status SendQuery(grpc::ServerContext *context,
    +                   const QueryRequest *request,
    +                   QueryResponse *response) override {
    +    std::unique_ptr<RequestState> request_state;
    +    // Check to see if the gRPC service has been shutdown.
    +    {
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      if (!running_) {
    +        return Status::CANCELLED;
    +      }
    +      // While we have a service lock, we add to the Queue. Note that we keep the service lock to protect ourselves from
    +      // a race condition in the kill() method.
    +      request_state.reset(new RequestState(request));
    +
    +      // Pushing to the queue will notify consumers.
    +      request_queue_.push(request_state.get());
    +    }
    +    DCHECK(request_state);
    +
    +    // We have pushed to the request queue, so all there is to do now is wait for Quickstep to process the request.
    +    request_state->waitForResponse();
    +    if (request_state->getCanceled()) {
    +      return Status::CANCELLED;
    +    }
    +    *response = request_state->getResponse();
    +    return Status::OK;
    +  }
    +
    +  /**
    +   * @brief The consumer thread waits for a request to materialize.
    +   * @return A non-owned RequestState.
    +   */
    +  RequestState* waitForRequest() {
    +    return request_queue_.popOne();
    +  }
    +
    +  /**
    +   * @brief Stops accepting further requests and cancels all pending requests.
    +   */
    +  void kill() {
    +    {
    +      // This action guarantees that no further requests are added to the queue.
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      running_ = false;
    +    }
    +    // Go through each pending request, and cancel them.
    +    while (!request_queue_.empty()) {
    +      request_queue_.popOne()->cancel();
    +    }
    +  }
    +
    + private:
    +  Mutex service_mtx_;
    +  bool running_;
    +  ThreadSafeQueue<RequestState*> request_queue_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkCliServiceImpl);
    +};
    +
    +class NetworkIOHandle : public IOHandle {
    + public:
    +  explicit NetworkIOHandle(RequestState* state)
    +      : request_state_(state) {}
    +
    +  ~NetworkIOHandle() override {
    +    // All the commands from the last network interaction have completed, return our response.
    +    // This signals to the producer thread that the interaction is complete.
    +    request_state_->responseReady(out_stream_.str(), err_stream_.str());
    +  }
    +
    +  FILE* out() override {
    +    return out_stream_.file();
    +  }
    +
    +  FILE* err() override {
    +    return err_stream_.file();
    +  }
    +
    +  std::string getCommand() override {
    +    return request_state_->getRequest();
    +  }
    +
    + private:
    +  MemStream out_stream_, err_stream_;
    +  RequestState* request_state_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkIOHandle);
    +};
    +
    +/**
    + * A network interface that uses gRPC to accept commands.
    + */
    +class NetworkIO : public IOInterface {
    + public:
    +  NetworkIO() {
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(GetAddress(), grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    LOG(INFO) << "Listening on " << GetAddress();
    +  }
    +
    +  ~NetworkIO() {
    --- End diff --
    
    Add `override`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112324538
  
    --- Diff: cli/LineReaderBuffered.hpp ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +#define QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +
    +#include "cli/LineReader.hpp"
    +
    +namespace quickstep {
    +
    +class LineReaderBuffered : public LineReader {
    --- End diff --
    
    I agree, though I was trying to keep consistency with the 2 other LR's: LRLineNoise, LRDumb


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112406425
  
    --- Diff: cli/LineReaderBuffered.hpp ---
    @@ -0,0 +1,73 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +#define QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +
    +#include <string>
    +
    +#include "cli/LineReader.hpp"
    +
    +namespace quickstep {
    +
    +class LineReaderBuffered : public LineReader {
    + public:
    +  /**
    +   * @brief A line reader which accepts a string buffer.
    +   * Other line readers are meant to support some form of user interaction. This linereader does not and is intended for
    +   * programmer use- it does not print anything to stdout. Therefore it ignores any prompt strings given to the
    +   * inherited constructor.
    +   * @param default_prompt Not used by this line reader, but required by interface.
    +   * @param continue_prompt Not used by this line reader, but required by interface.
    +   */
    +  LineReaderBuffered(const std::string &default_prompt,
    +                     const std::string &continue_prompt);
    +
    +  LineReaderBuffered();
    +
    +  ~LineReaderBuffered() {}
    --- End diff --
    
    Add `override` to indicate a derived class.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112836146
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,82 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "utility/Macros.hpp"
    +
    +namespace quickstep {
    +
    +/**
    + * An individual IO interaction with Quickstep.
    + */
    +class IOHandle {
    + public:
    +  IOHandle() {}
    +
    +  /**
    +   * @note Destructor should handle clean up of any IO state.
    +   */
    +  virtual ~IOHandle() {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  virtual FILE *out() = 0;
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  virtual FILE *err() = 0;
    +
    +  virtual std::string getCommand() = 0;
    --- End diff --
    
    Calling `getCommand` could cause the interface to change state (pop an IOHandle off a queue, for example)


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819528
  
    --- Diff: cli/LocalIO.hpp ---
    @@ -0,0 +1,88 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LOCAL_IO_HPP_
    +#define QUICKSTEP_CLI_LOCAL_IO_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "cli/CliConfig.h"
    +#include "cli/IOInterface.hpp"
    +#include "cli/LineReader.hpp"
    +#ifdef QUICKSTEP_USE_LINENOISE
    +#include "cli/LineReaderLineNoise.hpp"
    +typedef quickstep::LineReaderLineNoise LineReaderImpl;
    +#else
    +#include "cli/LineReaderDumb.hpp"
    +typedef quickstep::LineReaderDumb LineReaderImpl;
    +#endif
    +#include "utility/Macros.hpp"
    +
    +namespace quickstep {
    +
    +class LocalIOHandle : public IOHandle {
    + public:
    +  LocalIOHandle() : IOHandle(), command_("") {}
    +
    +  explicit LocalIOHandle(std::string const & command)
    +      : IOHandle(),
    +        command_(command) {}
    +
    +  ~LocalIOHandle() {}
    +
    +  FILE *out() override {
    +    return stdout;
    +  }
    +
    +  FILE *err() override {
    +    return stderr;
    +  }
    +
    +  std::string getCommand() override {
    +    return command_;
    +  }
    +
    + private:
    +  std::string command_;
    --- End diff --
    
    Could we mark `const`?


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r113041261
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,62 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <string>
    +
    +/**
    + * Virtual base defines a generic, file-based interface around IO.
    + */
    +class IOInterface {
    + public:
    +  IOInterface() {}
    +
    --- End diff --
    
    I removed the constructor but ran into a compiler error:
    ```
    /Users/cramja/workspace/quickstep/incubator-quickstep/cli/NetworkIO.cpp:50:12: error: constructor for 'quickstep::NetworkIO' must explicitly initialize the base class 'quickstep::IOInterface' which does not have a default constructor
    NetworkIO::NetworkIO() {
               ^
    /Users/cramja/workspace/quickstep/incubator-quickstep/cli/IOInterface.hpp:62:7: note: 'quickstep::IOInterface' declared here
    class IOInterface {
          ^
    ```
    On doing the suggestion in the output, that too gave a compiler error.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112407163
  
    --- Diff: cli/LineReaderBuffered.hpp ---
    @@ -0,0 +1,73 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +#define QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +
    +#include <string>
    +
    +#include "cli/LineReader.hpp"
    +
    +namespace quickstep {
    +
    +class LineReaderBuffered : public LineReader {
    + public:
    +  /**
    +   * @brief A line reader which accepts a string buffer.
    +   * Other line readers are meant to support some form of user interaction. This linereader does not and is intended for
    +   * programmer use- it does not print anything to stdout. Therefore it ignores any prompt strings given to the
    +   * inherited constructor.
    +   * @param default_prompt Not used by this line reader, but required by interface.
    +   * @param continue_prompt Not used by this line reader, but required by interface.
    +   */
    +  LineReaderBuffered(const std::string &default_prompt,
    +                     const std::string &continue_prompt);
    +
    +  LineReaderBuffered();
    +
    +  ~LineReaderBuffered() {}
    +
    +  /**
    +   * @brief Replaces the current buffer contents with new contents.
    +   * @param buffer Replacement text.
    +   */
    +  void setBuffer(std::string buffer) {
    +    leftover_ = buffer;
    +    buffer_empty_ = false;
    +  }
    +
    +  /**
    +   * @brief This is set to true after getNextCommand is called and runs out of input to process.
    +   * @return True if the buffer has been consumed.
    +   */
    +  bool bufferEmpty() {
    +    return buffer_empty_;
    +  }
    +
    + protected:
    +  std::string getLineInternal(const bool continuing) override;
    +
    + private:
    +  bool buffer_empty_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(LineReaderBuffered);
    +};
    +}  // namespace quickstep
    --- End diff --
    
    Add an empty line above.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819519
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,82 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <cstdio>
    +#include <string>
    +
    +#include "utility/Macros.hpp"
    +
    +namespace quickstep {
    +
    +/**
    + * An individual IO interaction with Quickstep.
    + */
    +class IOHandle {
    + public:
    +  IOHandle() {}
    +
    +  /**
    +   * @note Destructor should handle clean up of any IO state.
    +   */
    +  virtual ~IOHandle() {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  virtual FILE *out() = 0;
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  virtual FILE *err() = 0;
    +
    +  virtual std::string getCommand() = 0;
    --- End diff --
    
    Could we mark `getCommand` `const`?


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112411604
  
    --- Diff: cli/NetworkCliClientMain.cpp ---
    @@ -0,0 +1,62 @@
    +/**
    + * 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 <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <istream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +
    +#include "gflags/gflags.h"
    +
    +DECLARE_int32(port);
    +
    +using quickstep::LineReaderBuffered;
    +using quickstep::NetworkCliClient;
    +
    +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.
    +  NetworkCliClient qs_client(
    +    grpc::CreateChannel("localhost:" + std::to_string(quickstep::FLAGS_port),
    +                        grpc::InsecureChannelCredentials()));
    +
    +  // Read stdin until EOF, then we use a Line reader to divide query into parts.
    +  std::cin >> std::noskipws;
    +  std::istream_iterator<char> it(std::cin), end;
    +  std::string user_queries(it, end);
    +
    +  LineReaderBuffered linereader;
    +  linereader.setBuffer(user_queries);
    +  while (!linereader.bufferEmpty()) {
    +    std::string query = linereader.getNextCommand();
    +    if (query.size() > 1) {
    --- End diff --
    
    Change to `!query.empty()`?


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112825394
  
    --- Diff: cli/LineReaderBuffered.hpp ---
    @@ -0,0 +1,75 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +#define QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
    +
    +#include <string>
    +
    +#include "cli/LineReader.hpp"
    +#include "utility/Macros.hpp"
    +
    +namespace quickstep {
    +
    +class LineReaderBuffered : public LineReader {
    + public:
    +  /**
    +   * @brief A line reader which accepts a string buffer.
    +   * Other line readers are meant to support some form of user interaction. This linereader does not and is intended for
    +   * programmer use- it does not print anything to stdout. Therefore it ignores any prompt strings given to the
    +   * inherited constructor.
    +   * @param default_prompt Not used by this line reader, but required by interface.
    +   * @param continue_prompt Not used by this line reader, but required by interface.
    +   */
    +  LineReaderBuffered(const std::string &default_prompt,
    +                     const std::string &continue_prompt);
    +
    +  LineReaderBuffered();
    +
    +  ~LineReaderBuffered() override {}
    +
    +  /**
    +   * @brief Replaces the current buffer contents with new contents.
    +   * @param buffer Replacement text.
    +   */
    +  void setBuffer(std::string buffer) {
    +    leftover_ = buffer;
    +    buffer_empty_ = false;
    +  }
    +
    +  /**
    +   * @brief This is set to true after getNextCommand is called and runs out of input to process.
    +   * @return True if the buffer has been consumed.
    +   */
    +  bool bufferEmpty() {
    --- End diff --
    
    Mark `const`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112837404
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,286 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <cstdio>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/Macros.hpp"
    +#include "utility/MemStream.hpp"
    +#include "utility/ThreadSafeQueue.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
    + * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
    + */
    +class RequestState {
    + public:
    +  explicit RequestState(QueryRequest const *request)
    +      : response_ready_(false),
    +        canceled_(false),
    +        request_(request),
    +        condition_(mutex_.createConditionVariable()) {}
    +
    +  /**
    +   * @brief Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   * @param stdout_str Stdout from Quickstep.
    +   * @param stderr_str Stderr from Quickstep.
    +   */
    +  void responseReady(std::string stdout_str, std::string stderr_str) {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    response_message_.set_query_result(stdout_str);
    +    response_message_.set_error_result(stderr_str);
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @brief The producer thread blocks until Quickstep signals that it has finished.
    +   * @note Quickstep may either produce a query response or cancel. Both these actions must notify the condition.
    +   */
    +  void waitForResponse() {
    +    while (!response_ready_)
    +      condition_->await();
    +  }
    +
    +  /**
    +   * @brief Notifies the producer that its request will not be served by Quickstep.
    +   */
    +  void cancel() {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    canceled_ = true;
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @return The producer's query for Quickstep to process.
    +   */
    +  std::string getRequest() const {
    +    return request_->query();
    +  }
    +
    +  /**
    +   * @return The response message from Quickstep.
    +   */
    +  QueryResponse getResponse() const {
    +    DCHECK(response_ready_);
    +    return response_message_;
    +  }
    +
    +  /**
    +   * @return True if query was canceled.
    +   */
    +  bool getCanceled() const {
    +    DCHECK(response_ready_);
    +    return canceled_;
    +  }
    +
    + private:
    +  bool response_ready_;
    +  bool canceled_;
    +  QueryRequest const * request_;
    +  QueryResponse response_message_;
    +  Mutex mutex_;
    +  ConditionVariable *condition_;  // note: owned by the mutex.
    +
    +  DISALLOW_COPY_AND_ASSIGN(RequestState);
    +};
    +
    +}  // namespace networkio_internal
    +
    +using networkio_internal::RequestState;
    +
    +/**
    + * @brief Contains the callback methods which the gRPC service defines.
    + * When a request is made of gRPC, a gRPC worker thread is spun up and enters one of the callback methods
    + * (eg SendQuery). This thread keeps the network connection open while Quickstep processes the query. Concurrency
    + * control between the gRPC worker thread, and the Quickstep thread is controlled by a RequestState object which is
    + * created for each interaction.
    + */
    +class NetworkCliServiceImpl final : public NetworkCli::Service {
    + public:
    +  NetworkCliServiceImpl()
    +      : running_(true) {}
    +
    +  /**
    +   * @brief Handles gRPC request.
    +   * Sets the buffer in the RequestState, places the request on a queue, then waits for a response. The response shall
    +   * be triggered by a
    +   */
    +  Status SendQuery(grpc::ServerContext *context,
    +                   const QueryRequest *request,
    +                   QueryResponse *response) override {
    +    std::unique_ptr<RequestState> request_state;
    +    // Check to see if the gRPC service has been shutdown.
    +    {
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      if (!running_) {
    +        return Status::CANCELLED;
    +      }
    +      // While we have a service lock, we add to the Queue. Note that we keep the service lock to protect ourselves from
    +      // a race condition in the kill() method.
    +      request_state.reset(new RequestState(request));
    +
    +      // Pushing to the queue will notify consumers.
    +      request_queue_.push(request_state.get());
    +    }
    +    DCHECK(request_state);
    +
    +    // We have pushed to the request queue, so all there is to do now is wait for Quickstep to process the request.
    +    request_state->waitForResponse();
    +    if (request_state->getCanceled()) {
    +      return Status::CANCELLED;
    +    }
    +    *response = request_state->getResponse();
    +    return Status::OK;
    +  }
    +
    +  /**
    +   * @brief The consumer thread waits for a request to materialize.
    +   * @return A non-owned RequestState.
    +   */
    +  RequestState* waitForRequest() {
    +    return request_queue_.popOne();
    +  }
    +
    +  /**
    +   * @brief Stops accepting further requests and cancels all pending requests.
    +   */
    +  void kill() {
    +    {
    +      // This action guarantees that no further requests are added to the queue.
    +      std::unique_lock<Mutex> lock(service_mtx_);
    +      running_ = false;
    +    }
    +    // Go through each pending request, and cancel them.
    +    while (!request_queue_.empty()) {
    +      request_queue_.popOne()->cancel();
    +    }
    +  }
    +
    + private:
    +  Mutex service_mtx_;
    +  bool running_;
    +  ThreadSafeQueue<RequestState*> request_queue_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkCliServiceImpl);
    +};
    +
    +class NetworkIOHandle : public IOHandle {
    + public:
    +  explicit NetworkIOHandle(RequestState* state)
    +      : request_state_(state) {}
    +
    +  ~NetworkIOHandle() override {
    +    // All the commands from the last network interaction have completed, return our response.
    +    // This signals to the producer thread that the interaction is complete.
    +    request_state_->responseReady(out_stream_.str(), err_stream_.str());
    +  }
    +
    +  FILE* out() override {
    +    return out_stream_.file();
    +  }
    +
    +  FILE* err() override {
    +    return err_stream_.file();
    +  }
    +
    +  std::string getCommand() override {
    +    return request_state_->getRequest();
    +  }
    +
    + private:
    +  MemStream out_stream_, err_stream_;
    +  RequestState* request_state_;
    +
    +  DISALLOW_COPY_AND_ASSIGN(NetworkIOHandle);
    +};
    +
    +/**
    + * A network interface that uses gRPC to accept commands.
    + */
    +class NetworkIO : public IOInterface {
    + public:
    +  NetworkIO() {
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(GetAddress(), grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    LOG(INFO) << "Listening on " << GetAddress();
    +  }
    +
    +  ~NetworkIO() {
    +    service_.kill();
    +    server_->Shutdown();
    +    server_->Wait();
    +  }
    +
    +  IOHandle* getNextIOHandle() override {
    +    return new NetworkIOHandle(service_.waitForRequest());
    +  }
    +
    +  /**
    +   * @return The underlying service which interacts with gRPC.
    +   */
    +  NetworkCliServiceImpl& getService() {
    --- End diff --
    
    This was only being used by a test to kill the service so I changed it to:
    ```
      /**
       * @brief Kills the underlying gRPC service.
       */
      void killService() {
        service_.kill();
      }
    ```


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819680
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,286 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <cstdio>
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/Macros.hpp"
    +#include "utility/MemStream.hpp"
    +#include "utility/ThreadSafeQueue.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
    + * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
    + */
    +class RequestState {
    + public:
    +  explicit RequestState(QueryRequest const *request)
    +      : response_ready_(false),
    +        canceled_(false),
    +        request_(request),
    +        condition_(mutex_.createConditionVariable()) {}
    +
    +  /**
    +   * @brief Notifies that the consumer has finished consuming and that a response is ready.
    +   * To be called after the consumer has executed.
    +   * @param stdout_str Stdout from Quickstep.
    +   * @param stderr_str Stderr from Quickstep.
    +   */
    +  void responseReady(std::string stdout_str, std::string stderr_str) {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    response_message_.set_query_result(stdout_str);
    +    response_message_.set_error_result(stderr_str);
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @brief The producer thread blocks until Quickstep signals that it has finished.
    +   * @note Quickstep may either produce a query response or cancel. Both these actions must notify the condition.
    +   */
    +  void waitForResponse() {
    +    while (!response_ready_)
    +      condition_->await();
    +  }
    +
    +  /**
    +   * @brief Notifies the producer that its request will not be served by Quickstep.
    +   */
    +  void cancel() {
    +    std::unique_lock<Mutex> lock(mutex_);
    +    canceled_ = true;
    +    response_ready_ = true;
    +    condition_->signalOne();
    +  }
    +
    +  /**
    +   * @return The producer's query for Quickstep to process.
    +   */
    +  std::string getRequest() const {
    +    return request_->query();
    +  }
    +
    +  /**
    +   * @return The response message from Quickstep.
    +   */
    +  QueryResponse getResponse() const {
    +    DCHECK(response_ready_);
    +    return response_message_;
    +  }
    +
    +  /**
    +   * @return True if query was canceled.
    +   */
    +  bool getCanceled() const {
    +    DCHECK(response_ready_);
    +    return canceled_;
    +  }
    +
    + private:
    +  bool response_ready_;
    +  bool canceled_;
    +  QueryRequest const * request_;
    --- End diff --
    
    Change to `const QueryRequest *request_;`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112322839
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,61 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <string>
    +
    +/**
    + * Virtual base defines a generic, file-based interface around IO.
    + */
    +class IOInterface {
    + public:
    +  IOInterface() {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  virtual FILE* out() = 0;
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  virtual FILE* err() = 0;
    +
    +  /**
    +   * Requests a complete SQL command. This call may block until user input is given.
    +   * @note When the command is complete, commandComplete() should be called so that certain implementations can clean
    +   *    up state related to sending the command.
    +   * @return A string containing a quickstep command.
    +   */
    +  virtual std::string nextCommand() = 0;
    --- End diff --
    
    Yes, I think this makes sense


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112548387
  
    --- Diff: cli/tests/NetworkIO_unittest.cpp ---
    @@ -0,0 +1,182 @@
    +/**
    + * 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 <cstddef>
    +#include <cstdio>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +#include "cli/NetworkIO.hpp"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "threading/Thread.hpp"
    +
    +#include "glog/logging.h"
    +#include "gtest/gtest.h"
    +
    +using std::unique_ptr;
    +
    +namespace quickstep {
    +
    +static std::string const kQueryRequest = "O Captain! My Captain!";
    +static std::string const kQueryResponse = "Our fearful trip is done,";
    +
    +/**
    + * Contains a server instance for testing.
    + */
    +class TestNetworkIO {
    + public:
    +  TestNetworkIO()
    +      : service_(),
    +        server_address_("localhost:" + std::to_string(FLAGS_port)),
    +        server_(nullptr) {
    --- End diff --
    
    No needs for default constructors, including `service_` and `server_`.


---
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 #232: QUICKSTEP-87 Adds network cli interface.

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

    https://github.com/apache/incubator-quickstep/pull/232
  
    We need to sync up with the master branch before merge.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112836622
  
    --- Diff: cli/NetworkCliClientMain.cpp ---
    @@ -0,0 +1,62 @@
    +/**
    + * 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 <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <istream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +#include "cli/NetworkIO.hpp"
    +
    +
    +#include "gflags/gflags.h"
    +
    +using quickstep::LineReaderBuffered;
    +using quickstep::NetworkCliClient;
    +
    +int main(int argc, char **argv) {
    +  google::InitGoogleLogging(argv[0]);
    +  gflags::ParseCommandLineFlags(&argc, &argv, true);
    +
    --- End diff --
    
    Also added to QuickstepCli.cpp and the NetworkIO_unittest.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112549264
  
    --- Diff: cli/tests/NetworkIO_unittest.cpp ---
    @@ -0,0 +1,182 @@
    +/**
    + * 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 <cstddef>
    +#include <cstdio>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +#include "cli/NetworkIO.hpp"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "threading/Thread.hpp"
    +
    +#include "glog/logging.h"
    +#include "gtest/gtest.h"
    +
    +using std::unique_ptr;
    +
    +namespace quickstep {
    +
    +static std::string const kQueryRequest = "O Captain! My Captain!";
    +static std::string const kQueryResponse = "Our fearful trip is done,";
    +
    +/**
    + * Contains a server instance for testing.
    + */
    +class TestNetworkIO {
    + public:
    +  TestNetworkIO()
    +      : service_(),
    +        server_address_("localhost:" + std::to_string(FLAGS_port)),
    +        server_(nullptr) {
    +    grpc::ServerBuilder builder;
    +    builder.AddListeningPort(server_address_, grpc::InsecureServerCredentials());
    +    builder.RegisterService(&service_);
    +    server_ = builder.BuildAndStart();
    +    CHECK(server_) << "Failed to start server";
    +    LOG(INFO) << "TestSingleNodeServer listening on " << server_address_;
    +  }
    +
    +  // Gets a message from the Service worker.
    +  std::string getSentMessage() {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    +    std::unique_lock<std::mutex> lock(requestState.mutex_);
    +    while (!requestState.request_ready_)
    +      requestState.condition_.wait(lock);
    +
    +    EXPECT_EQ(requestState.request_ready_, true);
    +    EXPECT_EQ(requestState.response_ready_, false);
    +
    +    return requestState.request_buffer_;
    +  }
    +
    +  // Sets the response message of the Service worker. Alerts it that the request is ready.
    +  void setResponse(std::string response) {
    +    networkio_internal::RequestState& requestState = service_.getRequestState();
    +    requestState.response_message_.set_query_result(response);
    +
    +    EXPECT_EQ(requestState.response_ready_, false);
    +
    +    requestState.responseReady();
    +  }
    +
    +  NetworkCliServiceImpl& getService() {
    +    return service_;
    +  }
    +
    +  ~TestNetworkIO() {
    +    service_.kill();
    +    server_->Shutdown();
    +    server_->Wait();
    +  }
    +
    + private:
    +  NetworkCliServiceImpl service_;
    +  std::string server_address_;
    +  std::unique_ptr<grpc::Server> server_;
    +};
    +
    +/**
    + * Tests that killing the service will cancel requests.
    + */
    +TEST(NetworkIOTest, TestShutdown) {
    +  TestNetworkIO server;
    +
    +  server.getService().kill();
    +
    +  NetworkCliClient client(
    +    grpc::CreateChannel("localhost:" + std::to_string(FLAGS_port),
    +                        grpc::InsecureChannelCredentials()));
    +  QueryRequest request;
    +  request.set_query(kQueryRequest);
    +  QueryResponse response;
    +  Status status = client.SendQuery(request, &response);
    +  ASSERT_EQ(status.error_code(), grpc::CANCELLED);
    +}
    +
    +/**
    + * Tests a simple call and response to the Service.
    + */
    +TEST(NetworkIOTest, TestNetworkIOCommandInteraction) {
    +  NetworkIO networkIO;
    +  std::string const query_stmt = kQueryRequest + ";" + kQueryRequest;
    --- End diff --
    
    Do you mean `const std::string query_statement`?
    
    Also, why the second query is not ended with `;`?


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112819551
  
    --- Diff: cli/NetworkCliClientMain.cpp ---
    @@ -0,0 +1,62 @@
    +/**
    + * 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 <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <istream>
    +#include <memory>
    +#include <string>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/LineReaderBuffered.hpp"
    +#include "cli/NetworkCliClient.hpp"
    +#include "cli/NetworkIO.hpp"
    +
    +
    --- End diff --
    
    Remove one empty line.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112404840
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,62 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <string>
    --- End diff --
    
    We need the header file for `FILE`.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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

    https://github.com/apache/incubator-quickstep/pull/232#discussion_r112249411
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,61 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <string>
    +
    +/**
    + * Virtual base defines a generic, file-based interface around IO.
    + */
    +class IOInterface {
    + public:
    +  IOInterface() {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  virtual FILE* out() = 0;
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  virtual FILE* err() = 0;
    +
    +  /**
    +   * Requests a complete SQL command. This call may block until user input is given.
    +   * @note When the command is complete, commandComplete() should be called so that certain implementations can clean
    +   *    up state related to sending the command.
    +   * @return A string containing a quickstep command.
    +   */
    +  virtual std::string nextCommand() = 0;
    +
    +  /**
    +   * Notifies the IO system that the previously acquired command is complete.
    +   */
    +  virtual void commandComplete() {}
    +
    +  /**
    +   * Notifies the IO system that quickstep is shutting down.
    --- End diff --
    
    Missing the @ brief annotation


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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

    https://github.com/apache/incubator-quickstep/pull/232#discussion_r112249266
  
    --- Diff: cli/IOInterface.hpp ---
    @@ -0,0 +1,61 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
    +
    +#include <string>
    +
    +/**
    + * Virtual base defines a generic, file-based interface around IO.
    + */
    +class IOInterface {
    + public:
    +  IOInterface() {}
    +
    +  /**
    +   * @return A file handle for standard output.
    +   */
    +  virtual FILE* out() = 0;
    +
    +  /**
    +   * @return A file handle for error output.
    +   */
    +  virtual FILE* err() = 0;
    +
    +  /**
    +   * Requests a complete SQL command. This call may block until user input is given.
    --- End diff --
    
    Missing the @brief annotation 


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112406197
  
    --- Diff: cli/LineReaderBuffered.cpp ---
    @@ -0,0 +1,39 @@
    +/**
    + * 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 "cli/LineReaderBuffered.hpp"
    +
    +#include <iostream>
    +
    +namespace quickstep {
    +
    +LineReaderBuffered::LineReaderBuffered(const std::string &default_prompt,
    +                                       const std::string &continue_prompt)
    +  : LineReader("", ""),
    +    buffer_empty_(true) { }
    --- End diff --
    
    Remove this whitespace in `{ }`.
    
    Ditto 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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112835858
  
    --- Diff: cli/Flags.cpp ---
    @@ -87,4 +87,25 @@ DEFINE_bool(preload_buffer_pool, false,
                 "accepting queries (should also set --buffer_pool_slots to be "
                 "large enough to accomodate the entire database).");
     
    +static bool ValidatePort(const char *flagname, std::int32_t value) {
    +  int const min = 0, max = 65536;
    +  if (value > min && value < max) {
    +    return true;
    +  }
    +  std::cout << "Invalid value for --" << flagname << ": " << value
    +            << "\nUse ports between " << min << " and "
    +            << max << std::endl;
    +  return false;
    +}
    +
    +DEFINE_int32(cli_network_port, 3000,
    +             "Listens for TCP connections on this port when network mode is enabled. "
    +             "This is only used if the cli is set to use the network mode (--mode=network).");
    +DEFINE_validator(cli_network_port, &ValidatePort);
    +
    +DEFINE_string(cli_network_ip, "localhost",
    --- End diff --
    
    `0.0.0.0` is good. I originally tried `192.0.0.1` but that caused grpc to fail.


---
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 #232: QUICKSTEP-87 Adds network cli interfa...

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/232#discussion_r112417234
  
    --- Diff: cli/NetworkIO.hpp ---
    @@ -0,0 +1,227 @@
    +/**
    + * 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.
    + **/
    +
    +#ifndef QUICKSTEP_CLI_NETWORK_IO_HPP_
    +#define QUICKSTEP_CLI_NETWORK_IO_HPP_
    +
    +#include <grpc++/grpc++.h>
    +
    +#include <iostream>
    +#include <memory>
    +#include <string>
    +#include <vector>
    +
    +#include "cli/Flags.hpp"
    +#include "cli/IOInterface.hpp"
    +#include "cli/NetworkCli.grpc.pb.h"
    +#include "cli/NetworkCli.pb.h"
    +#include "threading/ConditionVariable.hpp"
    +#include "threading/Mutex.hpp"
    +#include "utility/MemStream.hpp"
    +
    +#include "gflags/gflags.h"
    +#include "glog/logging.h"
    +
    +using grpc::Channel;
    +using grpc::ClientContext;
    +using grpc::Server;
    +using grpc::Status;
    +
    +namespace quickstep {
    +
    +namespace networkio_internal {
    +
    +/**
    + * Contains state and helper methods for managing interactions between a producer/consumer (requestor/requestee)
    + * thread.
    + */
    +class RequestState {
    + public:
    +  RequestState() :
    +    request_ready_(false),
    +    response_ready_(false),
    +    request_buffer_(""),
    +    mutex_(),
    +    condition_() {}
    --- End diff --
    
    No needs for `request_buffer_`, `mutex_` and `condition_`.


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