You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2017/03/09 00:00:52 UTC

[1/3] kudu git commit: KUDU-1887: Add RpcContext::DiscardTransfer()

Repository: kudu
Updated Branches:
  refs/heads/master 72895966c -> 937064f91


KUDU-1887: Add RpcContext::DiscardTransfer()

DiscardTransfer() allows RPC handlers to release the memory backing an
inbound call, without being forced to respond to the RPC at the same
time.

Change-Id: I79e83a7b903d77e8309c3cdcfee6dbb85b5db4bb
Reviewed-on: http://gerrit.cloudera.org:8080/6212
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/62553e0b
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/62553e0b
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/62553e0b

Branch: refs/heads/master
Commit: 62553e0b6dcbdc9822680afd35bba07855d1059a
Parents: 7289596
Author: Henry Robinson <he...@cloudera.com>
Authored: Mon Feb 20 08:48:36 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Mar 8 18:37:26 2017 +0000

----------------------------------------------------------------------
 src/kudu/rpc/inbound_call.cc | 5 +++++
 src/kudu/rpc/inbound_call.h  | 7 ++++++-
 src/kudu/rpc/rpc-test-base.h | 2 ++
 src/kudu/rpc/rpc_context.cc  | 4 ++++
 src/kudu/rpc/rpc_context.h   | 8 ++++++++
 5 files changed, 25 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/62553e0b/src/kudu/rpc/inbound_call.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/inbound_call.cc b/src/kudu/rpc/inbound_call.cc
