You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@impala.apache.org by "Michael Ho (Code Review)" <ge...@cloudera.org> on 2017/09/09 16:02:48 UTC

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Michael Ho has uploaded a new change for review.

  http://gerrit.cloudera.org:8080/8023

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents some issues with thrift RPC in which a thread may be stuck in
an RPC call without being able to check for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, there are 64 service threads. The service
threads are shared across all queries so the RPC handler should avoid
blocking as much as possible. In thrift RPC implementation, we make a thrift
thread handling a TransmitData() RPC to block for extended period of time
when the receiver is not yet created when the call arrives. In KRPC
implementation, we store TransmitData() or EndDataStream() requests
which arrive before the receiver is ready in a per-receiver early sender list
stored in KrpcDataStreamMgr. These RPC calls will be processed and responded to
when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a blocked_sender_ queue to be processed
later. The stashed RPC request will not be responded to until it is processed
so as to exert back pressure to the client. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Build passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
29 files changed, 2,783 insertions(+), 163 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/1
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-PatchSet: 1
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Michael Ho <kw...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 4:

(86 comments)

Reply to some of the comments for now. Will look into removing ProtoRowBatch next. Will not rebase until next version of the patch is pushed.

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/common/status.cc
File be/src/common/status.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/common/status.cc@246
PS3, Line 246: void Status::FromProto(const StatusPB& status) {
> this is the same as FromThrift() effectively, right? Can we make the two lo
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/common/status.cc@262
PS3, Line 262: void Status::FreeMessage() noexcept {
> same comment. let's make this and ToThrift look the same so it's obvious th
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/exec/exchange-node.cc
File be/src/exec/exchange-node.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/exec/exchange-node.cc@109
PS3, Line 109:   RETURN_IF_CANCELLED(state);
> why do we do this in some Open() but not all? Should we just do it in ExecN
Actually, I noticed similar patterns in other exec nodes. Let me keep this line of change for now and do the refactoring in another change.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/rpc/rpc-mgr.h
File be/src/rpc/rpc-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/rpc/rpc-mgr.h@154
PS3, Line 154:   ~RpcMgr() {
> nit: one-liner?
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/rpc/rpc-mgr.cc
File be/src/rpc/rpc-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/rpc/rpc-mgr.cc@77
PS3, Line 77:   VLOG_QUERY << "Registered KRPC service: " << service_pool->service_name();
> Should we add a log message stating which services we registered with KRPC?
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@63
PS3, Line 63: tiple RPCs. The logical connection between a pair of client 
> I don't think that's accurate. see questions in krpc-data-stream-recvr.h ab
Comment rephrased.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@93
PS3, Line 93: After the first batch has been received, a sender continues to send batches, one 
> XXX check whether these are really different
Rephrased.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@94
PS3, Line 94: () RPC
> what buffer? do you mean queue?
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@108
PS3, Line 108: 
> what does that mean?  Is it saying that during unordinary operation, a send
It means the fragment instance completes without hitting any error. If a fragment instance ends early, it may end up not calling EOS() RPC. For instance, if there is any cancellation, the stream will just be torn down without sending EOS as it's expected that the receivers' fragments will be cancelled too.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@140
PS3, Line 140: RPCs, and may be cancell
> what are "it" and "its" here? "the sender" and "the RPC's"?
the result will be dropped silently by the RPC layer.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@141
PS3, Line 141: /// time. If an in-flight RPC is cancelled at the sender side, the reply from the receiver
> is that still true now that we have cancellation of RPCs?
Yup. If an RPC is cancelled before the result arrives, the KRPC code will just ignore the result.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@153
PS3, Line 153: 
> sending fragment?
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@164
PS3, Line 164:  structure is const
> is that because the recvr hasn't showed up yet, or because the stream's que
both. Comments updated.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@166
PS3, Line 166: 
> is that talking about the 'request' field below, or something different?
Yes.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@175
PS3, Line 175:   kudu::rpc::RpcContext* rpc_context;
> what's the relationship between this and proto_batch?
proto_batch is the inbound row_batch populated from information in 'request' and 'rpc_context'. I agree that it's not strictly necessary to keep it in TransmitDataCtx.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@178
PS3, Line 178:   /// such as the destination finst ID, plan node ID and the row batch header.
> who owns it?
'context'. Commends added.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@235
PS3, Line 235: the mai
> dest_node_id?
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@239
PS3, Line 239: Create a receiver for a specific fragment_instance_id/dest_node_id.
             :   /// If is_m
> that seems unnecessary but don't change it now.
The problem is that Close() of a receiver is not synchronized with the service threads which add the row batches. So, it's possible that there are still outstanding references to the receiver after it has been closed by the owning exchange node.

We should fix this by synchronizing the Close() of a receiver and all outstanding service threads which hold reference to it.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@246
PS3, Line 246: t
> 'proto_batch'?
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@248
PS3, Line 248: o
> 'request'.
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@266
PS3, Line 266: The RPC may not be res
> is this an RPC handler? I think we should just be explicit about which of t
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@267
PS3, Line 267: 
> what RPC is this talking about? If this is a handler, then it's clear.
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@274
PS3, Line 274: 
> Does it close or cancel? (or is there no difference?)
Cancel is more accurate. Close() will free all the buffered row batches. Cancel() will just mark all sender queues as cancelled so no more row batches can be enqueued.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@284
PS3, Line 284: 
> To be consistent with terminology used in class comment, maybe say "deferre
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@314
PS3, Line 314: 
> susper-nit: Capital 'W'
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@340
PS3, Line 340: 
> what is that saying? is that a misplaced comma or am I reading this wrong?
We sort all receiver IDs based on (finst_id, dest_node_id). To locate all receivers for a given fragment instance, we call std::set::lower_bound(finst_id, 0) to identify the first entry and then iterate until finst_id stops matching. I rephrased the comment to make it less confusing.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@341
PS3, Line 341: } else if (a.first.
> I don't understand this.  it kinda sounds like we're trying to be able to f
Clarified in the new comment.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@349
PS3, Line 349:     }
> hmm, I guess we need this now that we can't block the RPC thread?
Yes. In essence, anything which blocks needs to be either stashed somewhere or replied to with an error status.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@358
PS3, Line 358: ntRe
> Monotonic time
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@374
PS3, Line 374: 
> monotonic time
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@382
PS3, Line 382:   /// or b) the Maintenance() thread detects that the longest-waiting sender has been
> all this parallel startup stuff really needs to be revisited (but not for t
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@386
PS3, Line 386: 
> maybe call it DeserializeDeferred() or DeserializeWorker() to make it clear
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@404
PS3, Line 404:   void EnqueueDeferredBatch(DeserializeWorkItem&& payload);
> how about grouping this with Deferred function above since it's related. Al
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@413
PS3, Line 413: ndDat
> what's that?
Removed.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@414
PS3, Line 414: /// st
> I think that status is not getting checked by the caller. I thought Tim mad
Added check for return status.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@416
PS3, Line 416:   void AddEarlyClosedSender(const TUniqueId& fragment_instance_id,
> let's add a quick comment.
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@421
PS3, Line 421: n empty shared_ptr if
> RespondToTimedOutSender() or RespondTimeOutToSender()?
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@112
PS3, Line 112:   for (unique_ptr<TransmitDataCtx>& ctx : early_senders.waiting_sender_ctxs) {
             :      EnqueueDeferredBatch({recvr->fragment_instance_id(), move(ctx)});
             :      num_senders_waiting_->Increment(-1);
             :   }
             :   for (unique_ptr<EndDataStreamCtx>& ctx : early_senders.closed_sender_ctxs) {
             :      recvr->RemoveSender(ctx->request->sender_id());
             :      Status::OK().ToProto(ctx->response->mutable_status());
             :      ctx->rpc_context->RespondSuccess();
             :      num_senders_waiting_->Increment(-1);
             :   }
> It's not possible for the same sender to be in waiting_senders_ctxs and clo
Yes, it's impossible as the early senders shouldn't have sent EOS() without waiting for the reply for its previous TransmitData() RPC.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@140
PS3, Line 140:   }
             :   RecvrId recvr_id = make_pair(finst_id, dest_node_id);
             :   if (closed_stream_cache_.find(recvr_id) != closed_stream_cache_.end()) {
             :     *already_unregistered = true;
             :   }
             :   return shared_ptr<KrpcDataStreamRecvr>();
             : }
             : 
> I'm thinking it makes sense to prioritize finding the receiver with the ass
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@151
PS3, Line 151: vrId recvr_id = make_pair(finst_i
> We could merge the implementations of AddEarlySender() and AddEarlyClosedSe
Probably not worth it.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@174
PS3, Line 174:     // closed_stream_cache_), the sender is timed out by the maintenance thread.
> Add comment "In the worst case, this RPC is so late that the receiver is al
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@242
PS3, Line 242: 
             :   {
             :     // TODO: Move this to maintenance thread.
             :     // Remove any closed streams that have been in the cache for more than
             :     // STREAM_EXPIRATION_TIME_MS.
             :     lock_guard<mutex> l(lock_);
             :     ClosedStreamMap::iterator it = closed_stream_expirations_.begin();
             :     int64_t now = MonotonicMillis();
             :     int32_t before = closed_stream_cache_.size();
             :     while (it != closed_stream_expirations_.end() && it->first < now) {
             :       closed_stream_cache_.erase(it->second);
             :       closed_stream_expirations_.erase(it++);
             :     }
             :     DCHECK_EQ(closed_stream_cache_.size(), closed_stream_expirations_.size());
             :     int32_t after = closed_stream_cache_.size();
             :     if (before != after) {
             :       VLOG_QUERY << "Reduced stream ID cache from " << before << " items, to " << after
             :                  << ", eviction took: "
             :    
> Historically, we never had a maintenance thread, which is why we did the st
Not a faster response to EOS RPC because the RPC is responded to already at line 240 but I agree that we should avoid blocking the service threads for this chore.

Added a TODO.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@289
PS3, Line 289: }
             : 
             :   const string msg = Substitute(
             :       "Unknown row r
> Should this be a DCHECK instead?
This may still be useful if the receiver is closed twice due to error.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@303
PS3, Line 303: ile (iter != fragment_recvr_set_.end() && iter->first == finst_id) {
> Just realized that we could do this more efficiently. Instead of doing an O
Not sure I understand the proposed idea. As discussed offline, we shouldn't call FindRecvr() for more than the number of receivers in a fragment instance.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h
File be/src/runtime/krpc-data-stream-recvr.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h@48
PS3, Line 48: Single receiver of an m:n data stream.
> Either the "an" shouldn't be there or streams shouldn't be plural. But I'm 
Yes, it should be m:n data stream without 's'. So, the definition would be all (send, recvr) pairs.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h@116
PS3, Line 116: sponded t
> that doesn't seem accurate
Oops..fixed.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h@117
PS3, Line 117: an't be
> what's that?
Fixed.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h@126
PS3, Line 126: 
> mentioned above, we didn't clearly define what a "stream" actually is, so i
See reply above.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h@141
PS3, Line 141: 
> is this in bytes?
Yes. Fixed.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h@142
PS3, Line 142: _;
> wouldn't it be more accurate (and consistent with mgr terminology) to say "
Done


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@225
PS1, Line 225:     ++num_pending_enqueue_;
> But batch_queue_ will still be empty, so other callers of AddBatch() will s
The lock at line 196 serializes all callers but yes, this is fixed in PS4.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.cc@192
PS3, Line 192:   RpcContext* context) {
This lock is pessimistic and prevents multiple threads from deserializing multiple row batches in parallel.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.cc@253
PS3, Line 253:   lock_guard<SpinLock> l(lock_);
             :   DCHECK_GT(num_remaining_senders_, 0);
             :   num_remaining_senders_ = max(0, num_remaining_senders_ - 1);
             :   VLOG_FILE << "decremented senders: fragment_instance
> Is this that important that we have to do it while holding the lock?
Useful for debugging missing EOS but can be done outside of the lock. It's not on for the default log level though.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.h
File be/src/runtime/krpc-data-stream-sender.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.h@53
PS3, Line 53: ender instance, and is unique within a fragment.
> it's not clear what that means from just reading the header, though i know 
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.h@59
PS3, Line 59: ONED (broadc
> that's not documented.
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.h@97
PS3, Line 97:   friend class DataStreamTest;
> should that really be public? seems more like a worker function.
Moved to private.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.h@101
PS3, Line 101:   virtual Status Init(const std::vector<TExpr>& thrift_output_exprs,
> that seems weird. is it for testing? if so, can it be protected instead?
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.h@109
PS3, Line 109:   class Channel;
> why is that protected? is this a testing thing?
Overriding the Init() in DataSink which is protected. This is called from DataSink::Create() so it shouldn't be public. The derived classes of DataSink may still call DataSink::Init() so it's protected.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@89
PS3, Line 89: // Note that due to KUDU-2011, timeout cannot be used with outbound sidecars. The client
> I think we should add a reference to KUDU-2011 somewhere here like:
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@92
PS3, Line 92: r ref
> query? does it mean fragment instance?
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@124
PS3, Line 124: status if serialization
> is that still accurate?
Removed. Comments carried over from the thrift implementation.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@139
PS3, Line 139:   // Flushes any buffered row batches and sends the EOS RPC to close the channel.
> that could use a comment.
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@141
PS3, Line 141: 
             :   int64_t num_data_bytes_sent() const { return num_data_bytes_sent_; }
             : 
> those could be commented together to say they identify the destination. it'
The names are carried over from the existing data-stream-sender.cc. Comments may help.

Please see my reply to the comments about "req_" and "response_" fields below.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@152
PS3, Line 152:   int buffer_size_;
> Add "Not used for unpartitioned and random partitioned types".
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@159
PS3, Line 159: ress address_;
> caps since constant?
I renamed this to NUM_OUTBOUND_BATCHES for now. Google C++ style guideline (http://google.github.io/styleguide/cppguide.html#Constant_Names) suggests otherwise but I think we don't follow the guideline strictly (e.g. the use of LLVM_CLASS_NAME in various places).

Yes, there is implicit assumption about how the WaitForRpc() will wait for the in-flight RPC to complete so bumping this number will not provide any benefit. That said, it will help with readability to have a name for this value instead of using a magic constant.

I plan to rewrite this part of the code later to use a queue which will allow more than 2 outbound row batches. I don't want to add extra complication to this already large patch. Will leave a TODO.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@175
PS3, Line 175: llptr
> proxy_
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@178
PS3, Line 178:   // buffer for the serialized row batches. When one is used for the in-flight RPC,
             :   // the execution thread can continue to run and serialize another row batch to the
             :   // other entry. 'current_batch_idx_' is the index of the entry being used by the
             :   // in-flight or last completed RPC.
             :   // TODO: replace this with an actual queue. Schedule another RPC callback in the 
             :   // completion callback if the queue is not empty.
             :   CachedProtoRowBatch cached_proto
> Why do we need to store these in the class? Can't they be local in the func
We need to access these fields from the callback (e.g. due to retry).

The req_ fields may be generated on the fly but the response buffer definitely needs to live for the entire duration of RPC call. Will look into generating 'req_' on the fly as its lifetime need not be longer than the RPC invocation itself.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@197
PS3, Line 197:   RpcController rpc_controller_;
> why is that needed now? 
It saves some bandwidth if the receiver is already closed. It's also a stop-gap fix for IMPALA-3990.

If the remote receiver is closed (e.g. hitting a limit), it will live in the closed_stream_cache_ until it gets evicted after STREAM_EXPIRATION_TIME_MS ms. After the receiver is evicted from this cache, all future calls to this receiver will fail with a timeout. If we have this flag here, we will save the sender from hitting the timeout issue in most cases. Granted, this is not a fool-proof solution for IMPALA-3990 (e.g. the sender may not send anything for STREAM_EXPIRATION_TIME_MS ms) but it will help in most cases. In the long run, we should have a better answer for IMPALA-3990 in the long run. Please also see my previous reply to Tim's comments.

Cancelling the fragment instance may need some thought. For the current partitioning strategies, closing one of the channels shouldn't prevent a sender from sending to other channels unless there is only one channel.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@200
PS3, Line 200:   TransmitDataRequestPB req_;
> I think it would help to associate this with rpc_in_flight_ - move them adj
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@218
PS3, Line 218: 
> if we have the constant, shouldn't that use it?
This function is removed in the latest patch.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@229
PS3, Line 229: 
> nit: re-invokes or retries
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@242
PS3, Line 242:   // the actual RPC when the RPC is rescheduled.
> Add a comment "Should only be called from the main fragment instance thread
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@290
PS3, Line 290:   void EndDataStreamCompleteCb();
> It would be nice to separate out the responsibility of setting of certain s
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@334
PS3, Line 334:   SCOPED_TIMER(parent_->state_->total_network_send_timer());
             : 
             :   // Wait for in-flight RPCs to complete unless the parent sender is closed or cancelled.
             :   while(rpc_in_flight_ && !ShouldTerminate()) {
             :    
> seems simpler to just write:
Done. Will switch over to ConditionVariable as need arises in the future as the TimedWait() takes boost::unique_lock<boost::mutex>.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@397
PS3, Line 397: 
> CurrentProtoBatch().
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@402
PS3, Line 402:       remote_recvr_closed_ = true;
> Would the case where the RPC got cancelled from L547 fall here? Or would it
It can go to either line 406 or here, depending if the completion callback is invoked before the cancellation event is processed. Either way, the cancellation from L547 will not check for


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@528
PS3, Line 528:   return Status::OK();
> if (UNLIKELY(remote_recvr_closed_)) return Status::OK(); ?
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@644
PS3, Line 644:   return Status::OK();
> The RowBatch is serialized once per channel which is very wasteful. 
Fixed.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/row-batch.h
File be/src/runtime/row-batch.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/row-batch.h@52
PS3, Line 52: struct ProtoRowBatch {
> I think we should get rid of this structure all together. IMO, the indirect
ProtoRowBatch is a conceptual representation of a serialized row batch in both the sender and receiver side. A more appropriate name could be "SerializedRowBatch".  I find it easier to have everything encapsulated in ProtoRowBatch when passing it to RowBatch::Deserialize() and hide the details of how it's constructed inside the RPC handler.

Will look into removing the extra layer of indirection.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/row-batch.h@89
PS3, Line 89: CachedProtoRowBatch
> what is "cached" about this?
Had hard time coming up with a good name to indicate the re-use of the vector and string buffers below. May be "reusable" is closer to the actual meaning but it's also quite confusing.

Will switch to OutboundRowBatch and look into merging this with ProtoRowBatch above.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/service/data-stream-service.cc
File be/src/service/data-stream-service.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/service/data-stream-service.cc@49
PS3, Line 49:   // CloseSender() is guaranteed to eventually respond to this RPC so we don't do it here.
> nit: "CloseSender() is guaranteed to eventually respond to this RPC, so we 
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/data_stream_service.proto
File common/protobuf/data_stream_service.proto:

http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/data_stream_service.proto@29
PS3, Line 29:   // Id of this fragment in its role as a sender.
            :   optional int32 sender_id = 2;
            : 
> what are "IDs" in these cases? let's improve the documentation here. Especi
There is no equivalent of typedef in protobuf as far as I can tell.
Comments updated.


http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/row_batch.proto
File common/protobuf/row_batch.proto:

http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/row_batch.proto@30
PS3, Line 30: 
> in thrift we had TTupleId. Is there a reason we aren't defining those types
As far as I know, there is no equivalent of typedef in protobuf. We can try defining a message with a single field but this seems unnecessarily cumbersome.


http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/row_batch.proto@32
PS3, Line 32: = 2;
> what's tuple_data? not a field in this structure...
That's the tuple data sent as sidecar. Clarified in the new comments.


http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/row_batch.proto@39
PS3, Line 39: The 
> size of what?
Size of tuple_data. Comments fixed.


http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/row_batch.proto@42
PS3, Line 42: ion is applied.
> do we plan to fix that?
Fixed in latest patch.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 4
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Wed, 25 Oct 2017 22:54:52 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 6:

(5 comments)

Some initial comments for this round.

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/exec/kudu-util.h
File be/src/exec/kudu-util.h:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/exec/kudu-util.h@45
PS6, Line 45: status_
nit: use same mangling between this and KUDU_RETURN_IF_ERROR (prepend seems better too since it's not a member of a class and status_ is a comment member name).


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@97
PS6, Line 97: two things:
            : /// process it immediately, add it to a fixed-size 'batch queue' for later processing,
            : /// or defer processing the RPC if the 'batch-queue' is full
i'm confused what the "two" cases are from this comment.  Also, I think it's kind of confusing what "processing" means. Should it read something like:

two things: deserialize it immediately adding it to the a 'batch queue', or defer deserializing and respond to the RPC later if the 'batch queue' is full.

Also, the batch queue isn't really "fixed size", right?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@171
PS6, Line 171:     if (!blocked_senders_.empty()) {
we talked about this in person, but want to note it so I don't forget: this loop will deque a "random" number of blocked senders, until the first one happens to finish deserializing and shows up in batch_queue_. That seems wrong.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@220
PS6, Line 220: DCHECK_GT(num_remaining_senders_, 0);
why do we have this DCHECK (i.e. why is this condition important here), and could it be violated with out of order RPCs?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@237
PS6, Line 237: recvr_->ExceedsLimit(batch_size)
it seems like we should be ORing that with !block_senders_.empty(), or something. Otherwise, stuff that's been in the block_senders_ queue for a while can be passed by new stuff. i.e. block_senders_ senders can get starved -- maybe that explains longer than expected RPC times we've seen in some cases?



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 6
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Mon, 30 Oct 2017 19:35:45 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 8:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@211
PS8, Line 211: deferred_rpcs_.empty() && batch_queue_.empty()
> why not write this condition as:
Actually, I think the loop exiting condition is not quite right, which led to this confusin conditional.  The loop exiting condition for "we're done" should check that there are no more senders, and that there's nothing left to drain from the deferred_rpcs_ queue and that there's no pending insert into batch_queue_.

So, the third wait loop condition should be something like:

(num_remaining_senders > 0 || !deferred_rpcs_.empty() || num_pending_enqueue_ > 0)

and then this if-stmt conditional can just be:
if (batch_queue_.empty())

and then the DCHECK can be the negation of that third wait loop conditional:

// Wait loop is exited with an empty batch_queue_ only when there will be no more batches.
DCHECK(num_remaining_senders == 0 && deferred_rpcs_.empty() && num_pending_enqueue_ == 0);

And then you can get rid of the outer loop. That outer loop should be removed since it's effectively a busy wait (and I think we could get into a busy wait state in the previous patchsets).



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 8
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Tue, 07 Nov 2017 15:58:21 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Sailesh Mukil (Code Review)" <ge...@cloudera.org>.
Sailesh Mukil has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 1:

(5 comments)

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc@198
PS1, Line 198: AddData
Isn't the point of the deserialize pool to deserialize the payload early?
Here, we're just calling AddData() on the payloads for early senders after the corresponding receiver has been created.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@165
PS1, Line 165: Either we'll consume a row batch from batch_queue_, or it's empty
Shouldn't there always be something in the batch_queue_ if there's something in the blocked_senders_ list? Since we fill the blocked_senders_ only if the queue is at its limit.

And we also logically service the batches from batch_queue_ first before servicing the batches from the blocked_senders_ list.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@166
PS1, Line 166: There is a window
Just to make things clearer, could you specify what there's a window for?


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@225
PS1, Line 225: 
There is a problem here. When we release lock_here, an arbitrary number of senders could call AddBatch(), and all their batches would get enqueued even though the ExceedsLimit() would be true. This breaks the guarantee of the queue not being over committed more than a single batch.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@284
PS1, Line 284:   for (const auto& queue_entry: batch_queue_) delete queue_entry.second;
batch_queue_.clear() ?



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Tue, 26 Sep 2017 18:11:25 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 13:

(5 comments)

Rebased. Removed a scoped timer which is racy, removed unneeded WARN_UNUSED_RESULTS and fixed some clang-tidy errors.

http://gerrit.cloudera.org:8080/#/c/8023/10/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/10/be/src/runtime/krpc-data-stream-recvr.cc@408
PS10, Line 408: // Dequeues the deferred batch an
> not needed.
Done


http://gerrit.cloudera.org:8080/#/c/8023/10/be/src/runtime/krpc-data-stream-recvr.cc@425
PS10, Line 425: deferred_rpcs_.push(move(ctx));
> can be done without holding the lock.
Done


http://gerrit.cloudera.org:8080/#/c/8023/10/be/src/runtime/krpc-data-stream-recvr.cc@562
PS10, Line 562: // Add all batches to the same queu
There is a subtle race condition here. receiver may be closed already at this point so we need to do something similar to CANCEL_SAFE_SCOPED_TIMER(). However, the race may still happen if that macro is not used under the sender queue's lock. Removed from this patch for now and it will be added back in a follow on patch which introduces more diagnostics information.


http://gerrit.cloudera.org:8080/#/c/8023/10/be/src/runtime/krpc-data-stream-recvr.cc@569
PS10, Line 569: sender_queues_[use_sender_id]->De
Same comment as AddBatch().


http://gerrit.cloudera.org:8080/#/c/8023/10/be/src/runtime/row-batch.h
File be/src/runtime/row-batch.h:

http://gerrit.cloudera.org:8080/#/c/8023/10/be/src/runtime/row-batch.h@145
PS10, Line 145: 
> typo.
Done



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 13
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Thu, 09 Nov 2017 00:42:11 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 7:

(5 comments)

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h@430
PS7, Line 430: i
id


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h@436
PS7, Line 436: specifies
identifies


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@61
PS7, Line 61:  
nit: blank space


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.h
File be/src/runtime/krpc-data-stream-sender.h:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.h@136
PS7, Line 136: //
nit: ///


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc@182
PS7, Line 182:  It points to 
delete.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 7
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Fri, 03 Nov 2017 18:13:57 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 9:

(21 comments)

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-mgr.h@435
PS8, Line 435: 'num_requ
> 'num_request' requests
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-mgr.cc@62
PS8, Line 62: ds, C
> how was that chosen? do we have a test case that causes this queue to fill 
Just a large enough number to reduce the chance of the queue filling up.  Yes, need to come up with a test case to fill up this queue.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-mgr.cc@109
PS8, Line 109:       // Let the receiver take over the RPC payloads of early senders and process them
             :       // asynchronously.
             :       for (unique_ptr<TransmitD
> this comment seems out of place. this is more an implementation detail of t
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.h
File be/src/runtime/krpc-data-stream-recvr.h:

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.h@129
PS8, Line 129: n
> and start a deserialization task to process it asynchronously.
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.h@130
PS8, Line 130: ask to p
> this "transfer" is in the oppose direction of how our "Transfer" methods us
Renamed to TakeOverEarlySender().


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.h@197
PS8, Line 197: eive
> cpu time or wall time?
wall-clock time.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.h@200
PS8, Line 200: _;
> same question
wall-clock time.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@67
PS8, Line 67: data
> the resources
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@72
PS8, Line 72: imit, the
> RPC state
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@73
PS8, Line 73: nto 'deferred_
> we shouldn't normally refer to private fields in public class comments, but
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@87
PS8, Line 87:   void TakeOverEarlySender(std::unique_ptr<TransmitDataCtx> ctx);
> same comment as recvr header comment.
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@195
PS8, Line 195:   // cur_batch_ must be replaced with the
> I don't think we need this loop. see other comments in this function.
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@197
PS8, Line 197: atch = nullptr;
> nit: consider swapping the order of these so that the fast case comes first
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@210
PS8, Line 210:     }
> nit: i think we could do without some of the blank lines in this method to 
I leave a blank line between each loop exiting conditions.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@211
PS8, Line 211: 
> Actually, I think the loop exiting condition is not quite right, which led 
There is an invariant that EOS cannot be sent by a sender when there is outstanding TransmitData() RPC so we should be able to get by by just checking for the termination condition of: (num_remaining_senders_ == 0 && batch_queue_.empty())


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@218
PS8, Line 218: 
> given that we just checked the other two loop exit conditions, isn't this d
Converted to DCHECK().


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@229
PS8, Line 229: d
> to parallelize the CPU bound deserialization work.
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@239
PS8, Line 239:     }
> once you get rid of the loop, I think you'll be able to eliminate this unlo
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@389
PS8, Line 389:       status.ToProto(ctx->response->mutable_status());
> at this point, 'ctx' effectively takes ownership, right? we should add a co
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@406
PS8, Line 406:     const RowBatchHeaderPB& header = ctx->request->row_batch_header();
             :     AddBatchWork(batch_size, header, tuple_offsets
> nit: let's reorder these two lines since num_deferred_batches_ is counting 
Done


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-sender.cc@434
PS8, Line 434:   resp_.Clear();
> May want to call resp_.Clear() too.
Done



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 9
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Tue, 07 Nov 2017 21:49:49 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Hello Sailesh Mukil, Mostafa Mokhtar, Dan Hecht, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/8023

to look at the new patch set (#7).

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents some issues with thrift RPC in which a thread may be stuck in
an RPC call without being able to check for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, there are 64 service threads. The service
threads are shared across all queries so the RPC handler should avoid
blocking as much as possible. In thrift RPC implementation, we make a thrift
thread handling a TransmitData() RPC to block for extended period of time
when the receiver is not yet created when the call arrives. In KRPC
implementation, we store TransmitData() or EndDataStream() requests
which arrive before the receiver is ready in a per-receiver early sender list
stored in KrpcDataStreamMgr. These RPC calls will be processed and responded to
when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a blocked_sender_ queue to be processed
later. The stashed RPC request will not be responded to until it is processed
so as to exert back pressure to the client. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Build passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/exec/kudu-util.h
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/data-stream-sender.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M cmake_modules/FindProtobuf.cmake
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
33 files changed, 3,124 insertions(+), 180 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/7
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 7
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 10: Code-Review+2


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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 10
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Wed, 08 Nov 2017 20:22:06 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 8:

(36 comments)

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h@102
PS7, Line 102: the RPC state is
             : /// saved into the receiver's 'deferred_r
> it's not really the batch added. and it's not just a single structure for t
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h@104
PS7, Line 104: C is removed from the 'deferr
> how about: ... from a deferred RPC queue and the row batch is deserialized.
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h@393
PS7, Line 393:     EarlySendersList& operator=(EarlySendersList&& other) {
> quick comment for why we define a move constructor and move operator=, sinc
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h@430
PS7, Line 430: n
> id
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h@436
PS7, Line 436: nt instan
> identifies
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@61
PS7, Line 61:  
> nit: blank space
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@217
PS7, Line 217: 
> that shouldn't be possible in the DEFERRED_BATCHES case, right? so i'd prob
This is actually possible if the receiver is closed before the deserialization threads execute this task.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@236
PS7, Line 236:  << " node_id=" << r
> why is this possible in the waiting_sender_ctxs case but not the closed_sen
The response for the closed_sender_ctxs case is the same regardless of whether the receiver is unregistered or not.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@247
PS7, Line 247: !already_unregistere
> why is that possible?
It's possible that the receiver is cancelled and closed before the deserialization thread gets around to processing this request. In which case, the deferred RPCs would all be replied to in KrpcDataStreamRecvr::SenderQueue::Cancel().


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@248
PS7, Line 248:   AddEarlyClosedSender(finst_id, request, 
> So I guess we no longer multithread within a single sender queue (and for n
PS8 will queue multiple deserialization requests to drain multiple deferred RPCs at the same time.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.h
File be/src/runtime/krpc-data-stream-recvr.h:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.h@78
PS7, Line 78: The caller must call TransferAllResources()
            :   /// to acquire dat
> is that talking about calling TransferAllResources(), or can the caller do 
Yes, it's TransferAllResources(). Comments clarified.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@72
PS7, Line 72: h
> typo
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@97
PS7, Line 97: 
> the HasSpace name seems wrong for condition (1). From the name HasSpace, I 
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@113
PS7, Line 113: B* r
> deserialized or serialized size?
deserialized size


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@169
PS7, Line 169: 
> these aren't really deferred batches though. They are deferred RPCs (which 
Renamed to deferred_rpcs_.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@220
PS7, Line 220:       recvr_->num_buffered_bytes_.Add(-batch_queue_.front().first);
> rather than explaining what, we should explain why:
Simplified in the new patch.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@224
PS7, Line 224:       batch_queue_.pop_front();
> why do we need to drop the lock?
Comments added. TriggerDeferredBatchesDrain() may block. Holding the lock here may lead to dead-lock.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@263
PS7, Line 263:   KUDU_RETURN_IF_ERROR(rpc_context->GetInboundSidecar(
> let's add a comment:
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@269
PS7, Line 269: e data sidecar");
> I guess this only happens in the non-merging case, right?
Yes.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@274
PS7, Line 274: 
> I'm not sure what this dcheck is meant to prove.
This is to verify that Close() cannot proceed as num_pending_enqueue_ > 0. I guess line 284 servers similar purpose so it can be removed.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@322
PS7, Line 322: 
> starvation of a sender in the non-merging case.
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@324
PS7, Line 324: 
             :     unique_lock<SpinLock> l(lock_);
             :     // There should be one or more senders left when this function is called. The reason
             :     // is that EndDataStream RPC is not sent until all outstanding TransmitData() RPC has
             :     // been replied to. There is a
> that comment shoudl be inside HasSpace() now.
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@345
PS7, Line 345:       deferred_rpcs_.push(move(payload));
> are we able to exercise the deferred batches path in functional testing? if
Yes, my local mini-cluster tests shows that we do exercise this patch with concurrent queries but yeah, we should include it in functional tests.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@531
PS7, Line 531:       ADD_COUNTER(sender_side_profile_, "NumBatchesDeferred", TUnit::
> is that accurate?
Yes. DeregisterRecvr() will call CancelStream().


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.h
File be/src/runtime/krpc-data-stream-sender.h:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.h@136
PS7, Line 136: //
> nit: ///
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc@182
PS7, Line 182: 
> delete.
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc@235
PS7, Line 235: fail
> nit: delete call too. 'C' of RPC is call.
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc@257
PS7, Line 257: (e.g. the reactor thread was being shut down) or if the
> with thrift RPC, wouldn't we have retried making a connection and doing the
Yes. We will fail the query in KRPC if the connection failed. Filed IMPALA-6159. Please see JIRA for details.

Btw, I just double checked. The abort here is actually due to shutdown of the reactor thread not the connection. Comments updated.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc@273
PS7, Line 273: rpc_in_flight_
> rpc_in_flight_batch_
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc@554
PS7, Line 554:     while (rpc_in_flight_) rpc_done_cv_.wait(l);
             :   }
             :   batch_.reset();
             : }
> one liner?
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc@558
PS7, Line 558: 
> that becomes vacuous given the while loop condition.
Removed.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.h
File be/src/runtime/row-batch.h:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.h@349
PS7, Line 349: output_batch
> outbound_batch (or rename the param)
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.cc
File be/src/runtime/row-batch.cc:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.cc@426
PS7, Line 426:       (tuple_offsets.size() / sizeof(int32_t)) * sizeof(Tuple*);
> could you order this computation in the same order as thrift, to make it ea
Done


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.cc@432
PS7, Line 432: 
> what is this trying to compute? the size of the tuple_ptrs_? if so, it does
Good catch. Fixed.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.cc@433
PS7, Line 433: atch::GetSerializedSize(const OutboundRowBatch& batch
> what is this accounting for?
Nothing. Looks like thrift version has the same bug as this is not really used. I will refrain from updating the TRowBatch in this patch but updated the size calculation in GetDeserializedSize().


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.cc@440
PS7, Line 440: tuple_ptrs_size_, src->tuple_ptrs_size_);
> what's that accounting?
Left over from previous PS. Removed now.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 8
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Sun, 05 Nov 2017 20:23:47 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 13:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/8023/13/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/13/be/src/runtime/krpc-data-stream-recvr.cc@163
PS13, Line 163: condition_variable_any
these should be changed to impala ConditionVariables now, but you can do that as a follow on



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 13
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Thu, 09 Nov 2017 18:24:52 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 6:

(6 comments)

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@75
PS3, Line 75: g two 
> see comment in row-batch.h about this terminology.
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@178
PS3, Line 178:   // The two OutboundRowBatch which are re-used across multiple RPCs. Each entry contains
             :   // a RowBatchHeaderPB and the buffers for the serialized tuple offsets and data. When
             :   // one is being used for an in-flight RPC, the execution thread continues to run and
             :   // serializes another row batch into the other entry. 'current_batch_idx_' is the index
             :   // of the entry being used by the in-flight or last completed RPC.
             :   //
             :   // TODO: replace this with an ac
> We need to access these fields from the callback (e.g. due to retry).
req_ removed in PS5.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/row-batch.h
File be/src/runtime/row-batch.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/row-batch.h@52
PS3, Line 52: class OutboundRowBatch {
> ProtoRowBatch is a conceptual representation of a serialized row batch in b
Removed in PS5.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/row-batch.h@89
PS3, Line 89: 
> Had hard time coming up with a good name to indicate the re-use of the vect
Renamed to OutboundRowBatch in PS5.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/row-batch.h@431
PS3, Line 431:   ///
             :   /// 'uncompressed_size': Updated with the uncompressed size of 'tuple_data'.
             :   ///
             :   /// 'is_compressed': Sets to true if compression is applied on 'tuple_data'.
             :   /// False otherwise.
             :   ///
             :   /// Returns error status if serialization failed. Returns OK otherwise.
             :   /// TODO: clean this up once the thrift RPC implementation is removed.
             :   Status Serialize(bool full_dedup, vector<int32_t>* tuple_offsets, string* tuple_data,
             :       int64_t* uncompressed_size, bool* is_compressed);
             : 
             :   /// Shared implementation between thrift and protobuf to deserialize a row batch.
             :   ///
             :   /// 'input_tuple_offsets': an int32_t array of tuples; offsets into 'input_tuple_data'.
             :   /// Used for populating the tuples in the row batch with actual pointers.
             :   ///
             :   /// 'input_tuple_data': contains pointer and size of tuples' data buffer.
             :   /// If 'is_compressed' is true, the data is compressed.
             :   ///
             :   /// 'uncompressed_size': the uncompressed size of 'input_tuple_data' if it's compressed.
             :   ///
             :   /// 'is_compressed': True if 'input_tuple_data' is compressed.
             :   ///
             :   /// TODO: clean this up once the thrift RPC implementation is removed.
             :   void Deserialize(const kudu::Slice& tuple_offsets, const kudu::Slice& tuple_data,
             :       int64_t uncompressed_size, bool is_compressed);
             : 
             :   typedef FixedSizeHashTable<Tuple*, int> DedupMap;
             : 
             :   /// The total size of all data represented in this row batch (tuples and referenced
             :   /// string and collection data). This is the size of the row batch after removing all
             :   /// gaps in the auxiliary and deduplicated tuples (i.e. the smallest footprint for the
             :   /// row batch). If the distinct_tuples argument is non-null, full deduplication is
             :   /// enabled. The distinct_tuples map must be empty.
             :   int64_t TotalByteSize(DedupMap* distinct_tuples);
             : 
             :   void SerializeInternal(int64_t size, DedupMap* distinct_tuples,
             :       vector<int32_t>* tuple_offsets, string* tuple_data);
             : 
             :   /// All members below need to be handled in R
> let's leave a TODO about cleaning this all up once we can remove the thrift
TODO added. Cannot file a JIRA as apache JIRA is being re-indexed.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/service/data-stream-service.cc
File be/src/service/data-stream-service.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/service/data-stream-service.cc@64
PS3, Line 64: 
            : 
            : 
            : 
            : 
            : 
            : 
            : 
            : 
> see comment in row-batch.h.  I think we should do this later when we actual
Done



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 6
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Fri, 27 Oct 2017 20:14:41 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 7:

(81 comments)

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/exec/kudu-util.h
File be/src/exec/kudu-util.h:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/exec/kudu-util.h@45
PS6, Line 45: .ok()) 
> nit: use same mangling between this and KUDU_RETURN_IF_ERROR (prepend seems
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/exec-env.cc
File be/src/runtime/exec-env.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/exec-env.cc@89
PS6, Line 89:     "processing threads. If left at default value 0, it will be set to number of CPU "
> how are these defaults chosen?
I didn't really change these parameters from Henry's patch.

There are no good defaults for the service queue size any way (IMPALA-6116) but 1024 seems to be a reasonable bound for untracked memory consumption.

The latest PS sets num_svc_threads to match the number of cores on the system.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@97
PS6, Line 97: etion at any one
            : /// time). The rate of transmission is controlled by the receiver: a sender will only
            : /// schedule batch transmission when the previous transmissi
> i'm confused what the "two" cases are from this comment.  Also, I think it'
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@103
PS6, Line 103: batch que
> deserialized?
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@104
PS6, Line 104: moved from t
> what is this "respectively" referring to?
The two cases. Removed as this seems unnecessary.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@249
PS6, Line 249:  for Transm
> unclear what that means, maybe stale comment? And this should say something
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@258
PS6, Line 258: 
> isn't that part of "memory pointed to by 'request'"? If so and you want to 
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@294
PS6, Line 294: 
> How about getting rid of this typedef? The code seems easier to understand 
This data structure has been renamed to DeserializeTask and also changed in the latest patch.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@428
PS6, Line 428: 
> a sender
Done


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@314
PS3, Line 314: t
> Done
Oops..missed it. Will update in later patch.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.cc@59
PS6, Line 59: 
> I don't have a strong preference either way, but it'd be nice to be consist
My general guideline is to use lambda if it's one or two liners and use bind for larger functions. I guess there many different opinions on this topic. I try to optimize for readability.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.cc@322
PS6, Line 322: tonicMillis() + STREAM_E
> shouldn't be plural
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.h
File be/src/runtime/krpc-data-stream-recvr.h:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.h@119
PS6, Line 119: row batch i
> row batch is deserialized
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@52
PS6, Line 52: total amount of
> it's not clear what that means just from reading the comment. It'd be nice 
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@113
PS6, Line 113: erialized
> how about using unique_ptr since this owns the row batch (until it's transf
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@127
PS6, Line 127:   // If true, the receiver fragment for this stream got cancelled.
> given that a single soft limit is imposed across all sender queues, does it
For the non-merging case, there is essentially only one queue.

For the merging case, I can see that it's possible for a blocked row batch to a sender queue being passed by either new incoming row batches or blocked row batches to another sender queue. However, once a sender queue becomes empty, one of the blocked row batches will always be added so it won't be starved forever. The passing behavior is dependent on how quickly the sender queue is consumed. The quicker the row batches are consumed for a given queue, the sooner its blocked sender queue gets drained and the higher chance that it passes some slower queues. That seems to be fine as this implicitly exerts back pressure to the producers of the sender queue being consumed at a lower pace than others. I may need to put more thought on other implications for this unfairness across different queues but as we discussed offline, this doesn't necessarily need to be addressed in this patch.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@171
PS6, Line 171:   // their batches, allowing the rec
> we talked about this in person, but want to note it so I don't forget: this
The proposed fix is to notify the deserialization thread to kick off a loop which moves as many items from blocked_senders_ to batch_queue_ as possible. An item shouldn't be dequeued from the blocked_sender_ queue unless it can be added.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@201
PS6, Line 201:     if (is_cancelled_) return Status::CANCELLED;
missing return;


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@210
PS6, Line 210:     if (!batch_queue_.empty()) {
missing return;


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@220
PS6, Line 220: // if we haven't notified them alread
> why do we have this DCHECK (i.e. why is this condition important here), and
This is important as we shouldn't expect a EOS RPC until the preceding TransmitData() RPC has been replied to. So, number of senders should be > 0 at this point even for the blocked senders case.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@231
PS6, Line 231: 
> update
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@237
PS6, Line 237: 
> it seems like we should be ORing that with !block_senders_.empty(), or some
Fixed in the latest PS.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@320
PS6, Line 320:  time to resend the batch that we c
This is leaking memory. No point in calling release() here anyway.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@322
PS6, Line 322:     // up after the deferred batches to avoid starvation.
> doing this in Close() goes against the paradigm that Close() is only about 
The latest PS moves this to Cancel().


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.h
File be/src/runtime/krpc-data-stream-sender.h:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.h@57
PS6, Line 57: per_channel_buffer_size' is the soft limit in bytes of the buffering into the
            :   /// per-channel's accumula
> still not clear what that means. This isn't really the size of a buffer, is
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.h@111
PS6, Line 111: serialized row 
> serialized
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.h@130
PS6, Line 130: nt channel t
> outbound row
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.h@133
PS6, Line 133: Index of the next OutboundRowBatch to u
> Maybe say:
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@67
PS6, Line 67: can
> it looks like there's a third interface now: SerializeAndSendBatch() that t
We still need the function SerializeAndSendBatch() for the kudu / hash partitioned schemes. I can see how it's possible to hide it as a private function but then the random case needs to also provide an OutboundBatch for serialization. While we can use the outbound_batches_ in the KrpcDataStreamSender (similar to the unpartitioned case), it seems better to use the buffer of the chosen channel as we round-robin through them and this allows more parallelism than the two outbound_batches_ in KrpDataStreamSender.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@81
PS6, Line 81: Teardown
> Teardown()
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@82
PS6, Line 82: Teardown
> Teardown()
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@123
PS6, Line 123: ment
             :   // instance execution thread. 
> if the preceding RPC is still in-flight.
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@126
PS6, Line 126: co
> or if the preceding RPC failed.
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@135
PS6, Line 135: d tha
> frees
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@141
PS6, Line 141:   // Return error status if either the last TransmitData() RPC or EOS RPC failed.
> Returns error status if...
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@157
PS6, Line 157: 
> delete.
Done. Actually, this is still used in AddRow(). 'buffer_size_' can be removed though.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@166
PS6, Line 166:   // The row batch for accumulating rows copied from AddRow().
> is that actually used?
Not anymore in the new code.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@176
PS6, Line 176:  to keep per-channel bu
> the name of this is confusing because it's so similar to current_batch_idx_
Renamed to rpc_in_flight_batch_. I prefer to keep rpc_in_flight_ and rpc_in_flight_batch_ as separate as EOS RPC doesn't really use rpc_in_flight_batch_.

Moved rpc_in_flight_batch_ to be under lock_ in class declaration.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@178
PS6, Line 178: "UNPARTITIONED" scheme.
> The outbound row batches are double-buffered so that we can serialize the n
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@179
PS6, Line 179: 
             : 
             :   // Index into 'outbound_batches_' for the next availa
> then you can delete that (better to put the explanation for why we have two
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@181
PS6, Line 181: le OutboundRowBatch to serialize
             :   // into. This is read and written by the main execution thread. It
> that looks incorrect. current_batch_idx_ is incremented immediately after w
Renamed to next_batch_idx_;


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@191
PS6, Line 191:   // 'lock_' needs to be held when accessing the following fields.
> At least add the cross refernce:
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@201
PS6, Line 201: 
> Add a quick comment for proxy_ and rpc_controller_
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@237
PS6, Line 237: rpc_retry_i
> an RPC
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@244
PS6, Line 244: _fn, const
> typo
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@250
PS6, Line 250: e only called from a fragment
             :   // executor thread.
> it can return CANCELLED
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@258
PS6, Line 258: nder 
> code
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@257
PS6, Line 257:  (e.g. Connection object was shutdown due to network errors)
             :   // or if the parent sender h
> what does that mean? when can that happen?
If the connection is closed due to network error, all pending RPCs on that connection will be aborted.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@261
PS6, Line 261:  call
> code
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@264
PS6, Line 264: thread
> threads
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@275
PS6, Line 275: error status 
> needs updating
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@279
PS6, Line 279: remote
> threads
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@367
PS6, Line 367: mgr();
> in RetryCb(), this case explicitly becomes MarkDone(CANCELLED), but here he
I see. In which case, we may as well skip the check here since we are checking it in RetryCb() anyway.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@369
PS6, Line 369:     // the need to manage yet another thread pool.
> why do we need to schedule this on the reactor thread (rather than using ou
Mostly to avoid the complexity of managing yet another thread pool.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@382
PS6, Line 382: Y(controlle
> explicilty set that = Status::OK() to make it explicit that the first two i
Sure. I think the default value is Status::OK but yeah we can be more explicit about it.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@409
PS6, Line 409:   TransmitData
> is that needed? seems like a redundant memset
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@437
PS6, Line 437:   // deleted by destructor.
> comment why we need that: 
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@451
PS6, Line 451:   // If the remote receiver is closed already, there is no point in sending anything.
> FlushAndSendEos() does this DCHECK before the remote_recvr_closed_. make th
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@552
PS6, Line 552:   if (rpc_in_flight_) {
             :     rpc_controller_.Cancel();
> while (rpc_in_flight_) rpc_done_cv_.wait(l);
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@657
PS6, Line 657:   } else if (partition_type_ == TPartitionType::RANDOM || channels_.size() == 1) {
> mentioned above: rather than a third way to do this, how about just making 
Please see replies above.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h
File be/src/runtime/row-batch.h:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@50
PS6, Line 50: RPC outb
> I think we should say something about KRPC to at least give that hint. mayb
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@60
PS6, Line 60: sizeof(tuple_of
> sizeof(tuple_offsets_[0]) seems clearer and more robust
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@104
PS6, Line 104: OutboundRowBatch that 
> OutboundRowBatch
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@144
PS6, Line 144:   /// Populate a row batch from the serialized row batch header, decompress / copy
             :   /// the tuple's data into a buffer and convert all offsets in 'tuple_ofsets' back
             :   /// into pointers into the tuple data's buf
> stale
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@354
PS6, Line 354:   /// whether tuple_data is compressed. If an in-flight row is present in this row batch,
> we should preserve this comment when removing the thrift variant. So you co
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@424
PS6, Line 424:   ///                  return. There are a total of num_rows * num_tuples_per_row offsets.
> nit: i don't think we generally have all these line breaks between paramete
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@426
PS6, Line 426: _d
> delete space
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@444
PS6, Line 444: ncomp
> delete
Actually, the input argument names need to be fixed to match the definition.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@447
PS6, Line 447: 
> delete
Actually, the input argument names need to be fixed to match the definition.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@455
PS6, Line 455: s is the size
> input_*
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@531
PS6, Line 531: _scratch_;
> OutboundProtoRowBatch
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@537
PS6, Line 537: /// '_start_row_idx' is the starting row index.
> this seems like a hack and we could do something simpler, but let's leave i
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.cc
File be/src/runtime/row-batch.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.cc@241
PS6, Line 241:     string* tuple_data, int64_t* uncompressed_size, bool* is_compressed) {
> this comment was probably meant to be deleted?
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/util/network-util.cc
File be/src/util/network-util.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/util/network-util.cc@41
PS6, Line 41: using std::find;
> undo. Bad rebase.
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/util/network-util.cc@120
PS6, Line 120: return sock.Pa
> undo
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/data_stream_service.proto
File common/protobuf/data_stream_service.proto:

http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/data_stream_service.proto@29
PS6, Line 29: ance id,
> isn't this the id of the instance?  The comment in KrpcDataStreamSender is 
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/data_stream_service.proto@38
PS6, Line 38: tuple offsets' 
> tuple offsets' buffer
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/data_stream_service.proto@43
PS6, Line 43: the tuple's 
> tuple data's buffer
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/data_stream_service.proto@59
PS6, Line 59:   // Sender instance id, unique within a fragment.
> same
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/row_batch.proto
File common/protobuf/row_batch.proto:

http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/row_batch.proto@25
PS6, Line 25: Please see TransmitDataRequestPB for details.
> delete
Done


http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/row_batch.proto@32
PS6, Line 32: ptional int32 num_tuples_per_
> why is this needed? i don't see it used. The size of it is used, though it 
Yes, we actually only need the size. Updated in the new patch. It'd be good to at least include it in case there is any mismatch with the receiver's row desc (e.g. DCHECK will fire). Turns out that our thrift implementation has similar problem.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 7
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Fri, 03 Nov 2017 15:52:36 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has submitted this change and it was merged. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents avoids the possibility that a thread is stuck in the RPC code
for extended amount of time without checking for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, the number of service threads equals the
number of logical cores. The service threads are shared across all queries so
the RPC handler should avoid blocking as much as possible. In thrift RPC
implementation, we make a thrift thread handling a TransmitData() RPC to block
for extended period of time when the receiver is not yet created when the call
arrives. In KRPC implementation, we store TransmitData() or EndDataStream()
requests which arrive before the receiver is ready in a per-receiver early
sender list stored in KrpcDataStreamMgr. These RPC calls will be processed
and responded to when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a queue for deferred processing.
The stashed RPC requests will not be responded to until they are processed
so as to exert back pressure to the senders. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Builds {exhaustive/debug, core/release, asan} passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Reviewed-on: http://gerrit.cloudera.org:8080/8023
Reviewed-by: Michael Ho <kw...@cloudera.com>
Tested-by: Impala Public Jenkins
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/exec/kudu-util.h
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/data-stream-sender.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M cmake_modules/FindProtobuf.cmake
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
33 files changed, 3,155 insertions(+), 184 deletions(-)

Approvals:
  Michael Ho: Looks good to me, approved
  Impala Public Jenkins: Verified

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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: merged
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 14
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 3:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-sender.cc@103
PS2, Line 103: fragment_instance_id_(fragment_instance_id),
> address_(destination)
Done


http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-sender.cc@141
PS2, Line 141:  const TNetworkAddress add
> const TNetworkAddress address_;
Done


http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-sender.cc@485
PS2, Line 485:   batch_->CommitLastRow();
             :   return Status::OK();
             : }
             : 
             : void 
> This is broken if the RPC was rejected for FLAGS_backend_client_connection_
Changed to using async RPC for this in the new patch. This ensures that we can check for cancellation while waiting for replies from the remote server.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 3
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Mon, 09 Oct 2017 18:10:45 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 8:

(23 comments)

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-mgr.h@435
PS8, Line 435: a request
'num_request' requests


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-mgr.cc@62
PS8, Line 62: 10000
how was that chosen? do we have a test case that causes this queue to fill up?


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-mgr.cc@109
PS8, Line 109:       // Transfer the early senders into 'deferred_rpcs_' queue of the corresponding
             :       // sender queue. This makes sure new incoming RPCs won't pass these early senders,
             :       // leading to starvation.
this comment seems out of place. this is more an implementation detail of the receiver and handled properly inside ProcessEarlySender().  You could incorporate this in the comment for ProcessEarlySender() (to motivate why it uses the deferred_rpcs_ queue).


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.h
File be/src/runtime/krpc-data-stream-recvr.h:

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.h@129
PS8, Line 129: .
and start a deserialization task to process it asynchronously.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.h@130
PS8, Line 130: Transfer
this "transfer" is in the oppose direction of how our "Transfer" methods usually go (e.g. src->TransferResourcesOwnership(dest)). Maybe call this ProcessEarlySender() (though I don't love "process" either since it's so vague).


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.h@197
PS8, Line 197: time
cpu time or wall time?


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.h@200
PS8, Line 200: time
same question


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@67
PS8, Line 67: data
the resources


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@72
PS8, Line 72: 'payload'
RPC state


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@73
PS8, Line 73: deferred_rpcs_
we shouldn't normally refer to private fields in public class comments, but given this is an internal class to the recvr, we can leave this.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@87
PS8, Line 87:   void TransferEarlySender(std::unique_ptr<TransmitDataCtx> ctx);
same comment as recvr header comment.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@195
PS8, Line 195: while (current_batch_.get() == nullptr) {
I don't think we need this loop. see other comments in this function.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@197
PS8, Line 197: !is_cancelled_ && batch_queue_.empty()
nit: consider swapping the order of these so that the fast case comes first (!batch_queue_.empty()) but also to match the comment ("or we know we're done" corresponds to the is_cancelled_ and num_remaining_senders_ == 0 cases).


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@201
PS8, Line 201:       CANCEL_SAFE_SCOPED_TIMER(recvr_->data_arrival_timer_, &is_cancelled_);
             :       CANCEL_SAFE_SCOPED_TIMER(recvr_->inactive_timer_, &is_cancelled_);
             :       CANCEL_SAFE_SCOPED_TIMER(
             :           received_first_batch_ ? nullptr : recvr_->first_batch_wait_total_timer_,
             :           &is_cancelled_);
there's got to be a cleaner way to do this but ignore for now


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@210
PS8, Line 210: 
nit: i think we could do without some of the blank lines in this method to make more code fit on a screen


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@211
PS8, Line 211: deferred_rpcs_.empty() && batch_queue_.empty()
why not write this condition as:
num_renaming_senders_ == 0

then, it's more clear that these three conditions correspond to the loop exit conditions.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@218
PS8, Line 218: !batch_queue_.empty()
given that we just checked the other two loop exit conditions, isn't this definitely true? i.e. we don't need this guard it seems.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@227
PS8, Line 227: .
now that there might be space or the batch queue might be empty.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@229
PS8, Line 229: .
to parallelize the CPU bound deserialization work.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@230
PS8, Line 230:       // No point in dequeuing more than number of deserialization threads available.
true, though this doesn't quite make sense given that the thread pool is shared across all recvrs. but i guess it's an upper bound.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@239
PS8, Line 239:         l.unlock();
once you get rid of the loop, I think you'll be able to eliminate this unlock/lock/unlock and just drop the lock (via scope), which then also makes this easier to reason about.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@389
PS8, Line 389:     deferred_rpcs_.pop();
at this point, 'ctx' effectively takes ownership, right? we should add a comment that says that and that says we cannot return with "delete ctx".

Even better would be to move the front() into a temp unique_ptr in the function scope. Or you could always pop by moving into a unique_ptr and re-push_front() if you find there is no space. But the main point is it'd be nice to use a smart pointer to ensure we don't return and leak.


http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-recvr.cc@406
PS8, Line 406:     ++num_deserialize_tasks_pending_;
             :     COUNTER_ADD(recvr_->num_deferred_batches_, 1);
nit: let's reorder these two lines since num_deferred_batches_ is counting the nmber of things that go into deferred_rpcs_ while num_deserialization_tasks_pending_ is counting the number of tasks added to to the DS mgr.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 8
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Mon, 06 Nov 2017 21:58:30 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Hello Sailesh Mukil, Mostafa Mokhtar, Dan Hecht, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/8023

to look at the new patch set (#6).

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents some issues with thrift RPC in which a thread may be stuck in
an RPC call without being able to check for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, there are 64 service threads. The service
threads are shared across all queries so the RPC handler should avoid
blocking as much as possible. In thrift RPC implementation, we make a thrift
thread handling a TransmitData() RPC to block for extended period of time
when the receiver is not yet created when the call arrives. In KRPC
implementation, we store TransmitData() or EndDataStream() requests
which arrive before the receiver is ready in a per-receiver early sender list
stored in KrpcDataStreamMgr. These RPC calls will be processed and responded to
when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a blocked_sender_ queue to be processed
later. The stashed RPC request will not be responded to until it is processed
so as to exert back pressure to the client. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Build passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/exec/kudu-util.h
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/data-stream-sender.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M be/src/util/network-util.cc
M cmake_modules/FindProtobuf.cmake
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
34 files changed, 2,932 insertions(+), 175 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/6
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 6
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 3:

(16 comments)

Some more comments, still going though.

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@75
PS3, Line 75: cached
see comment in row-batch.h about this terminology.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@89
PS3, Line 89: // safe to free them once the callback has been invoked.
I think we should add a reference to KUDU-2011 somewhere here like:

Note that due to KUDU-2011, timeout cannot be used with outbound sidecars. The client has no idea when it is safe to reclaim the sidecar buffer (~RpcSidecar() should be the right place, except that's currently called too early).  RpcController::Cancel(), however, ensures that the callback is called only after the RPC layer no longer references the sidecar buffer.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@92
PS3, Line 92: query
query? does it mean fragment instance?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@124
PS3, Line 124: Shutdown the RPC thread
is that still accurate?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@139
PS3, Line 139:   int buffer_size_;
that could use a comment.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@141
PS3, Line 141:   const TNetworkAddress address_;
             :   TUniqueId fragment_instance_id_;
             :   PlanNodeId dest_node_id_;
those could be commented together to say they identify the destination. it's a little odd that plan node id is prefixed "dest" when the others are not.

it also seems weird that we need both these and the req_ field since shouldn't they just be stored there?  Or seems we should get rid of the req_ and just generate it when sending.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@159
PS3, Line 159: num_cached_proto_batches_
caps since constant?
also, can this ever be something other than 2 without writing the code? i.e. doesn't the code assume this value is 2 in various ways?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@175
PS3, Line 175: proxy
proxy_


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@197
PS3, Line 197:   bool remote_recvr_closed_ = false;
why is that needed now? 
also, shouldn't we do something different, at a higher level, in that case (like cancel this instance)?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@200
PS3, Line 200:   Status rpc_status_;
I think it would help to associate this with rpc_in_flight_ - move them adjacent and say that rpc_status_ is valid only when rpc_in_flight_ is false, or something.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@218
PS3, Line 218: 2
if we have the constant, shouldn't that use it?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@334
PS3, Line 334:   auto pred = [this]() -> bool { return !rpc_in_flight_ || ShouldTerminate(); };
             :   auto timeout = std::chrono::system_clock::now() + milliseconds(50);
             :   while (!rpc_done_cv_.wait_until(*lock, timeout, pred)) {
             :     timeout = system_clock::now() + milliseconds(50);
             :   }
seems simpler to just write:

while (rpc_in_flight_ && !ShouldTerminate()) {
  auto timeout = std::chrono::system_clock::now() + milliseconds(50);
  rpc_done_cv_.wait_until(*lock, timeout);
}

or even better to use wait_for() which takes the relative timeout.
Or should we use our ConditionVariable wrapper? Especially if we want to start instrumenting these things better. But if it's work to switch it over, it's okay to keep it condition_variable, but let's at least make the code more straight forward.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/row-batch.h
File be/src/runtime/row-batch.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/row-batch.h@52
PS3, Line 52: struct ProtoRowBatch {
I think we should get rid of this structure all together. IMO, the indirection just adds confusion.

On the receive side, it seems we can just get the header and sidecars directly from the request, which is already threaded through the RPC handler anyway.  Pulling it into a ProtoRowBatch just makes it unclear where the not yet deserialized rowbatch comes from.

On the send side, I think we should just work directly on CachedProtoRowBatch (but rename that thing, see below). The indirection through the ProtoRowBatch pointers (aka Slice) makes the lifetime and ownership harder to reason about.

I also found the name confusing this only the header is protobuf. The rest is KRPC specific stuff.

Any reason we shouldn't eliminate this?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/row-batch.h@89
PS3, Line 89: CachedProtoRowBatch
what is "cached" about this?

How about calling this OutboundRowBatch, RpcRowBatch, or SerializedRowBatch?

The first name maybe is best since it can really only be used for outbound, it seems.

ProtoRowBatch seems unnecessary in this case too, since we can just create the Slice on-the-fly when we want to send the OutboundRowBatch, no?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/row-batch.h@431
PS3, Line 431: 
             :   /// Overload for testing that allows the test to force the deduplication level.
             :   Status Serialize(TRowBatch* output_batch, bool full_dedup);
             : 
             :   /// Shared implementation between thrift and protobuf to serialize this row batch.
             :   ///
             :   /// 'full_dedup': true if full deduplication is used.
             :   ///
             :   /// 'tuple_offsets': Updated to contain offsets of all tuples into 'tuple_data' upon
             :   /// return . There are a total of num_rows * num_tuples_per_row offsets. An offset
             :   /// of -1 records a NULL.
             :   ///
             :   /// 'tuple_data': Updated to hold the serialized tuples' data. If 'compression_type'
             :   /// is THdfsCompression::LZ4, this is LZ4 compressed.
             :   ///
             :   /// 'uncompressed_size': Updated with the uncompressed size of 'tuple_data'.
             :   ///
             :   /// 'compression_type': Updated with the compression type applied on 'tuple_data'.
             :   /// THdfsCompression::NONE if there is no compression applied.
             :   ///
             :   /// Returns error status if serialization failed. Returns OK otherwise.
             :   Status Serialize(bool full_dedup, vector<int32_t>* tuple_offsets, string* tuple_data,
             :       int64_t* uncompressed_size, THdfsCompression::type* compression_type);
             : 
             :   /// Shared implementation between thrift and protobuf to deserialize a row batch.
             :   ///
             :   /// 'input_tuple_data': contains pointer and size of tuples' data buffer.
             :   /// If 'compression_type' is not THdfsCompression::NONE, tuple data is compressed.
             :   ///
             :   /// 'input_tuple_offsets': an int32_t array of tuples; offsets into 'input_tuple_data'.
             :   /// Used for populating the tuples in the row batch with actual pointers.
             :   ///
             :   /// 'uncompressed_size': the uncompressed size of 'input_tuple_data' if it's compressed.
             :   ///
             :   /// 'compression_type': If 'input_tuple_data' is compressed, it's the compression
             :   /// codec used.
             :   ///
             :   void Deserialize(const kudu::Slice& input_tuple_data,
             :       const kudu::Slice& input_tuple_offsets, int64_t uncompressed_size,
             :       THdfsCompression::type compression_type);
let's leave a TODO about cleaning this all up once we can remove the thrift implementation. do we have a JIRA to do that (not for milestone 1)?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/service/data-stream-service.cc
File be/src/service/data-stream-service.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/service/data-stream-service.cc@64
PS3, Line 64:   ProtoRowBatch batch;
            :   Status status = FromKuduStatus(context->GetInboundSidecar(
            :       request->row_batch_header().tuple_data_sidecar_idx(), &batch.tuple_data));
            :   if (status.ok()) {
            :     status = FromKuduStatus(context->GetInboundSidecar(
            :         request->row_batch_header().tuple_offsets_sidecar_idx(), &batch.tuple_offsets));
            :   }
            :   if (status.ok()) {
            :     batch.header = request->row_batch_header();
see comment in row-batch.h.  I think we should do this later when we actually want to deserialize the row batch.  We have to keep the 'request' around until that time anyway, right?



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 3
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Wed, 25 Oct 2017 17:19:38 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change.

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 1:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

Line 129: /// In exceptional circumstances, the data stream manager will garbage-collect the closed
There's a pre-existing flaw in the reasoning here that we should call out. "Exceptional circumstances" is vague and I think hides a distinction between an unhealthy cluster with extreme delays and the expected behaviour of certain long-running queries. I think the problem is an invalid assumption that the the receiver sends batches on a regular cadence with a bounded delay before the first batch is sent and when each subsequent batch is sent. That assumption is incorrect. I think we should call it out in this comment so that readers understand the current flaw. Here's an example where it's wrong.

Consider a plan with three fragments.

  F1 (long-running)
   |
   V
  F2 (limit = 1 on exchange)
   |
   V
  F3 (long-running selective scan)

1. The fragments all start up.
2. Instance 1 of F3 immediately finds and returns a matching row, which is sent to F2.
3. This causes F2 to hit its limit, close its exchange and tear itself down.
4. Let's assume F1 also has a lot of work to do and won't finish for 20 minutes
5. Instance 2 of F3 is still churning away on the scan. After 10 minutes it finally find a matching row.
6. F3 tries to send the row, can't find the receiver after a timeout and returns an error to the coordinator
7. The coordinator cancels the query and returns an error

There are two problems here:
1. The query failed when it shouldn't have
2. F3 wasn't cancelled when it was no longer needed and used lots of resources unnecessarily.

The JIRA is IMPALA-3990. I believe that the main reason we haven't seen this in practice is that it can only occur when there's a limit without order in a subquery. Most queries with that property are non-deterministic and it doesn't really make a lot of sense to have a long-running query that returns non-deterministic results.

But this actually blocked me from implementing early-close for joins with empty build sides, which is a nice optimisations.

There may also be a slightly different invalid assumption that the time between the receiver closing the exchange and the sender sending its last batch is bounded. That seems possible to solve with sender-side state if the receiver notifies the sender that the receiver was not present and the sender can infer it was closed cleanly.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-PatchSet: 1
Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 13: Verified+1


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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 13
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Thu, 09 Nov 2017 20:05:07 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Sailesh Mukil (Code Review)" <ge...@cloudera.org>.
Sailesh Mukil has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 3:

(23 comments)

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/rpc/rpc-mgr.cc
File be/src/rpc/rpc-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/rpc/rpc-mgr.cc@77
PS3, Line 77: 
Should we add a log message stating which services we registered with KRPC?

It might be useful later on as we add more services, while trying to debug user issues to know which services are on KRPC and which are on thrift. Granted there are other ways to find that, but this is easily accessible and straightforward.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@314
PS3, Line 314: w
susper-nit: Capital 'W'


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc@198
PS1, Line 198: deseria
> Deserialization pool's purpose is to avoid executing deserialization in lin
Hmm, this seems like it would be a nice thing to have. Is the absence of a MemTracker the only hindrance to early deserialization?

Is there some way we could add this to the process MemTracker if we can't attribute it to a query? If it's too complicated for now, let's track this with a JIRA and write down some ideas there for the next KRPC milestone.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc@227
PS1, Line 227:     // to make sure that the close operation is performed so add to per-recvr list of
             :     // pending closes. It's possible for a sender to issue EOS RPC without sending any
             :     // rows if no rows are m
> This may be a bit subtle but this is equivalent to the logic in the non-KRP
Ah you're right. Case 2 is what changed in IMPALA-5199, but it looks like that's automatically fixed here.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@112
PS3, Line 112:   for (unique_ptr<TransmitDataCtx>& ctx : early_senders.waiting_sender_ctxs) {
             :      EnqueueRowBatch({recvr->fragment_instance_id(), move(ctx)});
             :      num_senders_waiting_->Increment(-1);
             :   }
             :   for (unique_ptr<EndDataStreamCtx>& ctx : early_senders.closed_sender_ctxs) {
             :      recvr->RemoveSender(ctx->request->sender_id());
             :      Status::OK().ToProto(ctx->response->mutable_status());
             :      ctx->context->RespondSuccess();
             :      num_senders_waiting_->Increment(-1);
             :   }
It's not possible for the same sender to be in waiting_senders_ctxs and closed_sender_ctxs for a given receiver right?

Because if it would, it would make more sense to service the 'closed_sender_ctxs' before the 'waiting_sender_ctxs' since we may as well close the receiver instead of wasting CPU processing those RPCs for a while.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@140
PS3, Line 140:   while (range.first != range.second) {
             :     shared_ptr<KrpcDataStreamRecvr> recvr = range.first->second;
             :     if (recvr->fragment_instance_id() == finst_id &&
             :         recvr->dest_node_id() == node_id) {
             :       return recvr;
             :     }
             :     ++range.first;
             :   }
I'm thinking it makes sense to prioritize finding the receiver with the assumption that we will find it in the receiver_map_, rather than assume that it most likely will already be unregistered.

In other words, I think it may be more beneficial CPU-wise for general workloads to look in the 'receiver_map_' before looking into the 'closed_stream_cache_'.

What do you think?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@151
PS3, Line 151: KrpcDataStreamMgr::AddEarlySender
We could merge the implementations of AddEarlySender() and AddEarlyClosedSender() by using templates and some extra params, but maybe the code complexity isn't worth it.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@174
PS3, Line 174:     // that it is still preparing, so add payload to per-receiver list.
Add comment "In the worst case, this RPC is so late that the receiver is already unregistered and removed from the closed_stream_cache_, in which case it will be responded to by the Maintenance thread after FLAGS_datastream_sender_timeout_ms."


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@242
PS3, Line 242:  {
             :     // Remove any closed streams that have been in the cache for more than
             :     // STREAM_EXPIRATION_TIME_MS.
             :     lock_guard<mutex> l(lock_);
             :     ClosedStreamMap::iterator it = closed_stream_expirations_.begin();
             :     int64_t now = MonotonicMillis();
             :     int32_t before = closed_stream_cache_.size();
             :     while (it != closed_stream_expirations_.end() && it->first < now) {
             :       closed_stream_cache_.erase(it->second);
             :       closed_stream_expirations_.erase(it++);
             :     }
             :     DCHECK_EQ(closed_stream_cache_.size(), closed_stream_expirations_.size());
             :     int32_t after = closed_stream_cache_.size();
             :     if (before != after) {
             :       VLOG_QUERY << "Reduced stream ID cache from " << before << " items, to " << after
             :                  << ", eviction took: "
             :                  << PrettyPrinter::Print(MonotonicMillis() - now, TUnit::TIME_MS);
             :     }
             :   }
Historically, we never had a maintenance thread, which is why we did the stream cache eviction here.

However, now that we have the maintenance thread, it might make sense to move this logic there. What do you think?

It would mean faster responses to EOS RPCs


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@289
PS3, Line 289: const string msg = Substitute(
             :       "Unknown row receiver id: fragment_instance_id=$0, node_id=$1",
             :       PrintId(finst_id), node_id);
             :   LOG(ERROR) << msg;
Should this be a DCHECK instead?

Only the ExchangeNode is in charge of deregistering a receiver, which is controlled by a single thread. So this shouldn't happen at all, right?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.cc@303
PS3, Line 303: shared_ptr<KrpcDataStreamRecvr> recvr = FindRecvr(i->first, i->second, &unused);
Just realized that we could do this more efficiently. Instead of doing an O(n) FindRecvr() for every target receiver, we can just write a function that goes through the list once and compares against the finst_id and cancels each receiver.

Outside the scope of this patch though, leave a TODO if you agree.

Even better but a larger change, there's no reason why we have to call DataStreamMgr::Cancel() per fragment instance, since it's a singleton. We can just call DataStreamMgr::Cancel() once with the query ID and do all the cancellation in one O(n) lookup.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@225
PS1, Line 225: 
> We bumped recvr_->num_buffered_bytes_ at line 223 above so other callers of
But batch_queue_ will still be empty, so other callers of AddBatch() will skip the L213 check, and hence still get enqueued.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@271
PS1, Line 271: data_arrival_cv_.notify_all();
> why ?
Actually, nvm, we've already set is_cancelled_ = true while holding the lock.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.cc@253
PS3, Line 253:   VLOG_FILE << "decremented senders: fragment_instance_id="
             :             << recvr_->fragment_instance_id()
             :             << " node_id=" << recvr_->dest_node_id()
             :             << " #senders=" << num_remaining_senders_;
Is this that important that we have to do it while holding the lock?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@152
PS3, Line 152:   scoped_ptr<RowBatch> batch_;
Add "Not used for unpartitioned and random partitioned types".


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@178
PS3, Line 178:   // Protobuf request and response buffer for TransmitData() RPC.
             :   TransmitDataRequestPB req_;
             :   TransmitDataResponsePB resp_;
             : 
             :   // Protobuf request and response buffer for EndDataStream() RPC.
             :   EndDataStreamResponsePB eos_resp_;
             :   EndDataStreamRequestPB eos_req_;
Why do we need to store these in the class? Can't they be local in the functions that they're used in?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@229
PS3, Line 229: re-invoke
nit: re-invokes or retries


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@242
PS3, Line 242:   Status WaitForRpc(std::unique_lock<SpinLock>* lock) WARN_UNUSED_RESULT;
Add a comment "Should only be called from the main fragment instance thread."


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@290
PS3, Line 290:   // the header of the current serialized row batch.
It would be nice to separate out the responsibility of setting of certain state to different threads.

Eg: Here, add a comment saying "Should always be called in the context of a reactor thread".

So that any future work done on this file doesn't violate this rule by mistake.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@397
PS3, Line 397: proto_batch
CurrentProtoBatch().

Local var above is not needed.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@402
PS3, Line 402:       rpc_status = Status(resp_.status());
Would the case where the RPC got cancelled from L547 fall here? Or would it go to L406 (which it shouldn't)?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@528
PS3, Line 528:     RETURN_IF_ERROR(WaitForRpc(&l));
if (UNLIKELY(remote_recvr_closed_)) return Status::OK(); ?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/service/data-stream-service.cc
File be/src/service/data-stream-service.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/service/data-stream-service.cc@49
PS3, Line 49:   ExecEnv::GetInstance()->KrpcStreamMgr()->CloseSender(finst_id, request,
nit: "CloseSender() is guaranteed to eventually respond to this RPC, so we don't do it here."



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 3
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Wed, 11 Oct 2017 21:40:10 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 9:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.h
File be/src/runtime/krpc-data-stream-recvr.h:

http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.h@128
PS9, Line 128: Takes over the RPC payload of an early sender to 'deferred_rpcs_' queue of the
Takes over the RPC state of an early sender for deferred processing and kicks...



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 9
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Wed, 08 Nov 2017 00:54:50 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Hello Sailesh Mukil, Mostafa Mokhtar, Dan Hecht, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/8023

to look at the new patch set (#13).

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents avoids the possibility that a thread is stuck in the RPC code
for extended amount of time without checking for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, the number of service threads equals the
number of logical cores. The service threads are shared across all queries so
the RPC handler should avoid blocking as much as possible. In thrift RPC
implementation, we make a thrift thread handling a TransmitData() RPC to block
for extended period of time when the receiver is not yet created when the call
arrives. In KRPC implementation, we store TransmitData() or EndDataStream()
requests which arrive before the receiver is ready in a per-receiver early
sender list stored in KrpcDataStreamMgr. These RPC calls will be processed
and responded to when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a queue for deferred processing.
The stashed RPC requests will not be responded to until they are processed
so as to exert back pressure to the senders. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Builds {exhaustive/debug, core/release, asan} passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/exec/kudu-util.h
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/data-stream-sender.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M cmake_modules/FindProtobuf.cmake
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
33 files changed, 3,155 insertions(+), 184 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/13
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 13
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Mostafa Mokhtar (Code Review)" <ge...@cloudera.org>.
Mostafa Mokhtar has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 3:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.cc@644
PS3, Line 644:     for (int i = 0; i < channels_.size(); ++i) {
The RowBatch is serialized once per channel which is very wasteful. 
IMPALA-6041.
Compare to https://github.com/michaelhkw/incubator-impala/blob/krpc-testing-hung/be/src/runtime/data-stream-sender.cc#L429



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 3
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Wed, 11 Oct 2017 19:14:14 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Hello Sailesh Mukil, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/8023

to look at the new patch set (#3).

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents some issues with thrift RPC in which a thread may be stuck in
an RPC call without being able to check for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, there are 64 service threads. The service
threads are shared across all queries so the RPC handler should avoid
blocking as much as possible. In thrift RPC implementation, we make a thrift
thread handling a TransmitData() RPC to block for extended period of time
when the receiver is not yet created when the call arrives. In KRPC
implementation, we store TransmitData() or EndDataStream() requests
which arrive before the receiver is ready in a per-receiver early sender list
stored in KrpcDataStreamMgr. These RPC calls will be processed and responded to
when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a blocked_sender_ queue to be processed
later. The stashed RPC request will not be responded to until it is processed
so as to exert back pressure to the client. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Build passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M be/src/util/network-util.cc
M cmake_modules/FindProtobuf.cmake
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
32 files changed, 2,867 insertions(+), 166 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/3
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 3
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 7:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@199
PS7, Line 199:   DeserializeTask payload =
             :       {DeserializeTaskType::EARLY_SENDERS, finst_id, dest_node_id, 0};
             :   deserialize_pool_.Offer(move(payload));
> doesn't this mean we make early sender draining single threaded? shoudl we 
Yes. Actually, I just realized that early senders may also be passed by incoming row batches before they are deserialized by the deserialization thread pool, leading to extended response time for early senders.

A simpler scheme would be to actually put the early senders into the 'deferred_batches' queue of the corresponding sender's queue so new incoming row batches cannot pass it.

Regarding the parallelism for draining the deferred_batches queue, one simple thing to do is to enqueue as many deserialization requests as there are entries in the 'deferred_batch' queue. The deserialization thread logic will simply peek the first entry of the queue and try to insert it if there is space. An entry is popped off the queue only if it can be inserted. This may be wasteful if the 'batch_queue' fills up before all deserialization thread requests are drained but hopefully the peeking logic shouldn't take too long. We can be more fancy and record the deserialized size of each entry in deferred_batches_ and determine how many entries we can pop off deferred_batches_ queue without going over the limit.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@235
PS7, Line 235:     for (const unique_ptr<TransmitDataCtx>& ctx : early_senders.waiting_sender_ctxs) {
> shouldn't we process waiting_sender_ctxs before closed_sender_ctxs? Otherwi
Yes, it's impossible for the same sender in both queues at the same time but yeah, I can switch the order if it's easier to understand.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 7
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Sat, 04 Nov 2017 04:00:00 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Hello Sailesh Mukil, Mostafa Mokhtar, Dan Hecht, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/8023

to look at the new patch set (#5).

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents some issues with thrift RPC in which a thread may be stuck in
an RPC call without being able to check for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, there are 64 service threads. The service
threads are shared across all queries so the RPC handler should avoid
blocking as much as possible. In thrift RPC implementation, we make a thrift
thread handling a TransmitData() RPC to block for extended period of time
when the receiver is not yet created when the call arrives. In KRPC
implementation, we store TransmitData() or EndDataStream() requests
which arrive before the receiver is ready in a per-receiver early sender list
stored in KrpcDataStreamMgr. These RPC calls will be processed and responded to
when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a blocked_sender_ queue to be processed
later. The stashed RPC request will not be responded to until it is processed
so as to exert back pressure to the client. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Build passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/exec/kudu-util.h
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/data-stream-sender.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M be/src/util/network-util.cc
M cmake_modules/FindProtobuf.cmake
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
34 files changed, 2,929 insertions(+), 175 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/5
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 5
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 10:

(3 comments)

http://gerrit.cloudera.org:8080/#/c/8023/10/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/10/be/src/runtime/krpc-data-stream-recvr.cc@408
PS10, Line 408: deferred_rpcs_.front().release();
not needed.


http://gerrit.cloudera.org:8080/#/c/8023/10/be/src/runtime/krpc-data-stream-recvr.cc@425
PS10, Line 425: COUNTER_ADD(recvr_->num_deferred_batches_, 1);
can be done without holding the lock.


http://gerrit.cloudera.org:8080/#/c/8023/10/be/src/runtime/row-batch.h
File be/src/runtime/row-batch.h:

http://gerrit.cloudera.org:8080/#/c/8023/10/be/src/runtime/row-batch.h@145
PS10, Line 145: tuple_ofsets
typo.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 10
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Wed, 08 Nov 2017 19:32:05 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Hello Sailesh Mukil, Mostafa Mokhtar, Dan Hecht, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/8023

to look at the new patch set (#9).

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents some issues with thrift RPC in which a thread may be stuck in
an RPC call without being able to check for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, there are 64 service threads. The service
threads are shared across all queries so the RPC handler should avoid
blocking as much as possible. In thrift RPC implementation, we make a thrift
thread handling a TransmitData() RPC to block for extended period of time
when the receiver is not yet created when the call arrives. In KRPC
implementation, we store TransmitData() or EndDataStream() requests
which arrive before the receiver is ready in a per-receiver early sender list
stored in KrpcDataStreamMgr. These RPC calls will be processed and responded to
when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a blocked_sender_ queue to be processed
later. The stashed RPC request will not be responded to until it is processed
so as to exert back pressure to the client. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Build passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/exec/kudu-util.h
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/data-stream-sender.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M cmake_modules/FindProtobuf.cmake
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
33 files changed, 3,156 insertions(+), 183 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/9
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 9
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Impala Public Jenkins (Code Review)" <ge...@cloudera.org>.
Impala Public Jenkins has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 13:

Build started: https://jenkins.impala.io/job/gerrit-verify-dryrun/1454/


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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 13
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Impala Public Jenkins
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Thu, 09 Nov 2017 16:44:09 +0000
Gerrit-HasComments: No

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 7:

(37 comments)

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h@102
PS7, Line 102: the batch is added
             : /// to the receiver's 'deferred_batches_'
it's not really the batch added. and it's not just a single structure for the receiver (it may go into one of many queues for merging exchange). So how about saying:
... the RPC state is saved into a deferred queue.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h@104
PS7, Line 104: from the pending sender queue
how about: ... from a deferred RPC queue and the row batch is deserialized.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.h@393
PS7, Line 393: 
quick comment for why we define a move constructor and move operator=, since we don't typically want to define those.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@199
PS7, Line 199:   DeserializeTask payload =
             :       {DeserializeTaskType::EARLY_SENDERS, finst_id, dest_node_id, 0};
             :   deserialize_pool_.Offer(move(payload));
doesn't this mean we make early sender draining single threaded? shoudl we instead use the sender_id in this case as well and offer work per sender? or do we think this doesn't matter?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@217
PS7, Line 217: already_unregistered
that shouldn't be possible in the DEFERRED_BATCHES case, right? so i'd probably move this DCHECK into the cases below so you can tighten it up.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@235
PS7, Line 235:     for (const unique_ptr<TransmitDataCtx>& ctx : early_senders.waiting_sender_ctxs) {
shouldn't we process waiting_sender_ctxs before closed_sender_ctxs? Otherwise, if the same sender is in both lists we'll process those RPCs out of order. I guess that can't really happen given the current implementation of not responding to early RPCs and that senders only let one in flight, but it still seems to make more sense to do it the other way around.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@236
PS7, Line 236: already_unregistered
why is this possible in the waiting_sender_ctxs case but not the closed_sender_ctxs case?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@247
PS7, Line 247: already_unregistered
why is that possible?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-mgr.cc@248
PS7, Line 248: recvr->AddDeferredBatches(task.sender_id);
So I guess we no longer multithread within a single sender queue (and for non-merging, within a single receiver) doing it this way. I think that's okay but was it intentional?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.h
File be/src/runtime/krpc-data-stream-recvr.h:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.h@78
PS7, Line 78: The caller must acquire data from the
            :   /// returned batch
is that talking about calling TransferAllResources(), or can the caller do it directly?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@127
PS6, Line 127:   // If true, the receiver fragment for this stream got cancelled.
> For the non-merging case, there is essentially only one queue.
As mentioned elsewhere, I'm not totally convinced yet that this is the right way to do it but, yes, we can think about it more and change it later if necessary.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@72
PS7, Line 72: s
typo


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@77
PS7, Line 77: Adds as many deferred batches as possible
hmm I'm still not convinced this is the right thing to do (in the merging case). It seems like it's left up to chance as to the order that deferred batches are drained across the sender queues. But we can think about this more and address it later.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@97
PS7, Line 97: (1) 'batch_queue' is empty and there is no pending insertion
the HasSpace name seems wrong for condition (1). From the name HasSpace, I was expecting it only to check for condition (2) because that tells us if there is still space.  How about calling this CanEnqueue() or ShouldEnqueue()?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@113
PS7, Line 113: size
deserialized or serialized size?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@169
PS7, Line 169: deferred batches
these aren't really deferred batches though. They are deferred RPCs (which do contain batches). If you like the "deferred batches" terminology, I can live with it, but it seems somewhat misleading (though not worse than "blocked senders").


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@220
PS7, Line 220:     // if we haven't notified them already.
rather than explaining what, we should explain why:
We've either removed a batch from the sender queue (and so now there might be space to move in a deferred batch), or the sender queue is empty. In either case, try to process the deferred batches and move them to the sender queue.

Though can the second case (empty case) even happen? Shouldn't we have already started this process if the queue was empty.

That is, why is this so complicated. Why can't it just be:

(1) while batch_queue_.empty() wait for data arrival.
(2) Dequeue from batch queue and trigger deferred batch draining. 

i.e. that first while loop shoudn't need to care about the deferred_batches_ state. Whoever made the batch_queue_ empty should be responsible for triggering the draining so we can just wait for the arrival. And if there was nothing to drain, then the next batch to arrive would immediately go into the batch_queue_ anyway.  So it seems like this can all be simplified, am I missing something?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@223
PS7, Line 223: deferred_batches_.front()->request->sender_id();
that seems misleading in the case of !merging. sender_id will be some random one, but we'll funnel back into 0 anyway.  But I guess this sender queue doesn't know what it's own sender id is, so okay.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@224
PS7, Line 224:       l.unlock();
why do we need to drop the lock?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@263
PS7, Line 263:   COUNTER_ADD(recvr_->bytes_received_counter_, batch_size);
let's add a comment:
// Reserve queue space before dropping the lock.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@269
PS7, Line 269: deserialize multiple batches in parallel.
I guess this only happens in the non-merging case, right?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@274
PS7, Line 274: DCHECK_GT(num_pending_enqueue_, 0);
I'm not sure what this dcheck is meant to prove.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@322
PS7, Line 322: starvation
starvation of a sender in the non-merging case.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@324
PS7, Line 324: Note: It's important that we enqueue the new batch regardless of buffer limit if
             :     // the queue is currently empty. In the case of a merging receiver, batches are
             :     // received from a specific queue based on data order, and the pipeline will stall
             :     // if the merger is waiting for data from an empty queue that cannot be filled because
             :     // the limit has been reached.
that comment shoudl be inside HasSpace() now.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@345
PS7, Line 345: void KrpcDataStreamRecvr::SenderQueue::AddDeferredBatches() {
are we able to exercise the deferred batches path in functional testing? if not, I think we should figure out a way.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-recvr.cc@531
PS7, Line 531:   // All the sender queues will be cancelled after this call returns.
is that accurate?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@369
PS6, Line 369:     // the need to manage yet another thread pool.
> Mostly to avoid the complexity of managing yet another thread pool.
Okay. I thought maybe to keep things cache local.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc@235
PS7, Line 235: call
nit: delete call too. 'C' of RPC is call.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc@257
PS7, Line 257: (e.g. Connection object was shutdown due to network errors)
with thrift RPC, wouldn't we have retried making a connection and doing the RPC in this case?  and now with KRPC we will fail the query? Is that true?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc@273
PS7, Line 273: current_batch_
rpc_in_flight_batch_


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc@554
PS7, Line 554:     while (rpc_in_flight_) {
             :       rpc_done_cv_.wait(l);
             :     }
             :   }
one liner?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/krpc-data-stream-sender.cc@558
PS7, Line 558:   DCHECK(!rpc_in_flight_);
that becomes vacuous given the while loop condition.


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.h
File be/src/runtime/row-batch.h:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.h@349
PS7, Line 349: output_batch
outbound_batch (or rename the param)


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.cc
File be/src/runtime/row-batch.cc:

http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.cc@426
PS7, Line 426:   result += header.num_tuples_per_row() * sizeof(int32_t);
could you order this computation in the same order as thrift, to make it easy to see they are doing the same thing?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.cc@432
PS7, Line 432: batch.tuple_offsets_.size();
what is this trying to compute? the size of the tuple_ptrs_? if so, it doesn't look right since each offset will expand into a pointer, right? i.e. shouldn't it be tuple_offsets_.size() * sizeof(Tuple*).

And the thrift version looks wrong too.  And the Slice version is also different and wrong (since size() is different for Slice vs vector).


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.cc@433
PS7, Line 433: batch.header_.num_tuples_per_row() * sizeof(int32_t);
what is this accounting for?


http://gerrit.cloudera.org:8080/#/c/8023/7/be/src/runtime/row-batch.cc@440
PS7, Line 440: batch.header_.num_tuples_per_row() * sizeof(int32_t);
what's that accounting?



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 7
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Fri, 03 Nov 2017 23:07:09 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 6:

(15 comments)

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@123
PS6, Line 123: until
             :   // any in-flight RPC completes
if the preceding RPC is still in-flight.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@135
PS6, Line 135: free 
frees


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@258
PS6, Line 258: stack
code


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@261
PS6, Line 261: stack
code


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@264
PS6, Line 264: thread
threads


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@279
PS6, Line 279: thread
threads


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h
File be/src/runtime/row-batch.h:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@104
PS6, Line 104: CachedProtobufRowBatch
OutboundRowBatch


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@144
PS6, Line 144:   /// Populate a row batch from a serialized protobuf input_batch by copying
             :   /// input_batch's tuple_data into the row batch's mempool and converting all
             :   /// offsets in the data back into pointers.
stale


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@455
PS6, Line 455: tuple_offsets
input_*


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@531
PS6, Line 531: CachedProtobufRowBatch
OutboundProtoRowBatch


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/util/network-util.cc
File be/src/util/network-util.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/util/network-util.cc@41
PS6, Line 41: using kudu::Sockaddr;
undo. Bad rebase.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/util/network-util.cc@120
PS6, Line 120: Sockaddr sock;
undo


http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/data_stream_service.proto
File common/protobuf/data_stream_service.proto:

http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/data_stream_service.proto@38
PS6, Line 38: 'tuple_offsets'
tuple offsets' buffer


http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/data_stream_service.proto@43
PS6, Line 43: 'tuple_data'
tuple data's buffer


http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/row_batch.proto
File common/protobuf/row_batch.proto:

http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/row_batch.proto@25
PS6, Line 25: The indices of the sidecars are included in the header below.
delete



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 6
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Fri, 27 Oct 2017 22:41:44 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 3:

(34 comments)

Another batch of comments...

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@63
PS3, Line 63: ongoing transmission from a client to a server as a 'stream'
I don't think that's accurate. see questions in krpc-data-stream-recvr.h about stream definition.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@93
PS3, Line 93: process it immediately, add it to a fixed-size 'batch queue' for later processing
XXX check whether these are really different


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@164
PS3, Line 164: deferred processing
is that because the recvr hasn't showed up yet, or because the stream's queue is full, or both?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@235
PS3, Line 235: node_id
dest_node_id?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@239
PS3, Line 239: Ownership of the receiver is shared between this DataStream mgr instance and the
             :   /// caller.
that seems unnecessary but don't change it now.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@246
PS3, Line 246:  
'proto_batch'?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@248
PS3, Line 248: .
'request'.

Also document what 'response' and 'context' are.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@266
PS3, Line 266: Notifies the receiver 
is this an RPC handler? I think we should just be explicit about which of these methods are RPC handlers.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@267
PS3, Line 267: The RPC
what RPC is this talking about? If this is a handler, then it's clear.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@274
PS3, Line 274: Closes
Does it close or cancel? (or is there no difference?)


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@284
PS3, Line 284: RPCs which were buffered
To be consistent with terminology used in class comment, maybe say "deferred RPCs"


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@340
PS3, Line 340: ragment instance id, 0
what is that saying? is that a misplaced comma or am I reading this wrong?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@341
PS3, Line 341: instance id changes
I don't understand this.  it kinda sounds like we're trying to be able to find all instances of this fragment, but then wouldn't we iterate until the fragment id changes (not the instance id)?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@349
PS3, Line 349:   struct EarlySendersList {
hmm, I guess we need this now that we can't block the RPC thread?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@358
PS3, Line 358: Time
Monotonic time


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@374
PS3, Line 374: time
monotonic time


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@382
PS3, Line 382:   boost::unordered_set<RecvrId> closed_stream_cache_;
all this parallel startup stuff really needs to be revisited (but not for this change). it's too complex and brittle.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@386
PS3, Line 386: Deserialize
maybe call it DeserializeDeferred() or DeserializeWorker() to make it clearer that this is only for the deferred (slow) path, since the normal path will also have to deserialize (but doesn't use this code).


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@404
PS3, Line 404:   void EnqueueRowBatch(DeserializeWorkItem&& payload);
how about grouping this with Deferred function above since it's related. Also, I think the name should be less generic. Like maybe EnqueueDeferredBatch() or EnqueueDeferredRpc() (does the response happen before or after this deferred work?)


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@413
PS3, Line 413: block
what's that?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@414
PS3, Line 414: Status
I think that status is not getting checked by the caller. I thought Tim made Status warn on unused result -- why is it not catching this? (Or do we still need to annotate each method?).


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@416
PS3, Line 416:   inline uint32_t GetHashValue(const TUniqueId& fragment_instance_id, PlanNodeId node_id);
let's add a quick comment.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@421
PS3, Line 421: HandleTimedOutSenders
RespondToTimedOutSender() or RespondTimeOutToSender()?
Also the code only responds to a single sender, so the comment and name shouldn't be plural.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h
File be/src/runtime/krpc-data-stream-recvr.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h@48
PS3, Line 48: Single receiver of an m:n data streams
Either the "an" shouldn't be there or streams shouldn't be plural. But I'm not sure exactly how we define the "stream". Is it one pair of (sender,recvr), or is it the set of all (sender,recvr) pairs, or is it all pairs that contain this receiver (*, recvr)?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h@116
PS3, Line 116: discarded
that doesn't seem accurate


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h@117
PS3, Line 117: payload
what's that?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h@126
PS3, Line 126: Stream
mentioned above, we didn't clearly define what a "stream" actually is, so it's hard to know what this means.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h@141
PS3, Line 141: amount of buffering 
is this in bytes?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-recvr.h@142
PS3, Line 142: we stop acking
wouldn't it be more accurate (and consistent with mgr terminology) to say "we defer processing of incoming RPCs once..."


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.h
File be/src/runtime/krpc-data-stream-sender.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.h@53
PS3, Line 53: the buffer size in bytes allocated to each channel
it's not clear what that means from just reading the header, though i know you just carried this comment so okay to defer.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.h@59
PS3, Line 59: destinations
that's not documented.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.h@97
PS3, Line 97:       WARN_UNUSED_RESULT;
should that really be public? seems more like a worker function.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.h@101
PS3, Line 101:   int64_t GetNumDataBytesSent() const;
that seems weird. is it for testing? if so, can it be protected instead?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-sender.h@109
PS3, Line 109:       const TDataSink& tsink, RuntimeState* state) WARN_UNUSED_RESULT;
why is that protected? is this a testing thing?



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 3
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Thu, 12 Oct 2017 04:37:30 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 6:

(14 comments)

Note to self: remaining files: krpc-data-stream-{mgr,recvr}.cc

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/exec-env.cc
File be/src/runtime/exec-env.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/exec-env.cc@89
PS6, Line 89:     "Number of datastream service processing threads");
how are these defaults chosen?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h
File be/src/runtime/row-batch.h:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@50
PS6, Line 50: outbound
I think we should say something about KRPC to at least give that hint. maybe:

A KRPC outbound row batch...


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@60
PS6, Line 60: sizeof(int32_t)
sizeof(tuple_offsets_[0]) seems clearer and more robust


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@354
PS6, Line 354:   /// it is ignored. This function does not Reset().
we should preserve this comment when removing the thrift variant. So you could just put the new decl here now so we don't forget that.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@424
PS6, Line 424:   ///
nit: i don't think we generally have all these line breaks between parameter comments.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@426
PS6, Line 426:  .
delete space


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@444
PS6, Line 444: nput_
delete


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@447
PS6, Line 447: input_
delete


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.h@537
PS6, Line 537:   std::string compression_scratch_;
this seems like a hack and we could do something simpler, but let's leave it alone for now.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.cc
File be/src/runtime/row-batch.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/row-batch.cc@241
PS6, Line 241:   // as sidecars to the RpcController.
this comment was probably meant to be deleted?


http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/data_stream_service.proto
File common/protobuf/data_stream_service.proto:

http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/data_stream_service.proto@29
PS6, Line 29: fragment
isn't this the id of the instance?  The comment in KrpcDataStreamSender is clearer, let's copy that:
  /// Sender instance id, unique within a fragment.
  int sender_id_;


http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/data_stream_service.proto@59
PS6, Line 59:   // Id of this fragment in its role as a sender.
same


http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/row_batch.proto
File common/protobuf/row_batch.proto:

http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/row_batch.proto@32
PS3, Line 32: = 2;
> That's the tuple data sent as sidecar. Clarified in the new comments.
My point is that writing it like 'tuple_data' doesn't make sense since it's not a field in this struct. You should just write:
Size of the tuple data (sent as a sidecar) in bytes ...


http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/row_batch.proto
File common/protobuf/row_batch.proto:

http://gerrit.cloudera.org:8080/#/c/8023/6/common/protobuf/row_batch.proto@32
PS6, Line 32: epeated int32 row_tuples = 2;
why is this needed? i don't see it used. The size of it is used, though it seems like even that can be inferred from the descriptors.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 6
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Wed, 01 Nov 2017 21:48:21 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 2:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-recvr.cc@141
PS2, Line 141:   while (true) {
             :     // wait until something shows up or we know we're done
             :     while (!is_cancelled_ && batch_queue_.empty() && blocked_senders_.empty()
             :         && num_remaining_senders_ > 0) {
             :       VLOG_ROW << "wait arrival fragment_instance_id=" << recvr_->fragment_instance_id()
             :                << " node=" << recvr_->dest_node_id();
             :       // Don't count time spent waiting on the sender as active time.
             :       CANCEL_SAFE_SCOPED_TIMER(recvr_->data_arrival_timer_, &is_cancelled_);
             :       CANCEL_SAFE_SCOPED_TIMER(recvr_->inactive_timer_, &is_cancelled_);
             :       CANCEL_SAFE_SCOPED_TIMER(
             :           received_first_batch_ ? nullptr : recvr_->first_batch_wait_total_timer_,
             :           &is_cancelled_);
             :       data_arrival_cv_.wait(l);
             :     }
             : 
             :     if (is_cancelled_) return Status::CANCELLED;
             : 
             :     if (blocked_senders_.empty() && batch_queue_.empty()) {
             :       DCHECK_EQ(num_remaining_senders_, 0);
             :       return Status::OK();
             :     }
             : 
             :     received_first_batch_ = true;
             : 
             :     // Either we'll consume a row batch from batch_queue_, or it's empty. In either case,
             :     // take a blocked sender and retry delivering their batch. There is a window between
             :     // which a deferred batch is dequeued from blocked_senders_ queue and when it's
             :     // inserted into batch_queue_. However, a receiver won't respond to the sender until
             :     // the deferred row batch has been inserted. The sender will wait for all in-flight
             :     // RPCs to complete before sending EOS RPC so num_remaining_senders_ should be > 0.
             :     if (!blocked_senders_.empty()) {
             :       recvr_->mgr_->EnqueueRowBatch(
             :           {recvr_->fragment_instance_id(), move(blocked_senders_.front())});
             :       blocked_senders_.pop();
             :     }
             : 
             :     if (!batch_queue_.empty()) {
             :       RowBatch* result = batch_queue_.front().second;
             :       recvr_->num_buffered_bytes_.Add(-batch_queue_.front().first);
             :       VLOG_ROW << "fetched #rows=" << result->num_rows();
             :       current_batch_.reset(result);
             :       *next_batch = current_batch_.get();
             :       batch_queue_.pop_front();
             :       return Status::OK();
             :     }
> This loop may lead to live lock in the rare case in which blocked_senders_ 
Actually, mis-read the thing in the heat of debugging. If both queues are empty, we may return early in line 160 above if num_remaining_senders == 0. So, we shouldn't spin forever. Otherwise, the thread should sleep and wait in line 153. This loop tends to have the unfortunate behavior of popping all entries off blocked_senders_ first before dropping the lock and sleeping on line 153.

Although there is a window in which both queues are empty when a row batch is deserialized and moved from blocked_senders_ to batch_queue_, it should be impossible for num_remaining_senders_ to reach 0 in that window. The reason is that the sender of that row batch will not be responded to until after the row batch has been inserted into batch_queue_ (after it has been popped from blocked_senders_). In which case, batch_queue_ will become non-empty first before the remote sender gets a reply.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 2
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Fri, 06 Oct 2017 22:03:51 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Hello Sailesh Mukil, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/8023

to look at the new patch set (#2).

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents some issues with thrift RPC in which a thread may be stuck in
an RPC call without being able to check for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, there are 64 service threads. The service
threads are shared across all queries so the RPC handler should avoid
blocking as much as possible. In thrift RPC implementation, we make a thrift
thread handling a TransmitData() RPC to block for extended period of time
when the receiver is not yet created when the call arrives. In KRPC
implementation, we store TransmitData() or EndDataStream() requests
which arrive before the receiver is ready in a per-receiver early sender list
stored in KrpcDataStreamMgr. These RPC calls will be processed and responded to
when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a blocked_sender_ queue to be processed
later. The stashed RPC request will not be responded to until it is processed
so as to exert back pressure to the client. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Build passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
30 files changed, 2,822 insertions(+), 163 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/2
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 2
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 9:

(6 comments)

http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-mgr.cc@62
PS9, Line 62:       min(FLAGS_datastream_service_num_deserialization_threads, CpuInfo::num_cores()),
why disallow the setting of this greater than num_cores? Given that it takes locks, there could be some benefit to doing so, right?


http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.cc@67
PS9, Line 67: data
the resources


http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.cc@195
PS9, Line 195: cur_batch_
current_batch_


http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.cc@225
PS9, Line 225: DCHECK
shoudl we also DCHECK that num_pending_enqueue_ == 0 (otherwise, we could have acquired the lock while it was dropped for deserialization and there is still a row batch)?


http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.cc@248
PS9, Line 248: batch_queue_.pop_front();
this is part of the same operation as line 244 (both are adjusting the queue state), so mind moving it to be adjacent?


http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.cc@252
PS9, Line 252:     // Don't hold lock when calling EnqueueDeserializeTask() as it may block.
how about moving this to line 250 now that it's that scope that drops the lock. Also, it's important that this happens after batch_queue_.pop_front(), right? maybe note that.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 9
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Wed, 08 Nov 2017 01:32:58 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 10:

(6 comments)

http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-mgr.cc@62
PS9, Line 62:       FLAGS_datastream_service_num_deserialization_threads, 10000,
> why disallow the setting of this greater than num_cores? Given that it take
Yes. The assumption is that the critical section is small and most threads won't block for too long so no point in pushing above number of cores. To be consistent with other knobs we have (e.g. # service threads, reactor threads), we shouldn't impose an implicit upper bound.


http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.cc@67
PS9, Line 67: the
> the resources
Done


http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.cc@195
PS9, Line 195: current_ba
> current_batch_
Done


http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.cc@225
PS9, Line 225: DCHECK
> shoudl we also DCHECK that num_pending_enqueue_ == 0 (otherwise, we could h
Done


http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.cc@248
PS9, Line 248: current_batch_.reset(resu
> this is part of the same operation as line 244 (both are adjusting the queu
Done


http://gerrit.cloudera.org:8080/#/c/8023/9/be/src/runtime/krpc-data-stream-recvr.cc@252
PS9, Line 252:   // It's important that the dequeuing of 'deferred_rpcs_' is done after the entry
> how about moving this to line 250 now that it's that scope that drops the l
Done



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 10
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Wed, 08 Nov 2017 03:35:43 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 1:

(20 comments)

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@110
PS1, Line 110: sent
> nit: received
Done


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@110
PS1, Line 110: no TransmitData() RPCs will successfully deliver their
             : /// payload.
> Why would there be a TransmitData() RPC if EndDataStream() has already been
It's the expectation that the sender will not send any TransmitData() RPC after EndDataStream() RPC. Comments updated.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@129
PS1, Line 129: /// In exceptional circumstances, the data stream manager will garbage-collect the closed
Comments added.

 > sending its last batch is bounded. That seems possible to solve
 > with sender-side state if the receiver notifies the sender that the
 > receiver was not present and the sender can infer it was closed
 > cleanly.

Not sure I followed the proposed solution. The sender can track whether it has ever successfully sent a row batch to the receiver. However, in the example above, instance 2 of F3 has never sent a row batch to the receiver before it hits the limit and closes. In which case, it's not clear how the sender can differentiate between an early sender case (i.e. the receiver is still being prepared) vs a closed receiver.

It seems a more fool-proof solution is for coordinator to notify all backend nodes about completed/aborted/cancelled queries and that seems to be the absolutely safe point to remove closed stream entries. Alternately, we can use statestore update to broadcast this information and make the maintenance thread in DataStreamMgr remove the receiver entries based on the updates.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@133
PS1, Line 133: /// period expires.
> As per Tim's comment above, I would also reference IMPALA-3990 as a TODO he
Done


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@159
PS1, Line 159: Consider tracking, on the sender, whether a batch has been successfully sent or
             : ///   not. That's enough state to realise that a receiver has failed (rather than not
             : ///   prepared yet), and the data stream mgr can use that to fail an RPC fast, rather than
             : ///   having the closed-stream list.
> It would be nice to have a JIRA for this and reference it here.
Actually, now that I think about it, this idea may not work due to examples such as IMPALA-3990 above. The problem is that the receiver may have been closed (legitimately) even before a particular sender managed to send a batch to it. In which case, it would falsely assume that the receiver has failed. Similarly, if no rows were ever materialized from the sender side, we still need to closed-stream cache to differentiate between the closed receiver vs receiver which is still being prepared.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@353
PS1, Line 353: waiting_senders
> This is a little confusing to follow in the .cc file, since when I see "wai
Done


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@356
PS1, Line 356: closed_senders
> Similarly, we could call this 'closed_senders_ctxs'.
Done


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc@168
PS1, Line 168:       num_senders_waiting_->Increment(1);
             :       total_senders_waited_->Increment(1);
             :       RecvrId recvr_id = make_pair(fragment_instance_id, request->dest_node_id());
             :       auto payload =
             :           make_unique<TransmitDataCtx>(proto_batch, context, request, response);
             :       early_senders_map_[recvr_id].waiting_senders.push_back(move(payload));
> I'm wondering if it makes sense to add simple inline functions that encapsu
I don't find it too unreadable being inline but I guess it's less distracting if the logic is encapsulated in a function


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc@198
PS1, Line 198: AddData
> Isn't the point of the deserialize pool to deserialize the payload early?
Deserialization pool's purpose is to avoid executing deserialization in line in the main thread for early or blocked senders. For instances, if there are multiple row batches (from multiple early senders) for a given receiver, the deserialization thread can continue to deserialize the row batches in the queue while the main thread starts consuming the deserialized row batches.

While it may be ideal to deserialize the row batches for early senders, we may have a hard time accounting for the memory here as the MemTracker actually resides inside the receiver.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc@213
PS1, Line 213: If no receiver found, but not in the closed stream cache
> nit: If no receiver is found, and the receiver is not in the closed stream 
Done


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc@218
PS1, Line 218:       RecvrId recvr_id = make_pair(fragment_instance_id, dest_node_id);
             :       auto payload = make_unique<EndDataStreamCtx>(context, request, response);
             :       early_senders_map_[recvr_id].closed_senders.emplace_back(move(payload));
             :       num_senders_waiting_->Increment(1);
             :       total_senders_waited_->Increment(1);
> AddEarlyClosedSender() as per comment above, if you agree.
Done


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc@227
PS1, Line 227:   if (LIKELY(recvr != nullptr)) recvr->RemoveSender(request->sender_id());
             :   Status::OK().ToProto(response->mutable_status());
             :   context->RespondSuccess();
> This may need some modification based on the recent commit for IMPALA-5199:
This may be a bit subtle but this is equivalent to the logic in the non-KRPC implementation. There are 3 cases:

1. the receiver is not found and it has already been unregistered. In which case, there is not much we need to do and it's okay to return Status::OK().

2. the receiver is not found and it has not yet been unregistered, in which case, it's treated as early senders. In which case, we shouldn't reach here and we should return by line 223 above. The maintenance thread will be responsible for timing out EOS connections.

3. the receiver is found and we return Status::OK() in that case.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@75
PS1, Line 75: ("new data")
> I'm having some trouble understanding what this means. Could you please cla
Done


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@165
PS1, Line 165: Either we'll consume a row batch from batch_queue_, or it's empty
> Shouldn't there always be something in the batch_queue_ if there's somethin
The row batches in the blocked_senders_ queues are enqueued and deserialized in the context of the deserialization threads. In some cases, it's possible for the main thread to have exhausted the batch_queue_ before the deserialization threads get around to inserting entries in blocked_senders_ into batch_queue_.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@166
PS1, Line 166: There is a window
> Just to make things clearer, could you specify what there's a window for?
The deserialization thread pool is executing asynchronously to the main thread. Therefore, there may be a window in which a blocked senders' context is removed from blocked_senders_ and before it's enqueued into batch_queue_.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@225
PS1, Line 225: 
> There is a problem here. When we release lock_here, an arbitrary number of 
We bumped recvr_->num_buffered_bytes_ at line 223 above so other callers of AddBatch() shouldn't be able to go over the limits.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@271
PS1, Line 271: data_arrival_cv_.notify_all();
> Shouldn't this notify be done while holding the lock_ ?
why ?


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@284
PS1, Line 284:   for (const auto& queue_entry: batch_queue_) delete queue_entry.second;
> batch_queue_.clear() ?
Done


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@285
PS1, Line 285:   while (!blocked_senders_.empty()) {
> nit: Add comment: Respond to blocked senders' RPCs
Done


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-sender.cc@208
PS1, Line 208: %
> Should we do a bitwise (cur_batch_idx_ + 1) & 1 instead? Or would the compi
The compiler should take care of that.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Thu, 28 Sep 2017 18:28:31 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Hello Sailesh Mukil, Mostafa Mokhtar, Dan Hecht, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/8023

to look at the new patch set (#11).

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents avoids the possibility that a thread is stuck in the RPC code
for extended amount of time without checking for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, the number of service threads equals the
number of logical cores. The service threads are shared across all queries so
the RPC handler should avoid blocking as much as possible. In thrift RPC
implementation, we make a thrift thread handling a TransmitData() RPC to block
for extended period of time when the receiver is not yet created when the call
arrives. In KRPC implementation, we store TransmitData() or EndDataStream()
requests which arrive before the receiver is ready in a per-receiver early
sender list stored in KrpcDataStreamMgr. These RPC calls will be processed
and responded to when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a queue for deferred processing.
The stashed RPC requests will not be responded to until they are processed
so as to exert back pressure to the senders. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Builds {exhaustive/debug, core/release, asan} passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/exec/kudu-util.h
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/data-stream-sender.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M cmake_modules/FindProtobuf.cmake
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
33 files changed, 3,151 insertions(+), 184 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/11
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 11
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 2:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-recvr.cc@141
PS2, Line 141:   while (true) {
             :     // wait until something shows up or we know we're done
             :     while (!is_cancelled_ && batch_queue_.empty() && blocked_senders_.empty()
             :         && num_remaining_senders_ > 0) {
             :       VLOG_ROW << "wait arrival fragment_instance_id=" << recvr_->fragment_instance_id()
             :                << " node=" << recvr_->dest_node_id();
             :       // Don't count time spent waiting on the sender as active time.
             :       CANCEL_SAFE_SCOPED_TIMER(recvr_->data_arrival_timer_, &is_cancelled_);
             :       CANCEL_SAFE_SCOPED_TIMER(recvr_->inactive_timer_, &is_cancelled_);
             :       CANCEL_SAFE_SCOPED_TIMER(
             :           received_first_batch_ ? nullptr : recvr_->first_batch_wait_total_timer_,
             :           &is_cancelled_);
             :       data_arrival_cv_.wait(l);
             :     }
             : 
             :     if (is_cancelled_) return Status::CANCELLED;
             : 
             :     if (blocked_senders_.empty() && batch_queue_.empty()) {
             :       DCHECK_EQ(num_remaining_senders_, 0);
             :       return Status::OK();
             :     }
             : 
             :     received_first_batch_ = true;
             : 
             :     // Either we'll consume a row batch from batch_queue_, or it's empty. In either case,
             :     // take a blocked sender and retry delivering their batch. There is a window between
             :     // which a deferred batch is dequeued from blocked_senders_ queue and when it's
             :     // inserted into batch_queue_. However, a receiver won't respond to the sender until
             :     // the deferred row batch has been inserted. The sender will wait for all in-flight
             :     // RPCs to complete before sending EOS RPC so num_remaining_senders_ should be > 0.
             :     if (!blocked_senders_.empty()) {
             :       recvr_->mgr_->EnqueueRowBatch(
             :           {recvr_->fragment_instance_id(), move(blocked_senders_.front())});
             :       blocked_senders_.pop();
             :     }
             : 
             :     if (!batch_queue_.empty()) {
             :       RowBatch* result = batch_queue_.front().second;
             :       recvr_->num_buffered_bytes_.Add(-batch_queue_.front().first);
             :       VLOG_ROW << "fetched #rows=" << result->num_rows();
             :       current_batch_.reset(result);
             :       *next_batch = current_batch_.get();
             :       batch_queue_.pop_front();
             :       return Status::OK();
             :     }
This loop may lead to live lock in the rare case in which blocked_senders_ and batch_queue_ are both empty in the window in which the deserialization threads are still working on inserting the row batches in blocked_senders_ into batch_queue_ after batch_queue_ has been exhausted. Spinning here forever will not drop the lock, causing the deserialization threads to wait forever to insert into batch_queue_.


http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-sender.cc@485
PS2, Line 485:     // Sleep for sometime before retrying.
             :     if (RpcMgr::IsServerTooBusy(rpc_controller_)) {
             :       SleepForMs(FLAGS_rpc_retry_interval_ms);
             :       continue;
             :     }
This is broken if the RPC was rejected for FLAGS_backend_client_connection_num_retries number of times in a row. In which case, we will break out of the loop and return Status::OK(). This can lead to remote receiver hanging forever as it still thinks there are still active senders.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 2
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Fri, 06 Oct 2017 17:58:27 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 2:

(2 comments)

http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-sender.cc@103
PS2, Line 103: address_(MakeNetworkAddress(destination.hostname, destination.port)),
address_(destination)


http://gerrit.cloudera.org:8080/#/c/8023/2/be/src/runtime/krpc-data-stream-sender.cc@141
PS2, Line 141:  TNetworkAddress address_;
const TNetworkAddress address_;



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 2
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Fri, 29 Sep 2017 17:49:44 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Hello Sailesh Mukil, Mostafa Mokhtar, Dan Hecht, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/8023

to look at the new patch set (#12).

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents avoids the possibility that a thread is stuck in the RPC code
for extended amount of time without checking for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, the number of service threads equals the
number of logical cores. The service threads are shared across all queries so
the RPC handler should avoid blocking as much as possible. In thrift RPC
implementation, we make a thrift thread handling a TransmitData() RPC to block
for extended period of time when the receiver is not yet created when the call
arrives. In KRPC implementation, we store TransmitData() or EndDataStream()
requests which arrive before the receiver is ready in a per-receiver early
sender list stored in KrpcDataStreamMgr. These RPC calls will be processed
and responded to when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a queue for deferred processing.
The stashed RPC requests will not be responded to until they are processed
so as to exert back pressure to the senders. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Builds {exhaustive/debug, core/release, asan} passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/exec/kudu-util.h
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/data-stream-sender.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M cmake_modules/FindProtobuf.cmake
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
33 files changed, 3,154 insertions(+), 184 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/12
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 12
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 3:

(17 comments)

Some initial comments.

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/common/status.cc
File be/src/common/status.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/common/status.cc@246
PS3, Line 246:       }
this is the same as FromThrift() effectively, right? Can we make the two look the same to make that obvious?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/common/status.cc@262
PS3, Line 262: 
same comment. let's make this and ToThrift look the same so it's obvious they do the same things.
nit: also, could we order the functions consistently? We currently have ToThrift, FromThrift, FromProto, ToProto, and that ordering just makes it slightly slower to read through since it doesn't follow a pattern.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/exec/exchange-node.cc
File be/src/exec/exchange-node.cc:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/exec/exchange-node.cc@109
PS3, Line 109:   RETURN_IF_CANCELLED(state);
why do we do this in some Open() but not all? Should we just do it in ExecNode::Open() and remove the ones in the derived classes?  okay to do separately from this patch.


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/rpc/rpc-mgr.h
File be/src/rpc/rpc-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/rpc/rpc-mgr.h@154
PS3, Line 154:   }
nit: one-liner?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@94
PS3, Line 94: buffer
what buffer? do you mean queue?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@108
PS3, Line 108: During ordinary operation
what does that mean?  Is it saying that during unordinary operation, a sender can have both a TransmitData() and EndDataStream() call in-flight simultaneously?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@140
PS3, Line 140: it will quietly drop its
what are "it" and "its" here? "the sender" and "the RPC's"?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@141
PS3, Line 141: /// it returns.
is that still true now that we have cancellation of RPCs?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@153
PS3, Line 153: fragment
sending fragment?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@166
PS3, Line 166: request
is that talking about the 'request' field below, or something different?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@175
PS3, Line 175:   const TransmitDataRequestPB* request;
what's the relationship between this and proto_batch?


http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@178
PS3, Line 178:   /// responded to.
who owns it?


http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/data_stream_service.proto
File common/protobuf/data_stream_service.proto:

http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/data_stream_service.proto@29
PS3, Line 29:   optional int32 sender_id = 2;
            : 
            :   optional int32 dest_node_id = 3;
what are "IDs" in these cases? let's improve the documentation here. Especially since type is no longer PlanNodeId (and why is that?).


http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/row_batch.proto
File common/protobuf/row_batch.proto:

http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/row_batch.proto@30
PS3, Line 30: int32
in thrift we had TTupleId. Is there a reason we aren't defining those types as well to make the structure clearer?


http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/row_batch.proto@32
PS3, Line 32: tuple_data
what's tuple_data? not a field in this structure...


http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/row_batch.proto@39
PS3, Line 39: Size
size of what?


http://gerrit.cloudera.org:8080/#/c/8023/3/common/protobuf/row_batch.proto@42
PS3, Line 42: (TODO(KRPC): native enum)
do we plan to fix that?



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 3
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Tue, 10 Oct 2017 20:15:20 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has uploaded a new patch set (#4). ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents some issues with thrift RPC in which a thread may be stuck in
an RPC call without being able to check for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, there are 64 service threads. The service
threads are shared across all queries so the RPC handler should avoid
blocking as much as possible. In thrift RPC implementation, we make a thrift
thread handling a TransmitData() RPC to block for extended period of time
when the receiver is not yet created when the call arrives. In KRPC
implementation, we store TransmitData() or EndDataStream() requests
which arrive before the receiver is ready in a per-receiver early sender list
stored in KrpcDataStreamMgr. These RPC calls will be processed and responded to
when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a blocked_sender_ queue to be processed
later. The stashed RPC request will not be responded to until it is processed
so as to exert back pressure to the client. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Build passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/exec/kudu-util.h
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/data-stream-sender.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M be/src/util/network-util.cc
M cmake_modules/FindProtobuf.cmake
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
34 files changed, 2,969 insertions(+), 175 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/4
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 4
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 6:

(37 comments)

Next batch.

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@103
PS6, Line 103: processed
deserialized?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@104
PS6, Line 104: respectively
what is this "respectively" referring to?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@249
PS6, Line 249: proto batch
unclear what that means, maybe stale comment? And this should say something about the row batch being contained in 'request'.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@258
PS6, Line 258: the actual row batch
isn't that part of "memory pointed to by 'request'"? If so and you want to explicitly mention row batch, maybe say "including the serialized row batch"?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@294
PS6, Line 294:   typedef std::unique_ptr<TransmitDataCtx> DeserializeWorkItem;
How about getting rid of this typedef? The code seems easier to understand if the unique_ptr is visible in the fn decls. it's a bit harder than necessary to reasonable about DeserializeWorkItem& and &&, given that this is now directly a unique_ptr.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.h@428
PS6, Line 428: senders
a sender


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.h
File be/src/runtime/krpc-data-stream-recvr.h:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.h@119
PS6, Line 119: 'row_batch'
row batch is deserialized

('row_batch' isn't a variable in this context)


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.h
File be/src/runtime/krpc-data-stream-sender.h:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.h@57
PS6, Line 57: per_channel_buffer_size' is the buffer size in bytes allocated to each channel's
            :   /// accumulated row batch.
still not clear what that means. This isn't really the size of a buffer, is it? How about something like:

... is a soft limit on the buffering, in bytes, into the channel's accumulating row batch before it will be sent.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.h@111
PS6, Line 111: cached protobuf
serialized


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.h@130
PS6, Line 130: cached proto
outbound row


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.h@133
PS6, Line 133: Two OutboundRowBatch reused across RPCs
Maybe say:
The outbound row batches are double-buffered.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@67
PS6, Line 67: can
it looks like there's a third interface now: SerializeAndSendBatch() that takes an Impala rowbatch.

But now that we have the outbound batch on the sender, why not just use that for the RANDOM case and do SerializeBatch() then TransmitData(), so that we can simplify the abstraction?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@81
PS6, Line 81: TearDown
Teardown()


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@82
PS6, Line 82: TearDown
Teardown()


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@126
PS6, Line 126: . 
or if the preceding RPC failed.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@141
PS6, Line 141:   // Flushes any buffered row batches and sends the EOS RPC to close the channel.
Returns error status if...
Also indicate that it blocks until the EOS RPC is complete.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@157
PS6, Line 157: below
delete.

If that's all we need it for, maybe just remember the capacity?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@166
PS6, Line 166:   RuntimeState* runtime_state_ = nullptr;
is that actually used?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@176
PS6, Line 176: current_outbound_batch_
the name of this is confusing because it's so similar to current_batch_idx_, but it means something different (and often the opposite).  How about calling this:

rpc_outbound_batch_ or rpc_in_flight_batch_?

You could even get rid of the rpc_in_flight_ bool and use rpc_in_flight_batch_ != nullptr for that condition, but I don't have a strong preference. However, it would be good to at least group this with rpc_in_flight_ since they both correspond to the in-flight state.  Also, does the lock protect this (in which case it should fall after the lock_ given that comment).


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@178
PS6, Line 178: The two OutboundRowBatch which are re-used across multiple RPCs
The outbound row batches are double-buffered so that we can serialize the next batch while the other is still referenced by the in flight RPC.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@179
PS6, Line 179: When
             :   // one is being used for an in-flight RPC, the execution thread continues to run and
             :   // serializes another row batch into the other entry.
then you can delete that (better to put the explanation for why we have two next to that statement).


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@181
PS6, Line 181: 'current_batch_idx_' is the index
             :   // of the entry being used by the in-flight or last completed RPC.
that looks incorrect. current_batch_idx_ is incremented immediately after we start sending the row batch, so it points to the row batch to serialize into.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@191
PS6, Line 191:   // This is read and written by the main execution thread.
At least add the cross refernce:
See outbound_batches_.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@201
PS6, Line 201:   std::unique_ptr<DataStreamServiceProxy> proxy_;
Add a quick comment for proxy_ and rpc_controller_


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@237
PS6, Line 237: a KRPC call
an RPC


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@244
PS6, Line 244: prepeneded
typo


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@250
PS6, Line 250:  complete. Return error status if
             :   // the preceding RPC fails.
it can return CANCELLED


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@257
PS6, Line 257: The retry may not happen if the callback has been aborted
             :   // internally by KRPC stack 
what does that mean? when can that happen?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@275
PS6, Line 275: ProtoRowBatch
needs updating


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@367
PS6, Line 367: !ShouldTerminate()
in RetryCb(), this case explicitly becomes MarkDone(CANCELLED), but here he will use the failed RPC status. I don't think it matters since we probably don't check the rpc_status_ once ShouldTerminate() is set, but should this be consistent?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@369
PS6, Line 369:     rpc_mgr->messenger()->ScheduleOnReactor(
why do we need to schedule this on the reactor thread (rather than using our own thread pool)? If there's a reason for that, can you add a short comment explaining why?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@382
PS6, Line 382: rpc_status;
explicilty set that = Status::OK() to make it explicit that the first two if-blocks mean to set it to OK.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@409
PS6, Line 409:   req.Clear();
is that needed? seems like a redundant memset


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@437
PS6, Line 437:   req.release_row_batch_header();
comment why we need that: 
'req' owns (and will delete) 'header' otherwise.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@451
PS6, Line 451:   DCHECK(!rpc_in_flight_);
FlushAndSendEos() does this DCHECK before the remote_recvr_closed_. make them consistent since the invariants should be the same.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@552
PS6, Line 552:     auto pred = [this]() -> bool { return !rpc_in_flight_; };
             :     rpc_done_cv_.wait(l, pred);
while (rpc_in_flight_) rpc_done_cv_.wait(l);

seems simpler


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-sender.cc@657
PS6, Line 657:     RETURN_IF_ERROR(current_channel->SerializeAndSendBatch(batch));
mentioned above: rather than a third way to do this, how about just making this case the same as UNPARTITIONED but without the loop.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 6
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Tue, 31 Oct 2017 20:25:17 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Dan Hecht (Code Review)" <ge...@cloudera.org>.
Dan Hecht has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 6:

(8 comments)

Here's my last set of comments for this round.

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.cc@59
PS6, Line 59: boost::bind
I don't have a strong preference either way, but it'd be nice to be consistent between either using bind or [], rather than both...


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-mgr.cc@322
PS6, Line 322: RespondToTimedOutSenders
shouldn't be plural


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@52
PS6, Line 52: overflows the queue
it's not clear what that means just from reading the comment. It'd be nice to briefly explain that this is talking about the soft limit of the number of bytes across all sender queues.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@105
PS6, Line 105:   // number of pending row batch insertion.
             :   int num_pending_enqueue_ = 0;
it's not clear what a "pending insertion" means or why we have this.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@113
PS6, Line 113: RowBatch*
how about using unique_ptr since this owns the row batch (until it's transferred to current_batch_)?


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@127
PS6, Line 127:   queue<unique_ptr<TransmitDataCtx>> blocked_senders_;
given that a single soft limit is imposed across all sender queues, does it make sense that the blocked_senders_ are maintained per sender? Why don't we maintain a single blocked_senders_ list per datastream recvr?

Hmm, I guess we need to know if this sender has a blocked sender in GetBatch(). But given the single limit, it seems wrong that one sender's row batches can bypass another sender once we get into the blocked sender situation. i.e. the flow of batches across senders seems quite different depending on when the limit was reached.


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@231
PS6, Line 231: proto_batch
update


http://gerrit.cloudera.org:8080/#/c/8023/6/be/src/runtime/krpc-data-stream-recvr.cc@322
PS6, Line 322:     payload->rpc_context->RespondSuccess();
doing this in Close() goes against the paradigm that Close() is only about releasing (local) resources. We've been going that way because there might be no where to bubble up a status from Close().  At least RespondSuccess() doesn't return a status, I suppose.  But is there any place sooner we could do this? Does it make sense to do in during Cancel instead?



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 6
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Thu, 02 Nov 2017 00:10:49 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Hello Sailesh Mukil, Mostafa Mokhtar, Dan Hecht, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/8023

to look at the new patch set (#10).

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents some issues with thrift RPC in which a thread may be stuck in
an RPC call without being able to check for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, there are 64 service threads. The service
threads are shared across all queries so the RPC handler should avoid
blocking as much as possible. In thrift RPC implementation, we make a thrift
thread handling a TransmitData() RPC to block for extended period of time
when the receiver is not yet created when the call arrives. In KRPC
implementation, we store TransmitData() or EndDataStream() requests
which arrive before the receiver is ready in a per-receiver early sender list
stored in KrpcDataStreamMgr. These RPC calls will be processed and responded to
when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a blocked_sender_ queue to be processed
later. The stashed RPC request will not be responded to until it is processed
so as to exert back pressure to the client. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Build passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/exec/kudu-util.h
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/data-stream-sender.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M cmake_modules/FindProtobuf.cmake
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
33 files changed, 3,159 insertions(+), 183 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/10
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 10
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Hello Sailesh Mukil, Mostafa Mokhtar, Dan Hecht, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/8023

to look at the new patch set (#8).

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................

IMPALA-4856: Port data stream service to KRPC

This patch implements a new data stream service which utilizes KRPC.
Similar to the thrift RPC implementation, there are 3 major components
to the data stream services: KrpcDataStreamSender serializes and sends
row batches materialized by a fragment instance to a KrpcDataStreamRecvr.
KrpcDataStreamMgr is responsible for routing an incoming row batch to
the appropriate receiver. The data stream service runs on the port
FLAGS_krpc_port which is 29000 by default.

Unlike the implementation with thrift RPC, KRPC provides an asynchronous
interface for invoking remote methods. As a result, KrpcDataStreamSender
doesn't need to create a thread per connection. There is one connection
between two Impalad nodes for each direction (i.e. client and server).
Multiple queries can multi-plex on the same connection for transmitting
row batches between two Impalad nodes. The asynchronous interface also
prevents some issues with thrift RPC in which a thread may be stuck in
an RPC call without being able to check for cancellation. A TransmitData()
call with KRPC is in essence a trio of RpcController, a serialized protobuf
request buffer and a protobuf response buffer. The call is invoked via a
DataStreamService proxy object. The serialized tuple offsets and row batches
are sent via "sidecars" in KRPC to avoid extra copy into the serialized
request buffer.

Each impalad node creates a singleton DataStreamService object at start-up
time. All incoming calls are served by a service thread pool created as part
of DataStreamService. By default, there are 64 service threads. The service
threads are shared across all queries so the RPC handler should avoid
blocking as much as possible. In thrift RPC implementation, we make a thrift
thread handling a TransmitData() RPC to block for extended period of time
when the receiver is not yet created when the call arrives. In KRPC
implementation, we store TransmitData() or EndDataStream() requests
which arrive before the receiver is ready in a per-receiver early sender list
stored in KrpcDataStreamMgr. These RPC calls will be processed and responded to
when the receiver is created or when timeout occurs.

Similarly, there is limited space in the sender queues in KrpcDataStreamRecvr.
If adding a row batch to a queue in KrpcDataStreamRecvr causes the buffer limit
to exceed, the request will be stashed in a blocked_sender_ queue to be processed
later. The stashed RPC request will not be responded to until it is processed
so as to exert back pressure to the client. An alternative would be to reply with
an error and the request / row batches need to be sent again. This may end up
consuming more network bandwidth than the thrift RPC implementation. This change
adopts the behavior of allowing one stashed request per sender.

All rpc requests and responses are serialized using protobuf. The equivalent of
TRowBatch would be ProtoRowBatch which contains a serialized header about the
meta-data of the row batch and two Kudu Slice objects which contain pointers to
the actual data (i.e. tuple offsets and tuple data).

This patch is based on an abandoned patch by Henry Robinson.

TESTING
-------

* Build passed with FLAGS_use_krpc=true.

TO DO
-----

* Port some BE tests to KRPC services.

Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
---
M be/src/common/status.cc
M be/src/common/status.h
M be/src/exec/data-sink.cc
M be/src/exec/exchange-node.cc
M be/src/exec/kudu-util.h
M be/src/rpc/CMakeLists.txt
M be/src/rpc/rpc-mgr.cc
M be/src/rpc/rpc-mgr.h
M be/src/runtime/CMakeLists.txt
M be/src/runtime/data-stream-mgr-base.h
M be/src/runtime/data-stream-mgr.h
M be/src/runtime/data-stream-recvr.h
M be/src/runtime/data-stream-sender.h
M be/src/runtime/exec-env.cc
M be/src/runtime/exec-env.h
M be/src/runtime/krpc-data-stream-mgr.cc
M be/src/runtime/krpc-data-stream-mgr.h
M be/src/runtime/krpc-data-stream-recvr.cc
M be/src/runtime/krpc-data-stream-recvr.h
A be/src/runtime/krpc-data-stream-sender.cc
A be/src/runtime/krpc-data-stream-sender.h
M be/src/runtime/row-batch.cc
M be/src/runtime/row-batch.h
M be/src/service/CMakeLists.txt
A be/src/service/data-stream-service.cc
A be/src/service/data-stream-service.h
M be/src/service/impala-server.cc
M cmake_modules/FindProtobuf.cmake
M common/protobuf/CMakeLists.txt
A common/protobuf/common.proto
A common/protobuf/data_stream_service.proto
A common/protobuf/row_batch.proto
M common/thrift/generate_error_codes.py
33 files changed, 3,138 insertions(+), 183 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/Impala-ASF refs/changes/23/8023/8
-- 
To view, visit http://gerrit.cloudera.org:8080/8023
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 8
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Tim Armstrong (Code Review)" <ge...@cloudera.org>.
Tim Armstrong has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 3:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/3/be/src/runtime/krpc-data-stream-mgr.h@414
PS3, Line 414: Status
> I think that status is not getting checked by the caller. I thought Tim mad
GCC 4.9.2 doesn't support [[nodiscard]] - we need to upgrade GCC to get this in GCC builds. Any clang build (including clang-tidy) will catch this - LMK if it doesn't.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 3
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Reviewer: Tim Armstrong <ta...@cloudera.com>
Gerrit-Comment-Date: Thu, 12 Oct 2017 21:06:02 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 8:

(1 comment)

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/8/be/src/runtime/krpc-data-stream-sender.cc@434
PS8, Line 434:   proxy_->TransmitDataAsync(req, &resp_, &rpc_controller_,
May want to call resp_.Clear() too.



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 8
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Mon, 06 Nov 2017 22:06:34 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Sailesh Mukil (Code Review)" <ge...@cloudera.org>.
Sailesh Mukil has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 1:

(14 comments)

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h
File be/src/runtime/krpc-data-stream-mgr.h:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@110
PS1, Line 110: sent
nit: received


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@110
PS1, Line 110: no TransmitData() RPCs will successfully deliver their
             : /// payload.
Why would there be a TransmitData() RPC if EndDataStream() has already been sent? Doesn't the sender send it only if it knows all its TransmitData() RPCs have been processed?


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@133
PS1, Line 133: /// period expires.
As per Tim's comment above, I would also reference IMPALA-3990 as a TODO here for later fixing.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@159
PS1, Line 159: Consider tracking, on the sender, whether a batch has been successfully sent or
             : ///   not. That's enough state to realise that a receiver has failed (rather than not
             : ///   prepared yet), and the data stream mgr can use that to fail an RPC fast, rather than
             : ///   having the closed-stream list.
It would be nice to have a JIRA for this and reference it here.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@353
PS1, Line 353: waiting_senders
This is a little confusing to follow in the .cc file, since when I see "waiting_senders", I expect it to be a set of some unique identifiers for a Sender ID.

Although this is unique to a specific sender, it would be a little clearer to call this 'waiting_senders_ctxs'.

Let me know what you think.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.h@356
PS1, Line 356: closed_senders
Similarly, we could call this 'closed_senders_ctxs'.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc
File be/src/runtime/krpc-data-stream-mgr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc@168
PS1, Line 168:       num_senders_waiting_->Increment(1);
             :       total_senders_waited_->Increment(1);
             :       RecvrId recvr_id = make_pair(fragment_instance_id, request->dest_node_id());
             :       auto payload =
             :           make_unique<TransmitDataCtx>(proto_batch, context, request, response);
             :       early_senders_map_[recvr_id].waiting_senders.push_back(move(payload));
I'm wondering if it makes sense to add simple inline functions that encapsulate this functionality; for the sake of readability.

Eg: AddEarlyWaitingSender(), AddEarlyClosedSender()


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc@213
PS1, Line 213: If no receiver found, but not in the closed stream cache
nit: If no receiver is found, and the receiver is not in the closed stream cache as well, we still need...


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc@218
PS1, Line 218:       RecvrId recvr_id = make_pair(fragment_instance_id, dest_node_id);
             :       auto payload = make_unique<EndDataStreamCtx>(context, request, response);
             :       early_senders_map_[recvr_id].closed_senders.emplace_back(move(payload));
             :       num_senders_waiting_->Increment(1);
             :       total_senders_waited_->Increment(1);
AddEarlyClosedSender() as per comment above, if you agree.


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-mgr.cc@227
PS1, Line 227:   if (LIKELY(recvr != nullptr)) recvr->RemoveSender(request->sender_id());
             :   Status::OK().ToProto(response->mutable_status());
             :   context->RespondSuccess();
This may need some modification based on the recent commit for IMPALA-5199:
https://github.com/apache/incubator-impala/commit/5119ced50c0e0c4001621c9d4da598c187bdb580


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc
File be/src/runtime/krpc-data-stream-recvr.cc:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@75
PS1, Line 75: ("new data")
I'm having some trouble understanding what this means. Could you please clarify?


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@271
PS1, Line 271: data_arrival_cv_.notify_all();
Shouldn't this notify be done while holding the lock_ ?


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-recvr.cc@285
PS1, Line 285:   while (!blocked_senders_.empty()) {
nit: Add comment: Respond to blocked senders' RPCs


http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-sender.cc
File be/src/runtime/krpc-data-stream-sender.cc:

http://gerrit.cloudera.org:8080/#/c/8023/1/be/src/runtime/krpc-data-stream-sender.cc@208
PS1, Line 208: %
Should we do a bitwise (cur_batch_idx_ + 1) & 1 instead? Or would the compiler take care of that?



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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Mon, 25 Sep 2017 14:49:37 +0000
Gerrit-HasComments: Yes

[Impala-ASF-CR] IMPALA-4856: Port data stream service to KRPC

Posted by "Michael Ho (Code Review)" <ge...@cloudera.org>.
Michael Ho has posted comments on this change. ( http://gerrit.cloudera.org:8080/8023 )

Change subject: IMPALA-4856: Port data stream service to KRPC
......................................................................


Patch Set 13: Code-Review+2

Retested the new PS. Carry+2.


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

Gerrit-Project: Impala-ASF
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: Ic0b8c1e50678da66ab1547d16530f88b323ed8c1
Gerrit-Change-Number: 8023
Gerrit-PatchSet: 13
Gerrit-Owner: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Dan Hecht <dh...@cloudera.com>
Gerrit-Reviewer: Michael Ho <kw...@cloudera.com>
Gerrit-Reviewer: Mostafa Mokhtar <mm...@cloudera.com>
Gerrit-Reviewer: Sailesh Mukil <sa...@cloudera.com>
Gerrit-Comment-Date: Thu, 09 Nov 2017 16:35:40 +0000
Gerrit-HasComments: No