index 03e7da4..f837dbc 100644
--- a/src/kudu/rpc/inbound_call.cc
+++ b/src/kudu/rpc/inbound_call.cc
@@ -300,6 +300,7 @@ vector<uint32_t> InboundCall::GetRequiredFeatures() const {
 }
 
 Status InboundCall::GetInboundSidecar(int idx, Slice* sidecar) const {
+  DCHECK(transfer_) << "Sidecars have been discarded";
   if (idx < 0 || idx >= header_.sidecar_offsets_size()) {
     return Status::InvalidArgument(strings::Substitute(
             "Index $0 does not reference a valid sidecar", idx));
@@ -308,5 +309,9 @@ Status InboundCall::GetInboundSidecar(int idx, Slice* sidecar) const {
   return Status::OK();
 }
 
+void InboundCall::DiscardTransfer() {
+  transfer_.reset();
+}
+
 } // namespace rpc
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/62553e0b/src/kudu/rpc/inbound_call.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/inbound_call.h b/src/kudu/rpc/inbound_call.h
index ea6eade..20b36ff 100644
--- a/src/kudu/rpc/inbound_call.h
+++ b/src/kudu/rpc/inbound_call.h
@@ -78,7 +78,8 @@ class InboundCall {
   Status ParseFrom(gscoped_ptr<InboundTransfer> transfer);
 
   // Return the serialized request parameter protobuf.
-  const Slice &serialized_request() const {
+  const Slice& serialized_request() const {
+    DCHECK(transfer_) << "Transfer discarded before parameter parsing";
     return serialized_request_;
   }
 
@@ -190,6 +191,10 @@ class InboundCall {
   // returns an error.
   Status GetInboundSidecar(int idx, Slice* sidecar) const;
 
+  // Releases the buffer that contains the request + sidecar data. It is an error to
+  // access sidecars or serialized_request() after this method is called.
+  void DiscardTransfer();
+
  private:
   friend class RpczStore;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/62553e0b/src/kudu/rpc/rpc-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc-test-base.h b/src/kudu/rpc/rpc-test-base.h
index 75ef792..ac22628 100644
--- a/src/kudu/rpc/rpc-test-base.h
+++ b/src/kudu/rpc/rpc-test-base.h
@@ -184,6 +184,8 @@ class GenericCalculatorService : public ServiceIf {
     resp.set_size2(sidecar2.size());
     resp.set_data2(reinterpret_cast<const char*>(sidecar2.data()), sidecar2.size());
 
+    // Drop the sidecars etc, just to confirm that it's safe to do so.
+    incoming->DiscardTransfer();
     incoming->RespondSuccess(resp);
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/62553e0b/src/kudu/rpc/rpc_context.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_context.cc b/src/kudu/rpc/rpc_context.cc
index e93e093..06fd8c5 100644
--- a/src/kudu/rpc/rpc_context.cc
+++ b/src/kudu/rpc/rpc_context.cc
@@ -155,6 +155,10 @@ const RemoteUser& RpcContext::remote_user() const {
   return call_->remote_user();
 }
 
+void RpcContext::DiscardTransfer() {
+  call_->DiscardTransfer();
+}
+
 const Sockaddr& RpcContext::remote_address() const {
   return call_->remote_address();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/62553e0b/src/kudu/rpc/rpc_context.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_context.h b/src/kudu/rpc/rpc_context.h
index 12e8907..3bb4e33 100644
--- a/src/kudu/rpc/rpc_context.h
+++ b/src/kudu/rpc/rpc_context.h
@@ -162,6 +162,14 @@ class RpcContext {
   // Return the identity of remote user who made this call.
   const RemoteUser& remote_user() const;
 
+  // Discards the memory associated with the inbound call's payload. All previously
+  // obtained sidecar slices will be invalidated by this call. It is an error to call
+  // GetInboundSidecar() after this method. request_pb() remains valid.
+  // This is useful in the case where the server wishes to delay responding to an RPC
+  // (perhaps to control the rate of RPC requests), but knows that the RPC payload itself
+  // won't be processed any further.
+  void DiscardTransfer();
+
   // Return the remote IP address and port which sent the current RPC call.
   const Sockaddr& remote_address() const;
 


[3/3] kudu git commit: Fix Webserver option 'password_file' to pass correct Squeasel option 'global_auth_file' for enabling HTTP authorization.

Posted by da...@apache.org.
Fix Webserver option 'password_file' to pass correct Squeasel option
'global_auth_file' for enabling HTTP authorization.

Also add test case for web UI .htpasswd support.

Change-Id: I2d30f450abfb3d0addc0eef39bcf78c87e4298c5
Reviewed-on: http://gerrit.cloudera.org:8080/6300
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/937064f9
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/937064f9
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/937064f9

Branch: refs/heads/master
Commit: 937064f9187e07d2d1880d61cf67792eefe9a82d
Parents: 5a24fe6
Author: hahao <ha...@cloudera.com>
Authored: Tue Mar 7 13:11:37 2017 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Wed Mar 8 23:37:00 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/CMakeLists.txt    |  2 +-
 src/kudu/security/test/test_pass.cc | 40 ++++++++++++++++++++++++++++++++
 src/kudu/security/test/test_pass.h  | 33 ++++++++++++++++++++++++++
 src/kudu/server/webserver-test.cc   | 21 +++++++++++++++++
 src/kudu/server/webserver.cc        |  2 +-
 5 files changed, 96 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/937064f9/src/kudu/security/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/security/CMakeLists.txt b/src/kudu/security/CMakeLists.txt
index c24deef..0dc7d0f 100644
--- a/src/kudu/security/CMakeLists.txt
+++ b/src/kudu/security/CMakeLists.txt
@@ -93,7 +93,7 @@ if (NOT NO_TESTS)
   set(SECURITY_TEST_SRCS
     security-test-util.cc
     test/mini_kdc.cc
-    test/test_certs.cc)
+    test/test_certs.cc test/test_pass.cc)
 
   add_library(security-test ${SECURITY_TEST_SRCS})
   target_link_libraries(security-test

http://git-wip-us.apache.org/repos/asf/kudu/blob/937064f9/src/kudu/security/test/test_pass.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/test_pass.cc b/src/kudu/security/test/test_pass.cc
new file mode 100644
index 0000000..9a0ab46
--- /dev/null
+++ b/src/kudu/security/test/test_pass.cc
@@ -0,0 +1,40 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/security/test/test_pass.h"
+
+#include "kudu/util/env.h"
+#include "kudu/util/path_util.h"
+
+using std::string;
+
+namespace kudu {
+namespace security {
+
+Status CreateTestHTPasswd(const string& dir,
+                          string* passwd_file) {
+
+  // In the format of user:realm:digest. Digest is generated bases on
+  // password 'test'.
+  const char *kHTPasswd = "test:0.0.0.0:e4c02fbc8e89377a942ffc6b1bc3a566";
+  *passwd_file = JoinPathSegments(dir, "test.passwd");
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kHTPasswd, *passwd_file));
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/937064f9/src/kudu/security/test/test_pass.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/test_pass.h b/src/kudu/security/test/test_pass.h
new file mode 100644
index 0000000..c0974d0
--- /dev/null
+++ b/src/kudu/security/test/test_pass.h
@@ -0,0 +1,33 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <string>
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace security {
+
+// Creates .htpasswd for HTTP basic authentication in the format
+// of 'user:realm:digest', returning the path in '*passwd_file'.
+Status CreateTestHTPasswd(const std::string &dir,
+                          std::string *passwd_file);
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/937064f9/src/kudu/server/webserver-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/webserver-test.cc b/src/kudu/server/webserver-test.cc
index 82f68e2..6142319 100644
--- a/src/kudu/server/webserver-test.cc
+++ b/src/kudu/server/webserver-test.cc
@@ -24,6 +24,7 @@
 #include "kudu/gutil/strings/util.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/security/test/test_certs.h"
+#include "kudu/security/test/test_pass.h"
 #include "kudu/server/default-path-handlers.h"
 #include "kudu/server/webserver.h"
 #include "kudu/util/curl_util.h"
@@ -50,6 +51,11 @@ void SetSslOptions(WebserverOptions* opts) {
                                         &password));
   opts->private_key_password_cmd = strings::Substitute("echo $0", password);
 }
+
+void SetHTPasswdOptions(WebserverOptions* opts) {
+  CHECK_OK(security::CreateTestHTPasswd(GetTestDataDirectory(),
+                                        &opts->password_file));
+}
 } // anonymous namespace
 
 class WebserverTest : public KuduTest {
@@ -66,6 +72,7 @@ class WebserverTest : public KuduTest {
     opts.port = 0;
     opts.doc_root = static_dir_;
     if (use_ssl()) SetSslOptions(&opts);
+    if (use_htpasswd()) SetHTPasswdOptions(&opts);
     server_.reset(new Webserver(opts));
 
     AddDefaultPathHandlers(server_.get());
@@ -80,6 +87,7 @@ class WebserverTest : public KuduTest {
  protected:
   // Overridden by subclasses.
   virtual bool use_ssl() const { return false; }
+  virtual bool use_htpasswd() const { return false; }
 
   EasyCurl curl_;
   faststring buf_;
@@ -94,6 +102,19 @@ class SslWebserverTest : public WebserverTest {
   bool use_ssl() const override { return true; }
 };
 
+class PasswdWebserverTest : public WebserverTest {
+ protected:
+  bool use_htpasswd() const override { return true; }
+};
+
+// Send a HTTP request with no username and password. It should reject
+// the request as the .htpasswd is presented to webserver.
+TEST_F(PasswdWebserverTest, TestPasswd) {
+  Status status = curl_.FetchURL(strings::Substitute("http://$0/", addr_.ToString()),
+                                 &buf_);
+  ASSERT_EQ("Remote error: HTTP 401", status.ToString());
+}
+
 TEST_F(WebserverTest, TestIndexPage) {
   curl_.set_return_headers(true);
   ASSERT_OK(curl_.FetchURL(strings::Substitute("http://$0/", addr_.ToString()),

http://git-wip-us.apache.org/repos/asf/kudu/blob/937064f9/src/kudu/server/webserver.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/webserver.cc b/src/kudu/server/webserver.cc
index 6d493b5..743eee8 100644
--- a/src/kudu/server/webserver.cc
+++ b/src/kudu/server/webserver.cc
@@ -193,7 +193,7 @@ Status Webserver::Start() {
       return Status::InvalidArgument(ss.str());
     }
     LOG(INFO) << "Webserver: Password file is " << opts_.password_file;
-    options.push_back("global_passwords_file");
+    options.push_back("global_auth_file");
     options.push_back(opts_.password_file);
   }
 


[2/3] kudu git commit: [sys_catalog] no copying into intermediate KuduPartialRow

Posted by da...@apache.org.
[sys_catalog] no copying into intermediate KuduPartialRow

Use KuduPartialRow::SetStringNoCopy() instead of
KuduPartialRow::SetString() while setting fields for the system catalog
table.  Copying the data into the intermediate KuduPartialRow object is
not necessary: the intermediate object is not used once the data is
encoded into the result WriteRequestPB via RowOperationsPBEncoder.

Change-Id: I0dbcb822d8f85e1101846409e90e2e5af0afc4f3
Reviewed-on: http://gerrit.cloudera.org:8080/6315
Tested-by: Alexey Serbin <as...@cloudera.com>
Reviewed-by: David Ribeiro Alves <dr...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/5a24fe67
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/5a24fe67
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/5a24fe67

Branch: refs/heads/master
Commit: 5a24fe6742611c71fe272b37a1fd6afcc089c998
Parents: 62553e0
Author: Alexey Serbin <as...@cloudera.com>
Authored: Wed Mar 8 08:09:40 2017 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Mar 8 23:16:15 2017 +0000

----------------------------------------------------------------------
 src/kudu/master/sys_catalog.cc | 18 ++++++++++--------
 1 file changed, 10 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/5a24fe67/src/kudu/master/sys_catalog.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/sys_catalog.cc b/src/kudu/master/sys_catalog.cc
index 1d698ff..2d54fcc 100644
--- a/src/kudu/master/sys_catalog.cc
+++ b/src/kudu/master/sys_catalog.cc
@@ -590,13 +590,12 @@ Status SysCatalogTable::AddCertAuthorityEntry(
 
   KuduPartialRow row(&schema_);
   CHECK_OK(row.SetInt8(kSysCatalogTableColType, CERT_AUTHORITY_INFO));
-  CHECK_OK(row.SetString(kSysCatalogTableColId, kSysCertAuthorityEntryId));
-  CHECK_OK(row.SetString(kSysCatalogTableColMetadata, metadata_buf));
+  CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColId, kSysCertAuthorityEntryId));
+  CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColMetadata, metadata_buf));
   RowOperationsPBEncoder enc(req.mutable_row_operations());
   enc.Add(RowOperationsPB::INSERT, row);
-  RETURN_NOT_OK(SyncWrite(&req, &resp));
 
-  return Status::OK();
+  return SyncWrite(&req, &resp);
 }
 
 Status SysCatalogTable::AddTskEntry(const SysTskEntryPB& entry) {
@@ -613,11 +612,14 @@ Status SysCatalogTable::AddTskEntry(const SysTskEntryPB& entry) {
   faststring metadata_buf;
   pb_util::SerializeToString(entry, &metadata_buf);
 
+  // This is crucial to keep entry_id alive until its put into the
+  // WriteRequestPB object by RowOperationsPBEncoder.
+  const string entry_id = TskSeqNumberToEntryId(entry.tsk().key_seq_num());
+
   KuduPartialRow row(&schema_);
   CHECK_OK(row.SetInt8(kSysCatalogTableColType, TSK_ENTRY));
-  CHECK_OK(row.SetString(kSysCatalogTableColId,
-                         TskSeqNumberToEntryId(entry.tsk().key_seq_num())));
-  CHECK_OK(row.SetString(kSysCatalogTableColMetadata, metadata_buf));
+  CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColId, entry_id));
+  CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColMetadata, metadata_buf));
   RowOperationsPBEncoder enc(req.mutable_row_operations());
   enc.Add(RowOperationsPB::INSERT, row);
 
@@ -633,7 +635,7 @@ Status SysCatalogTable::RemoveTskEntries(const set<string>& entry_ids) {
   for (const auto& id : entry_ids) {
     KuduPartialRow row(&schema_);
     CHECK_OK(row.SetInt8(kSysCatalogTableColType, TSK_ENTRY));
-    CHECK_OK(row.SetString(kSysCatalogTableColId, id));
+    CHECK_OK(row.SetStringNoCopy(kSysCatalogTableColId, id));
     RowOperationsPBEncoder enc(req.mutable_row_operations());
     enc.Add(RowOperationsPB::DELETE, row);
   }