[kudu] branch master updated (76d540c13 -> 18d40679a)

2022-08-03 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


from 76d540c13 [tool] Add gflag to control the display of hash info when 
show partition info.
 new 8df970f7a KUDU-2671 fix updating partition end keys for unbounded 
ranges
 new 18d40679a KUDU-2671 range-specific hash schemas in 'kudu table create' 
CLI

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../org/apache/kudu/client/TestAlterTable.java |  10 +-
 src/kudu/client/flex_partitioning_client-test.cc   | 184 -
 src/kudu/common/partition-test.cc  |  32 ++--
 src/kudu/common/partition.cc   | 118 -
 src/kudu/tools/create-table-tool-test.cc   | 130 +--
 src/kudu/tools/tool.proto  |  40 +++--
 src/kudu/tools/tool_action_table.cc|  64 ++-
 7 files changed, 477 insertions(+), 101 deletions(-)



[kudu] 01/02: KUDU-2671 fix updating partition end keys for unbounded ranges

2022-08-03 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 8df970f7a6520bb0dc0f9cc89ad7f62ab349e84d
Author: Alexey Serbin 
AuthorDate: Fri Jul 29 19:57:01 2022 -0700

KUDU-2671 fix updating partition end keys for unbounded ranges

This patch fixes an issue with updating the hash bucket components for
partitions' end keys in case of unbounded ranges.  I also added new test
scenarios that allowed to spot the issue.  The newly added scenarios
would fail if not including the fix.  In addition, I updated a few other
test scenarios to match the current logic of updating the hash partition
component of partition end key.

The previous implementation of updating the hash components tried to
avoid holes in the partition keyspace by carrying over iotas to next
hash dimension index, but with the introduction of range-specific hash
schemas it's no longer possible to do so while keeping the result
key ranges disjoint.

For example, consider the following partitioning:
  [-inf, 0) x 3 buckets x 3 buckets; [0, +inf) x 2 buckets x 2 buckets

The original set of ranges looks like the following:
  [ (0, 0,  ""), (0, 0, "0") )
  [ (0, 1,  ""), (0, 1, "0") )
  [ (0, 2,  ""), (0, 2, "0") )
  [ (1, 0,  ""), (1, 0, "0") )
  [ (1, 1,  ""), (1, 1, "0") )
  [ (1, 2,  ""), (1, 2, "0") )
  [ (2, 0,  ""), (2, 0, "0") )
  [ (2, 1,  ""), (2, 1, "0") )
  [ (2, 2,  ""), (2, 2, "0") )

  [ (0, 0, "0"), (0, 0,  "") )
  [ (0, 1, "0"), (0, 1,  "") )
  [ (1, 0, "0"), (1, 0,  "") )
  [ (1, 1, "0"), (1, 1,  "") )

The previous implementation would transform the key range
[ (0, 1, "0"), (0, 1,  "") ) into [ (0, 1, "0"), (1, 0,  "") ), but
that would put the key range [ (0, 2,  ""), (0, 2, "0") ) inside the
transformed one. That would mess up the interval logic of the partition
pruning and the client's metacache.

As it turns out, the continuous keyspace was just a nice thing to have
since the partition pruning and the client metacache work fine with
sparse keyspaces as well.

Change-Id: I3775f914a3b7cdc294a26911663c2d848f4e491b
Reviewed-on: http://gerrit.cloudera.org:8080/18808
Tested-by: Kudu Jenkins
Reviewed-by: Mahesh Reddy 
Reviewed-by: Abhishek Chennaka 
Reviewed-by: Attila Bukor 
---
 .../org/apache/kudu/client/TestAlterTable.java |  10 +-
 src/kudu/client/flex_partitioning_client-test.cc   | 184 -
 src/kudu/common/partition-test.cc  |  32 ++--
 src/kudu/common/partition.cc   | 118 -
 4 files changed, 279 insertions(+), 65 deletions(-)

diff --git 
a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java 
b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
index f0270b446..c10b6402d 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
@@ -253,12 +253,16 @@ public class TestAlterTable {
 
 KuduScanner scanner = client.newScannerBuilder(table)
 .setProjectedColumnNames(Lists.newArrayList("c0Key", 
"c1")).build();
+int rowCount = 0;
 while (scanner.hasMoreRows()) {
   RowResultIterator it = scanner.nextRows();
-  assertTrue(it.hasNext());
-  RowResult rr = it.next();
-  assertEquals(rr.getInt(0), rr.getInt(1));
+  while (it.hasNext()) {
+RowResult rr = it.next();
+assertEquals(rr.getInt(0), rr.getInt(1));
+++rowCount;
+  }
 }
+assertEquals(101, rowCount);
   }
 
   @Test
diff --git a/src/kudu/client/flex_partitioning_client-test.cc 
b/src/kudu/client/flex_partitioning_client-test.cc
index 6b42eff1b..66d504a83 100644
--- a/src/kudu/client/flex_partitioning_client-test.cc
+++ b/src/kudu/client/flex_partitioning_client-test.cc
@@ -27,14 +27,15 @@
 #include 
 #include 
 
-#include "kudu/client/client.h"
 #include "kudu/client/client-test-util.h"
+#include "kudu/client/client.h"
 #include "kudu/client/scan_batch.h"
 #include "kudu/client/scan_predicate.h"
 #include "kudu/client/schema.h"
 #include "kudu/client/value.h"
 #include "kudu/client/write_op.h"
 #include "kudu/common/partial_row.h"
+#include "kudu/common/partition.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/ref_co

[kudu] branch master updated: [tool] Add gflag to control the display of hash info when show partition info.

2022-08-02 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 76d540c13 [tool] Add gflag to control the display of hash info when 
show partition info.
76d540c13 is described below

commit 76d540c137c991127865d5d2455d113677a9afe9
Author: kedeng 
AuthorDate: Fri Jul 29 11:52:54 2022 +0800

[tool] Add gflag to control the display of hash info when show partition 
info.

The command with 'show_hash_partition_info' looks like:
`kudu table list --list_tablets --show_tablet_partition_info 
--show_hash_partition_info   
[-negotiation_timeout_ms=] [-timeout_ms=]`

The output of the command with 'show_hash_partition_info' looks like:
`
TestTableListPartition
  T f7537632388b46a394b818979a17920c : HASH (key_hash0) PARTITION 0, HASH 
(key_hash1, key_hash2) PARTITION 0, RANGE (key_range) PARTITION 0 <= VALUES < 1
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T 4f1a71d9907b49a1aff479fbafc65a42 : HASH (key_hash0) PARTITION 0, HASH 
(key_hash1, key_hash2) PARTITION 0, RANGE (key_range) PARTITION 2 <= VALUES < 3
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T b11db1b8fc304cc19a0211df45818efc : HASH (key_hash0) PARTITION 0, HASH 
(key_hash1, key_hash2) PARTITION 1, RANGE (key_range) PARTITION 0 <= VALUES < 1
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T db401470d3374d9bba75f56677c1d1c2 : HASH (key_hash0) PARTITION 0, HASH 
(key_hash1, key_hash2) PARTITION 1, RANGE (key_range) PARTITION 2 <= VALUES < 3
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T ead7e1bbc3cd465cb755118e769b8c72 : HASH (key_hash0) PARTITION 0, HASH 
(key_hash1, key_hash2) PARTITION 2, RANGE (key_range) PARTITION 0 <= VALUES < 1
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T 19ffc549b26f49928ecb552224860bf2 : HASH (key_hash0) PARTITION 0, HASH 
(key_hash1, key_hash2) PARTITION 2, RANGE (key_range) PARTITION 2 <= VALUES < 3
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T 38c95bd395404dbda094826f5054aced : HASH (key_hash0) PARTITION 1, HASH 
(key_hash1, key_hash2) PARTITION 0, RANGE (key_range) PARTITION 0 <= VALUES < 1
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T 8c257d4c5ebc4bffb5c27beeef1e9355 : HASH (key_hash0) PARTITION 1, HASH 
(key_hash1, key_hash2) PARTITION 0, RANGE (key_range) PARTITION 2 <= VALUES < 3
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T 6c6d1f1e4df8456f9853beaab6bd7bee : HASH (key_hash0) PARTITION 1, HASH 
(key_hash1, key_hash2) PARTITION 1, RANGE (key_range) PARTITION 0 <= VALUES < 1
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T 05f27b37443c4b7e843e1d7bba2fb5ee : HASH (key_hash0) PARTITION 1, HASH 
(key_hash1, key_hash2) PARTITION 1, RANGE (key_range) PARTITION 2 <= VALUES < 3
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T 48ece2705bc14819b1a38aa489f39e50 : HASH (key_hash0) PARTITION 1, HASH 
(key_hash1, key_hash2) PARTITION 2, RANGE (key_range) PARTITION 0 <= VALUES < 1
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T 768cea86cf8543258313e242afa8f057 : HASH (key_hash0) PARTITION 1, HASH 
(key_hash1, key_hash2) PARTITION 2, RANGE (key_range) PARTITION 2 <= VALUES < 3
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907
`

The command without 'show_hash_partition_info' looks like:
`kudu table list --list_tablets --show_tablet_partition_info 
  [-negotiation_timeout_ms=] 
[-timeout_ms=]`

The output of the command without 'show_hash_partition_info' looks like:
`
TestTableListPartition
  T f7537632388b46a394b818979a17920c : RANGE (key_range) PARTITION 0 <= 
VALUES < 1
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T 4f1a71d9907b49a1aff479fbafc65a42 : RANGE (key_range) PARTITION 2 <= 
VALUES < 3
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T b11db1b8fc304cc19a0211df45818efc : RANGE (key_range) PARTITION 0 <= 
VALUES < 1
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T db401470d3374d9bba75f56677c1d1c2 : RANGE (key_range) PARTITION 2 <= 
VALUES < 3
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T ead7e1bbc3cd465cb755118e769b8c72 : RANGE (key_range) PARTITION 0 <= 
VALUES < 1
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T 19ffc549b26f49928ecb552224860bf2 : RANGE (key_range) PARTITION 2 <= 
VALUES < 3
L 883ba129547f4e9a90134fac739334f1 127.18.13.1:34907

  T 38c95bd395404dbda094826f5054aced : RANGE (key

[kudu] branch master updated: [tools] update on 'kudu table create' command

2022-07-29 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 7ea5fe5ea [tools] update on 'kudu table create' command
7ea5fe5ea is described below

commit 7ea5fe5eae2e03fe2dc88e69a0dfe82d3f2d7088
Author: Alexey Serbin 
AuthorDate: Thu Jul 28 19:02:11 2022 -0700

[tools] update on 'kudu table create' command

This patch updates the code related to the kudu CLI command
'table create':
  * added 'owner' field to CreateTablePB/JSON
  * added 'comment' field to CreateTablePB/JSON
  * updated corresponding comments in the tool.proto file
  * fixed the example of the JSON object for CreateTablePB
  * made the example of JSON object for CreateTablePB redable
  * re-ordered string constants in tool_action_table.cc
  * other minor updates

Change-Id: Ib2855ee07e0dae4df669e6dd1d5ad4281d3b906b
Reviewed-on: http://gerrit.cloudera.org:8080/18796
Tested-by: Kudu Jenkins
Reviewed-by: Abhishek Chennaka 
Reviewed-by: Attila Bukor 
---
 src/kudu/tools/tool.proto   |  23 ---
 src/kudu/tools/tool_action_table.cc | 124 
 2 files changed, 98 insertions(+), 49 deletions(-)

diff --git a/src/kudu/tools/tool.proto b/src/kudu/tools/tool.proto
index b73151376..dc735e137 100644
--- a/src/kudu/tools/tool.proto
+++ b/src/kudu/tools/tool.proto
@@ -426,9 +426,9 @@ message PartitionPB {
 // Column names of columns included in the range. All columns must be
 // a component of the primary key.
 repeated string columns = 1;
-// Range bound.
+// Range bounds.
 repeated RangeBoundPB range_bounds = 2;
-// Range split.
+// Range splits.
 repeated SplitValuePB range_splits = 3;
   }
 
@@ -467,17 +467,22 @@ message SchemaPB {
 // is converted to the PB. Used for creating a new table by kudu tool.
 message CreateTablePB {
   optional string table_name = 1;
-  // Representation of a table's schema, include columns's message and
-  // primary keys.
+  // Representation of a table's schema.
   optional SchemaPB schema = 2;
-  // The table partition message, include hash partition and range partition.
+  // Information on the table partitioning.
   optional PartitionPB partition = 3;
-  //Number of tablet replica
+  // Number of replicas for table's tablets.
   optional int32 num_replicas = 4;
   // The table's extra configuration properties.
   optional ExtraConfigPB extra_configs = 5;
-  // The dimension label for tablets that were created during table creation. 
Used for
-  // dimension-specific placement of tablet replicas corresponding to the 
partitions of
-  // the newly created table.
+  // The dimension label for tablets that were created during table creation.
+  // Used for dimension-specific placement of tablet replicas corresponding
+  // to the partitions of the newly created table.
   optional string dimension_label = 6;
+  // The owner for the newly created table. If not specified, the owner is
+  // automatically set to the effective OS user name that the kudu CLI tool is
+  // run with.
+  optional string owner = 7;
+  // Table's comment.
+  optional string comment = 8;
 }
diff --git a/src/kudu/tools/tool_action_table.cc 
b/src/kudu/tools/tool_action_table.cc
index 2ebc2f9b7..ad4dd3053 100644
--- a/src/kudu/tools/tool_action_table.cc
+++ b/src/kudu/tools/tool_action_table.cc
@@ -273,23 +273,71 @@ class TableAlter {
 
 namespace {
 
-const char* const kNewTableNameArg = "new_table_name";
-const char* const kColumnNameArg = "column_name";
-const char* const kNewColumnNameArg = "new_column_name";
-const char* const kKeyArg = "primary_key";
-const char* const kConfigNameArg = "config_name";
-const char* const kConfigValueArg = "config_value";
-const char* const kErrorMsgArg = "unable to parse value $0 for column $1 of 
type $2";
-const char* const kTableRangeLowerBoundArg = "table_range_lower_bound";
-const char* const kTableRangeUpperBoundArg = "table_range_upper_bound";
-const char* const kDefaultValueArg = "default_value";
-const char* const kCompressionTypeArg = "compression_type";
-const char* const kEncodingTypeArg = "encoding_type";
-const char* const kBlockSizeArg = "block_size";
-const char* const kColumnCommentArg = "column_comment";
-const char* const kCreateTableJSONArg = "create_table_json";
-const char* const kReplicationFactorArg = "replication_factor";
-const char* const kDataTypeArg = "data_type";
+constexpr const char* const kBlockSizeArg = "block_size";
+constexpr const char* const kColumnCommentArg = "column_comment";
+constexpr const char* const kColumnNameArg = "column_name";
+constexpr 

[kudu] branch master updated: [tools] KUDU-2671 Update the kudu table describe tool

2022-07-28 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 9d571f954 [tools] KUDU-2671 Update the kudu table describe tool
9d571f954 is described below

commit 9d571f954302a52249435726cc67dd0b8fd582f7
Author: Abhishek Chennaka 
AuthorDate: Wed Jul 27 18:05:31 2022 -0400

[tools] KUDU-2671 Update the kudu table describe tool

This patch updates the kudu table describe tool to output ranges
which have a custom hash schema with the corresponding hash schema.
If a partition has multiple custom hash dimensions they are
separated by a space.

Change-Id: I9fa2eb965051a5f63d1c482e6fd43ff654ec6364
Reviewed-on: http://gerrit.cloudera.org:8080/18794
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin 
---
 src/kudu/tools/kudu-admin-test.cc   | 58 +
 src/kudu/tools/tool_action_table.cc |  6 ++--
 2 files changed, 61 insertions(+), 3 deletions(-)

diff --git a/src/kudu/tools/kudu-admin-test.cc 
b/src/kudu/tools/kudu-admin-test.cc
index 45e66b292..8602b74f4 100644
--- a/src/kudu/tools/kudu-admin-test.cc
+++ b/src/kudu/tools/kudu-admin-test.cc
@@ -2086,6 +2086,64 @@ TEST_F(AdminCliTest, TestDescribeTableNoOwner) {
   ASSERT_STR_CONTAINS(stdout, "OWNER \n");
 }
 
+TEST_F(AdminCliTest, TestDescribeTableCustomHashSchema) {
+  NO_FATALS(BuildAndStart({}, {}, {}, /*create_table*/false));
+  KuduSchema schema;
+
+  // Build the schema
+  {
+KuduSchemaBuilder builder;
+builder.AddColumn("key_range")->Type(KuduColumnSchema::INT32)->NotNull();
+builder.AddColumn("key_hash0")->Type(KuduColumnSchema::INT32)->NotNull();
+builder.AddColumn("key_hash1")->Type(KuduColumnSchema::INT32)->NotNull();
+builder.SetPrimaryKey({"key_range", "key_hash0", "key_hash1"});
+ASSERT_OK(builder.Build());
+  }
+
+  constexpr const char* const kTableName = "table_with_custom_hash_schema";
+  unique_ptr table_creator(client_->NewTableCreator());
+  table_creator->table_name(kTableName)
+  .schema()
+  .add_hash_partitions({"key_hash0"}, 2)
+  .set_range_partition_columns({"key_range"})
+  .num_replicas(1);
+
+  // Create a KuduRangePartition with custom hash schema
+  {
+unique_ptr lower(schema.NewRow());
+CHECK_OK(lower->SetInt32("key_range", 0));
+unique_ptr upper(schema.NewRow());
+CHECK_OK(upper->SetInt32("key_range", 100));
+unique_ptr partition(
+new client::KuduRangePartition(lower.release(), upper.release()));
+partition->add_hash_partitions({"key_hash1"}, 3);
+table_creator->add_custom_range_partition(partition.release());
+  }
+
+  // Create a partition with table wide hash schema
+  {
+unique_ptr lower(schema.NewRow());
+CHECK_OK(lower->SetInt32("key_range", 100));
+unique_ptr upper(schema.NewRow());
+CHECK_OK(upper->SetInt32("key_range", 200));
+table_creator->add_range_partition(lower.release(), upper.release());
+  }
+
+  // Create the table and run the tool
+  ASSERT_OK(table_creator->Create());
+  string stdout;
+  ASSERT_OK(RunKuduTool(
+  {
+  "table",
+  "describe",
+  cluster_->master()->bound_rpc_addr().ToString(),
+  kTableName,
+  },
+  ));
+  ASSERT_STR_CONTAINS(stdout, "PARTITION 0 <= VALUES < 100 HASH(key_hash1) 
PARTITIONS 3,\n"
+  "PARTITION 100 <= VALUES < 200");
+}
+
 TEST_F(AdminCliTest, TestListTabletWithPartition) {
   FLAGS_num_tablet_servers = 1;
   FLAGS_num_replicas = 1;
diff --git a/src/kudu/tools/tool_action_table.cc 
b/src/kudu/tools/tool_action_table.cc
index f6f82bbaa..2ebc2f9b7 100644
--- a/src/kudu/tools/tool_action_table.cc
+++ b/src/kudu/tools/tool_action_table.cc
@@ -435,9 +435,9 @@ Status DescribeTable(const RunnerContext& context) {
   continue;
 }
 auto range_partition_str =
-
partition_schema.RangePartitionDebugString(partition.begin().range_key(),
-   partition.end().range_key(),
-   schema_internal);
+
partition_schema.RangeWithCustomHashPartitionDebugString(partition.begin().range_key(),
+ 
partition.end().range_key(),
+ 
schema_internal);
 partition_strs.emplace_back(std::move(range_partition_str));
   }
   cout << partition_schema.DisplayString(schema_internal, partition_strs)



[kudu] branch master updated: [catalog_manager] KUDU-2671 fix bug in CreatePartitionsForRange()

2022-07-28 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new e572e5967 [catalog_manager] KUDU-2671 fix bug in 
CreatePartitionsForRange()
e572e5967 is described below

commit e572e59673f6dade62f96eb38b6417ef018bc7bf
Author: Alexey Serbin 
AuthorDate: Tue Jul 26 16:44:42 2022 -0700

[catalog_manager] KUDU-2671 fix bug in CreatePartitionsForRange()

This patch fixes a bug in PartitionSchema::CreatePartitionsForRange().
The manifestation of the bug was the inability to add an unbounded range
with custom hash schema (e.g. [0, +inf)) by AlterTable due to a conflict
with already existing range (e.g., [-inf, 0)) when if fact there was no
conflict at all.  The root cause was the assumption that PartitionSchema
contained information on the range to be added in its internal map
'hash_schema_idx_by_encoded_range_start_' but that wasn't the case,
so GetHashSchemaForRange() would return the table-wide hash schema for
the new range being added instead of proper range-specific hash schema.
That lead to incorrect updating of range boundaries in
UpdatePartitionBoundaries(), producing wrong results.

This patch also contains a new test scenario that allowed to reproduce
the issue: the new scenario is failing without the fix applied.

Change-Id: I33a2bdea2e71bf4b567664c0166e9fbc07c4b882
Reviewed-on: http://gerrit.cloudera.org:8080/18793
Tested-by: Kudu Jenkins
Reviewed-by: Mahesh Reddy 
Reviewed-by: Abhishek Chennaka 
Reviewed-by: Attila Bukor 
---
 src/kudu/client/flex_partitioning_client-test.cc |  63 ++
 src/kudu/common/partition.cc | 103 ++-
 src/kudu/common/partition.h  |  11 ++-
 3 files changed, 137 insertions(+), 40 deletions(-)

diff --git a/src/kudu/client/flex_partitioning_client-test.cc 
b/src/kudu/client/flex_partitioning_client-test.cc
index 3a89aea24..6b42eff1b 100644
--- a/src/kudu/client/flex_partitioning_client-test.cc
+++ b/src/kudu/client/flex_partitioning_client-test.cc
@@ -1775,6 +1775,69 @@ TEST_F(FlexPartitioningAlterTableTest, 
ReadAndWriteToCustomRangePartition) {
   }
 }
 
+TEST_F(FlexPartitioningAlterTableTest, 
ReadAndWriteToUnboundedCustomRangePartition) {
+  constexpr const char* const kTableName =
+  "ReadAndWriteToUnboundedCustomRangePartition";
+  unique_ptr table_creator(client_->NewTableCreator());
+  unique_ptr lower(schema_.NewRow());
+  ASSERT_OK(lower->SetInt32(kKeyColumn, -100));
+  unique_ptr upper(schema_.NewRow());
+  ASSERT_OK(upper->SetInt32(kKeyColumn, 100));
+  table_creator->table_name(kTableName)
+  .schema(_)
+  .num_replicas(1)
+  .add_hash_partitions({ kKeyColumn }, 2)
+  .set_range_partition_columns({ kKeyColumn })
+  .add_range_partition(lower.release(), upper.release());
+  ASSERT_OK(table_creator->Create());
+
+  unique_ptr 
table_alterer(client_->NewTableAlterer(kTableName));
+  {
+auto p = CreateRangePartitionNoLowerBound(-100);
+ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 3, 1));
+table_alterer->AddRangePartition(p.release());
+  }
+  {
+auto p = CreateRangePartitionNoUpperBound(100);
+ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 4, 2));
+table_alterer->AddRangePartition(p.release());
+  }
+  ASSERT_OK(table_alterer->Alter());
+  NO_FATALS(CheckTabletCount(kTableName, 9));  // 2 + 3 + 4 = 9
+
+  // Make sure it's possible to insert rows into the table for all the existing
+  // the partitions: first check the range of table-wide schema, then check
+  // the ranges with custom hash schemas.
+  ASSERT_OK(InsertTestRows(kTableName, -100, 100));
+  NO_FATALS(CheckTableRowsNum(kTableName, 200));
+  ASSERT_OK(InsertTestRows(kTableName, -200, -100));
+  NO_FATALS(CheckTableRowsNum(kTableName, 300));
+  ASSERT_OK(InsertTestRows(kTableName, 100, 200));
+  NO_FATALS(CheckTableRowsNum(kTableName, 400));
+  ASSERT_OK(InsertTestRows(kTableName, INT32_MIN, INT32_MIN + 100));
+  NO_FATALS(CheckTableRowsNum(kTableName, 500));
+  ASSERT_OK(InsertTestRows(kTableName, INT32_MAX - 100, INT32_MAX));
+  NO_FATALS(CheckTableRowsNum(kTableName, 600));
+
+  NO_FATALS(CheckTableRowsNum(kTableName, kKeyColumn, -300, 300, 400));
+
+  // Drop a partition in the middle and re-scan with various ranges.
+  {
+unique_ptr 
table_alterer_drop(client_->NewTableAlterer(kTableName));
+unique_ptr lower_drop(schema_.NewRow());
+ASSERT_OK(lower_drop->SetInt32(kKeyColumn, -100));
+unique_ptr upper_drop(schema_.NewRow());
+ASSERT_OK(upper_drop->SetInt32(kKeyColumn, 100));
+table_alterer_drop->DropRangePartition(lower_drop.release(), 
upper_drop.release());
+ASSERT_OK(table_alterer_drop->Alter());
+  }
+  N

[kudu] branch master updated: [java] KUDU-2671 fix bug in PartitionSchema::getHashSchemaForRange()

2022-07-27 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 3b91db407 [java] KUDU-2671 fix bug in 
PartitionSchema::getHashSchemaForRange()
3b91db407 is described below

commit 3b91db407d9973f154567e0b280f7fe39899c557
Author: Alexey Serbin 
AuthorDate: Mon Jul 25 16:26:09 2022 -0700

[java] KUDU-2671 fix bug in PartitionSchema::getHashSchemaForRange()

This patch fixes a bug in PartitionSchema::getHashSchemaForRange()
that manifested itself when a right-side unbounded range with custom
hash schema was present in a table.

The patch also contains test scenarios that allowed to see the
manifestation of the bug before the fix; from now they allow to spot
regressions in the corresponding code, if any.  These scenarios cover
both {Create,Alter}Table code paths when adding right-side unbounded
ranges with custom hash schemas.

Change-Id: Ib0ef5bbe4528ce5c1d4c9591a8152cf7ec4af0bb
Reviewed-on: http://gerrit.cloudera.org:8080/18782
Tested-by: Kudu Jenkins
Reviewed-by: Attila Bukor 
---
 .../org/apache/kudu/client/PartitionSchema.java|  11 ++-
 .../org/apache/kudu/client/TestAlterTable.java | 110 +
 .../java/org/apache/kudu/client/TestKuduTable.java |  89 +
 3 files changed, 206 insertions(+), 4 deletions(-)

diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/PartitionSchema.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/PartitionSchema.java
index e62ac8dbf..50851a4ae 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/PartitionSchema.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/PartitionSchema.java
@@ -205,7 +205,8 @@ public class PartitionSchema {
   /**
* Find hash schema for the given encoded range key. Depending on the
* partition schema and the key, it might be either table-wide or a custom
-   * hash schema for a particular range.
+   * hash schema for a particular range. Just as a convention, this method
+   * returns the table-wide hash schema for keys in non-covered ranges.
*
* @return hash bucket schema for the encoded range key
*/
@@ -221,10 +222,12 @@ public class PartitionSchema {
 if (entry == null) {
   return hashBucketSchemas;
 }
-// Check if 'rangeKey' is in the range (null upper boundary means unbounded
-// range partition).
+// Check if 'rangeKey' is in the range.
+// NOTE: the right boundary is exclusive; an empty array for upper boundary
+//   means that the range partition is unbounded.
 final byte[] upper = entry.upper;
-if (upper == null || Bytes.memcmp(rangeKey, upper) < 0) {
+Preconditions.checkNotNull(upper);
+if (upper.length == 0 || Bytes.memcmp(rangeKey, upper) < 0) {
   return entry.hashSchemas;
 }
 return hashBucketSchemas;
diff --git 
a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java 
b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
index 7e722e0de..f0270b446 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
@@ -599,6 +599,116 @@ public class TestAlterTable {
 client.deleteTable(tableName);
   }
 
+  /**
+   * Test altering a table, adding unbounded range partitions
+   * with custom hash schema.
+   */
+  @Test(timeout = 10)
+  public void testAlterAddUnboundedRangeWithCustomHashSchema() throws 
Exception {
+ArrayList columns = new ArrayList<>(2);
+columns.add(new ColumnSchema.ColumnSchemaBuilder("c0", Type.INT32)
+.nullable(false)
+.key(true)
+.build());
+columns.add(new ColumnSchema.ColumnSchemaBuilder("c1", Type.INT32)
+.nullable(false)
+.build());
+final Schema schema = new Schema(columns);
+
+CreateTableOptions createOptions =
+new CreateTableOptions()
+.setRangePartitionColumns(ImmutableList.of("c0"))
+.addHashPartitions(ImmutableList.of("c0"), 2, 0)
+.setNumReplicas(1);
+// Add range partition [-100, 100) with the table-wide hash schema
+// (to be added upon creating the new table below).
+{
+  PartialRow lower = schema.newPartialRow();
+  lower.addInt("c0", -100);
+  PartialRow upper = schema.newPartialRow();
+  upper.addInt("c0", 100);
+  createOptions.addRangePartition(lower, upper);
+}
+// Add unbounded range partition [100, +inf) with custom hash schema.
+{
+  PartialRow lower = schema.newPartialRow();
+  lower.addInt("c0", 100);
+  PartialRow upper = schema.newPartialRow();
+  RangePartitionWithCustomHashSche

[kudu] branch master updated: [tool] Add partitions show with tablet id

2022-07-27 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new a6e8aa2fd [tool] Add partitions show with tablet id
a6e8aa2fd is described below

commit a6e8aa2fd5063abb2cc5c8944c9d19fdb38b8aa0
Author: kedeng 
AuthorDate: Tue Jun 14 15:43:05 2022 +0800

[tool] Add partitions show with tablet id

With this commit help, we can get the corresponding relationship
between partitions and tablet id. Which will help us to observe
the partitions more easily.

The command looks like:
`kudu table list --list_tablets --show_tablet_partition_info  
 [-negotiation_timeout_ms=] [-timeout_ms=]`

The output of the command looks like:
`
TestTableListPartition
  T e72c529ad1a8481390683fd0fe3fb917 : HASH (key_hash0) PARTITION 0, HASH 
(key_hash1, key_hash2) PARTITION 0, RANGE (key_range) PARTITION 0 <= VALUES < 1
L 8afe7243874943058e849fecebb84d98 127.28.231.1:11503

  T 674c732628b74fbab0a03a11f357b432 : HASH (key_hash0) PARTITION 0, HASH 
(key_hash1, key_hash2) PARTITION 0, RANGE (key_range) PARTITION 2 <= VALUES < 3
L 8afe7243874943058e849fecebb84d98 127.28.231.1:11503

  T 1c9a5a6f62d1402aba04dc84f32b6b04 : HASH (key_hash0) PARTITION 0, HASH 
(key_hash1, key_hash2) PARTITION 1, RANGE (key_range) PARTITION 0 <= VALUES < 1
L 8afe7243874943058e849fecebb84d98 127.28.231.1:11503

  T 44046ee1b43a43f4b59dad75f611ac0a : HASH (key_hash0) PARTITION 0, HASH 
(key_hash1, key_hash2) PARTITION 1, RANGE (key_range) PARTITION 2 <= VALUES < 3
L 8afe7243874943058e849fecebb84d98 127.28.231.1:11503

  T 12ca618464074d8a9ae3f9581701bcf0 : HASH (key_hash0) PARTITION 0, HASH 
(key_hash1, key_hash2) PARTITION 2, RANGE (key_range) PARTITION 0 <= VALUES < 1
L 8afe7243874943058e849fecebb84d98 127.28.231.1:11503

  T edc1cbdae0934df9a48866fe65cfbec8 : HASH (key_hash0) PARTITION 0, HASH 
(key_hash1, key_hash2) PARTITION 2, RANGE (key_range) PARTITION 2 <= VALUES < 3
L 8afe7243874943058e849fecebb84d98 127.28.231.1:11503

  T 91ebccd823634878a6e0a8a4e941aa4f : HASH (key_hash0) PARTITION 1, HASH 
(key_hash1, key_hash2) PARTITION 0, RANGE (key_range) PARTITION 0 <= VALUES < 1
L 8afe7243874943058e849fecebb84d98 127.28.231.1:11503

  T 0057ad8aea0d4ad3a626b6417f14bcac : HASH (key_hash0) PARTITION 1, HASH 
(key_hash1, key_hash2) PARTITION 0, RANGE (key_range) PARTITION 2 <= VALUES < 3
L 8afe7243874943058e849fecebb84d98 127.28.231.1:11503

  T de3f9629a97148069aae9e598cb0d3fa : HASH (key_hash0) PARTITION 1, HASH 
(key_hash1, key_hash2) PARTITION 1, RANGE (key_range) PARTITION 0 <= VALUES < 1
L 8afe7243874943058e849fecebb84d98 127.28.231.1:11503

  T d7f9c9103f1943ef9b5131f03e9f8738 : HASH (key_hash0) PARTITION 1, HASH 
(key_hash1, key_hash2) PARTITION 1, RANGE (key_range) PARTITION 2 <= VALUES < 3
L 8afe7243874943058e849fecebb84d98 127.28.231.1:11503

  T 63777094ede14232a5822a3ac50925fd : HASH (key_hash0) PARTITION 1, HASH 
(key_hash1, key_hash2) PARTITION 2, RANGE (key_range) PARTITION 0 <= VALUES < 1
L 8afe7243874943058e849fecebb84d98 127.28.231.1:11503

  T 87f91f80595140fd9c9582a186a3f52b : HASH (key_hash0) PARTITION 1, HASH 
(key_hash1, key_hash2) PARTITION 2, RANGE (key_range) PARTITION 2 <= VALUES < 3
L 8afe7243874943058e849fecebb84d98 127.28.231.1:11503
 `

Change-Id: Ia9c3b62111c7f302ea934324d1636f79731f18f1
Reviewed-on: http://gerrit.cloudera.org:8080/18617
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin 
---
 src/kudu/client/client-internal.cc   |  23 +-
 src/kudu/client/client-internal.h|  12 ++-
 src/kudu/common/partition.h  |   2 -
 src/kudu/integration-tests/cluster_itest_util.cc |  18 
 src/kudu/integration-tests/cluster_itest_util.h  |   7 ++
 src/kudu/master/catalog_manager.cc   |  39 ++---
 src/kudu/master/catalog_manager.h|   7 ++
 src/kudu/master/master.proto |  10 +++
 src/kudu/tools/kudu-admin-test.cc| 100 +++
 src/kudu/tools/tool_action_table.cc  |  50 +++-
 10 files changed, 248 insertions(+), 20 deletions(-)

diff --git a/src/kudu/client/client-internal.cc 
b/src/kudu/client/client-internal.cc
index 5c650599f..9095ae9c2 100644
--- a/src/kudu/client/client-internal.cc
+++ b/src/kudu/client/client-internal.cc
@@ -444,12 +444,15 @@ Status KuduClient::Data::WaitForAlterTableToFinish(
 
 Status KuduClient::Data::ListTablesWithInfo(KuduClient* client,
 vector* tables_info,
-co

[kudu] branch master updated: [c++-client] KUDU-2671 more tests for unbounded ranges (CreateTable)

2022-07-27 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new df51b540b [c++-client] KUDU-2671 more tests for unbounded ranges 
(CreateTable)
df51b540b is described below

commit df51b540bb1f5fb40790389b77a90df589bf1363
Author: Alexey Serbin 
AuthorDate: Fri Jul 22 15:59:37 2022 -0700

[c++-client] KUDU-2671 more tests for unbounded ranges (CreateTable)

This patch adds more tests for Kudu C++ client covering the paths
involving CreateTable RPC for tables with unbounded ranges having custom
hash schemas.

Change-Id: I584f613ddf3341156d02c4e7c9603b71584cfcf4
Reviewed-on: http://gerrit.cloudera.org:8080/18775
Reviewed-by: Mahesh Reddy 
Tested-by: Kudu Jenkins
Reviewed-by: Yingchun Lai 
Reviewed-by: Attila Bukor 
---
 src/kudu/client/flex_partitioning_client-test.cc | 91 
 1 file changed, 91 insertions(+)

diff --git a/src/kudu/client/flex_partitioning_client-test.cc 
b/src/kudu/client/flex_partitioning_client-test.cc
index 1d3909940..3a89aea24 100644
--- a/src/kudu/client/flex_partitioning_client-test.cc
+++ b/src/kudu/client/flex_partitioning_client-test.cc
@@ -220,6 +220,13 @@ class FlexPartitioningTest : public KuduTest {
 return CreateRangePartition(schema_, kKeyColumn, lower_bound, upper_bound);
   }
 
+  RangePartition CreateRangePartitionNoLowerBound(int32_t upper_bound) {
+unique_ptr upper(schema_.NewRow());
+CHECK_OK(upper->SetInt32(kKeyColumn, upper_bound));
+return unique_ptr(
+new KuduRangePartition(schema_.NewRow(), upper.release()));
+  }
+
   RangePartition CreateRangePartitionNoUpperBound(int32_t lower_bound) {
 unique_ptr lower(schema_.NewRow());
 CHECK_OK(lower->SetInt32(kKeyColumn, lower_bound));
@@ -1041,6 +1048,90 @@ TEST_F(FlexPartitioningCreateTableTest, 
DISABLED_NoUpperBoundRangeCustomHashSche
   NO_FATALS(CheckTableRowsNum(kTableName, 555));
 }
 
+// Create a table unbounded ranges with custom hash schemas.
+TEST_F(FlexPartitioningCreateTableTest, UnboundedRangesWithCustomHashSchemas) {
+  constexpr const char* const kTableName =
+  "UnboundedRangesWithCustomHashSchemas";
+
+  unique_ptr table_creator(client_->NewTableCreator());
+  table_creator->table_name(kTableName)
+  .schema(_)
+  .num_replicas(1)
+  .add_hash_partitions({ kKeyColumn }, 2)
+  .set_range_partition_columns({ kKeyColumn });
+
+  // Add a range partition with custom hash sub-partitioning rules:
+  // 3 buckets with hash based on the "key" column with hash seed 1.
+  {
+auto p = CreateRangePartitionNoLowerBound(0);
+ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 3, 1));
+table_creator->add_custom_range_partition(p.release());
+  }
+
+  // Add a range partition with custom hash sub-partitioning rules:
+  // 5 buckets with hash based on the "key" column with hash seed 2.
+  {
+auto p = CreateRangePartitionNoUpperBound(0);
+ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 5, 2));
+table_creator->add_custom_range_partition(p.release());
+  }
+
+  ASSERT_OK(table_creator->Create());
+  NO_FATALS(CheckTabletCount(kTableName, 8));
+
+  ASSERT_OK(InsertTestRows(kTableName, -250, -125));
+  NO_FATALS(CheckTableRowsNum(kTableName, 125));
+  ASSERT_OK(InsertTestRows(kTableName, 125, 250));
+  NO_FATALS(CheckTableRowsNum(kTableName, 250));
+  ASSERT_OK(InsertTestRows(kTableName, -125, 125));
+  NO_FATALS(CheckTableRowsNum(kTableName, 500));
+}
+
+// Similar to the UnboundedRangesWithCustomHashSchemas above, but with
+// additional range having table-wide table schema.
+TEST_F(FlexPartitioningCreateTableTest,
+   TwoUnboundedRangesWithCustomHashSchemasAndOneInBetween) {
+  constexpr const char* const kTableName =
+  "TwoUnboundedRangesWithCustomHashSchemasAndOneInBetween";
+
+  unique_ptr table_creator(client_->NewTableCreator());
+  table_creator->table_name(kTableName)
+  .schema(_)
+  .num_replicas(1)
+  .add_hash_partitions({ kKeyColumn }, 3)
+  .set_range_partition_columns({ kKeyColumn });
+
+  // Add a range partition with the table-wide hash partitioning rules.
+  {
+unique_ptr lower(schema_.NewRow());
+ASSERT_OK(lower->SetInt32(kKeyColumn, -111));
+unique_ptr upper(schema_.NewRow());
+ASSERT_OK(upper->SetInt32(kKeyColumn, 111));
+table_creator->add_range_partition(lower.release(), upper.release());
+  }
+
+  // Add a range partition with custom hash sub-partitioning rules:
+  // 2 buckets with hash based on the "key" column with hash seed 1.
+  {
+auto p = CreateRangePartitionNoLowerBound(-111);
+ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 2, 1));
+table_creator->add_custom_range_partition(p.release());
+  }
+  {
+auto p =

[kudu] branch master updated: [threadpool] Fix scheduler thread a coredump bug

2022-07-26 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 6e48a43c4 [threadpool] Fix scheduler thread a coredump bug
6e48a43c4 is described below

commit 6e48a43c4505b4a4d281f7f732c63f51cec6bc63
Author: shenxingwuying 
AuthorDate: Tue Jun 28 18:13:37 2022 +0800

[threadpool] Fix scheduler thread a coredump bug

Scheduler thread will schedule some tasks.
If a SchedulerTask's theadpool token is shutdown,
Scheduling the task would coredump.

The patch check the token whether is nullptr,
and check the Submit(...) whether is shutdowning.
And add an unit test.

Change-Id: Ie5bc5511a745f3dc12dfe1a6a1813ece41ccc2a8
Reviewed-on: http://gerrit.cloudera.org:8080/18674
Reviewed-by: Yingchun Lai 
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin 
---
 src/kudu/util/threadpool-test.cc | 17 +++--
 src/kudu/util/threadpool.cc  | 32 ++---
 src/kudu/util/threadpool.h   | 52 +---
 3 files changed, 82 insertions(+), 19 deletions(-)

diff --git a/src/kudu/util/threadpool-test.cc b/src/kudu/util/threadpool-test.cc
index 31e57723c..25dc561a9 100644
--- a/src/kudu/util/threadpool-test.cc
+++ b/src/kudu/util/threadpool-test.cc
@@ -190,10 +190,23 @@ TEST_F(ThreadPoolTest, TestSubmitAfterShutdown) {
 
   ASSERT_OK(RebuildPoolWithScheduler(1, 1));
   unique_ptr token = 
pool_->NewToken(ThreadPool::ExecutionMode::SERIAL);
+  ASSERT_OK(token->Schedule(, 200));
+  token->Shutdown();
+  SleepFor(MonoDelta::FromMilliseconds(500));
   pool_->Shutdown();
   ASSERT_TRUE(token->Schedule(, 
1000).IsServiceUnavailable());
 }
 
+TEST_F(ThreadPoolTest, TokenShutdownBeforeSchedulerExecute) {
+  ASSERT_OK(RebuildPoolWithScheduler(1, 1));
+  unique_ptr token = 
pool_->NewToken(ThreadPool::ExecutionMode::SERIAL);
+  ASSERT_OK(token->Schedule(, 200));
+  token->Shutdown();
+  SleepFor(MonoDelta::FromMilliseconds(500));
+  ASSERT_TRUE(token->Schedule(, 
1000).IsServiceUnavailable());
+  pool_->Shutdown();
+}
+
 TEST_F(ThreadPoolTest, TestThreadPoolWithNoMinimum) {
   constexpr int kIdleTimeoutMs = 1;
   ASSERT_OK(RebuildPoolWithBuilder(
@@ -255,7 +268,6 @@ TEST_F(ThreadPoolTest, 
TestThreadPoolWithSchedulerAndNoMinimum) {
   unique_ptr token = 
pool_->NewToken(ThreadPool::ExecutionMode::SERIAL);
   ASSERT_OK(token->Submit([]() { latch.Wait(); }));
   ASSERT_EQ(4, pool_->num_threads());
-
   ASSERT_OK(token->Schedule([]() { latch.Wait(); }, kDelayMs));
   ASSERT_OK(token->Schedule([]() { latch.Wait(); }, 
static_cast(kDelayMs * 1.2)));
   ASSERT_EQ(4, pool_->num_threads());
@@ -1052,8 +1064,9 @@ TEST_P(ThreadPoolTestTokenTypes, TestTokenWaitForAll) {
   const int kNumSubmissions = 20;
   Random r(SeedRandom());
   vector> tokens;
+  tokens.resize(kNumTokens);
   for (int i = 0; i < kNumTokens; i++) {
-tokens.emplace_back(pool_->NewToken(GetParam()));
+tokens[i] = pool_->NewToken(GetParam());
   }
 
   atomic v(0);
diff --git a/src/kudu/util/threadpool.cc b/src/kudu/util/threadpool.cc
index 95c16178c..38e56d423 100644
--- a/src/kudu/util/threadpool.cc
+++ b/src/kudu/util/threadpool.cc
@@ -142,16 +142,32 @@ void SchedulerThread::RunLoop() {
 vector pending_tasks;
 {
   MutexLock auto_lock(mutex_);
-  auto upper_it = tasks_.upper_bound(now);
-  for (auto it = tasks_.begin(); it != upper_it; it++) {
+  auto upper_it = future_tasks_.upper_bound(now);
+  for (auto it = future_tasks_.begin(); it != upper_it; it++) {
 pending_tasks.emplace_back(std::move(it->second));
   }
-  tasks_.erase(tasks_.begin(), upper_it);
+  future_tasks_.erase(future_tasks_.begin(), upper_it);
 }
 
 for (const auto& task : pending_tasks) {
-  ThreadPoolToken* token = task.thread_pool_token_;
-  CHECK_OK(token->Submit(task.f));
+  ThreadPoolToken* token = task.thread_pool_token;
+  while (token != nullptr) {
+Status s = token->Submit(task.f);
+if (s.ok()) {
+  break;
+}
+DCHECK(s.IsServiceUnavailable())
+<< Substitute("threadpool token Submit status: $0", s.ToString());
+
+if (!token->MaySubmitNewTasks()) {
+  // threadpool token is Shutdown, skip the task.
+  break;
+}
+// If developers use ThreadPoolToken::Schedule(...) too frequent, 
blocking queue's
+// capacity will be full, then retry submit the task again.
+VLOG(1) << Substitute("threadpool token Submit status: $0, retry the 
task", s.ToString());
+SleepFor(MonoDelta::FromMilliseconds(1));
+  }
 }
   }
 }
@@ -401,14 +417,14 @@ Status ThreadPool::Init() {
 }
 
 void ThreadPool::Shutdown() {
-  Mut

[kudu] branch master updated: [ranger-kms] KUDU-3385 - part2: mini_ranger init

2022-07-22 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new f9517b181 [ranger-kms] KUDU-3385 - part2: mini_ranger init
f9517b181 is described below

commit f9517b181101bd122b73d9826d80d3b31641542d
Author: Zoltan Chovan 
AuthorDate: Tue May 3 20:47:20 2022 +0200

[ranger-kms] KUDU-3385 - part2: mini_ranger init

For Kudu to be able to provide data at rest encryption, a separate key
management service can be used for encrypting/decrypting keys.

This patch lays the groundwork for integrating RangerKMS as such a
service, enabling us to create mini test clusters with it, thus making
it possible to test the required interactions properly.

Two existing services mini_ranger and mini_pg were modified, so that the
mini_pg instance could be shared between mini_ranger and mini_ranger_kms
and other future services as well.

Change-Id: I11617468245068dd732fb3f2578bb086b2f6024f
Reviewed-on: http://gerrit.cloudera.org:8080/18645
Reviewed-by: Alexey Serbin 
Reviewed-by: Attila Bukor 
Tested-by: Kudu Jenkins
---
 CMakeLists.txt |   1 +
 src/kudu/mini-cluster/CMakeLists.txt   |   1 +
 src/kudu/mini-cluster/external_mini_cluster.cc |  25 +-
 src/kudu/mini-cluster/external_mini_cluster.h  |  25 +-
 src/kudu/postgres/mini_postgres.cc |   3 +-
 src/kudu/postgres/mini_postgres.h  |   8 +
 .../{mini-cluster => ranger-kms}/CMakeLists.txt|  61 +-
 src/kudu/ranger-kms/mini_ranger_kms.cc | 306 +
 src/kudu/ranger-kms/mini_ranger_kms.h  | 164 +
 src/kudu/ranger-kms/mini_ranger_kms_configs.h  | 723 +
 src/kudu/ranger/mini_ranger-test.cc|  10 +-
 src/kudu/ranger/mini_ranger.cc |  20 +-
 src/kudu/ranger/mini_ranger.h  |  31 +-
 src/kudu/ranger/ranger_client-test.cc  |   6 +-
 src/kudu/util/env_util.cc  |  53 +-
 src/kudu/util/env_util.h   |   8 +
 16 files changed, 1364 insertions(+), 81 deletions(-)

diff --git a/CMakeLists.txt b/CMakeLists.txt
index 6ef5877db..4ce900a59 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -1493,6 +1493,7 @@ add_subdirectory(src/kudu/master)
 add_subdirectory(src/kudu/mini-cluster)
 add_subdirectory(src/kudu/postgres)
 add_subdirectory(src/kudu/ranger)
+add_subdirectory(src/kudu/ranger-kms)
 add_subdirectory(src/kudu/rebalance)
 add_subdirectory(src/kudu/rpc)
 add_subdirectory(src/kudu/security)
diff --git a/src/kudu/mini-cluster/CMakeLists.txt 
b/src/kudu/mini-cluster/CMakeLists.txt
index 047c08c05..479e83f44 100644
--- a/src/kudu/mini-cluster/CMakeLists.txt
+++ b/src/kudu/mini-cluster/CMakeLists.txt
@@ -38,6 +38,7 @@ set(MINI_CLUSTER_LIBS
   mini_hms
   mini_kdc
   mini_ranger
+  mini_ranger_kms
   server_base_proto
   tablet_proto
   tserver
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc 
b/src/kudu/mini-cluster/external_mini_cluster.cc
index 91e79dce8..7a765b83b 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -35,6 +35,8 @@
 #include "kudu/client/client.h"
 #include "kudu/client/master_rpc.h"
 #include "kudu/fs/fs.pb.h"
+#include "kudu/postgres/mini_postgres.h"
+#include "kudu/ranger-kms/mini_ranger_kms.h"
 #include "kudu/rpc/rpc_header.pb.h"
 #include "kudu/server/key_provider.h"
 #if !defined(NO_CHRONY)
@@ -131,6 +133,7 @@ ExternalMiniClusterOptions::ExternalMiniClusterOptions()
   principal("kudu"),
   hms_mode(HmsMode::NONE),
   enable_ranger(false),
+  enable_ranger_kms(false),
   enable_encryption(FLAGS_encrypt_data_at_rest),
   logtostderr(true),
   start_process_timeout(MonoDelta::FromSeconds(70)),
@@ -323,9 +326,12 @@ Status ExternalMiniCluster::Start() {
   }
 #endif // #if !defined(NO_CHRONY) ...
 
-  if (opts_.enable_ranger) {
+  if (opts_.enable_ranger || opts_.enable_ranger_kms) {
+if (!postgres_ || !postgres_->IsStarted()) {
+  postgres_.reset(new postgres::MiniPostgres(cluster_root(), 
GetBindIpForExternalServer(0)));
+}
 string host = GetBindIpForExternalServer(0);
-ranger_.reset(new ranger::MiniRanger(cluster_root(), host));
+ranger_.reset(new ranger::MiniRanger(cluster_root(), host, postgres_));
 if (opts_.enable_kerberos) {
 
   // The SPNs match the ones defined in mini_ranger_configs.h.
@@ -357,6 +363,21 @@ Status ExternalMiniCluster::Start() {
   "Failed to write Ranger client config");
   }
 
+  if (opts_.enable_ranger_kms) {
+string host = GetBindIpForExternalServer(0);
+ranger_kms

[kudu] branch master updated: KUDU-2671 range-specific hash schema support in Python client

2022-07-22 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 3946ce5f8 KUDU-2671 range-specific hash schema support in Python client
3946ce5f8 is described below

commit 3946ce5f89604477b2a4af03f38de97d384affd2
Author: Alexey Serbin 
AuthorDate: Mon Jul 18 11:37:38 2022 -0700

KUDU-2671 range-specific hash schema support in Python client

With this patch, it's now possible to operate on ranges having custom
hash schemas in Kudu Python client applications.  In essence, the newly
added API directly maps into the Kudu C++ client API.  This patch also
contains tests to cover the newly introduced functionality.

Change-Id: I61426fadc45d70805cf99461d559f0152a79f4a0
Reviewed-on: http://gerrit.cloudera.org:8080/18771
Reviewed-by: Attila Bukor 
Tested-by: Kudu Jenkins
---
 python/kudu/client.pyx   | 139 +++
 python/kudu/libkudu_client.pxd   |  12 
 python/kudu/tests/test_client.py | 109 +-
 3 files changed, 259 insertions(+), 1 deletion(-)

diff --git a/python/kudu/client.pyx b/python/kudu/client.pyx
index 1a09fbb31..78c20925c 100644
--- a/python/kudu/client.pyx
+++ b/python/kudu/client.pyx
@@ -419,6 +419,7 @@ cdef class Client:
 PartialRow lower_bound
 PartialRow upper_bound
 PartialRow split_row
+KuduRangePartition* range_partition
 
 # Apply hash partitioning.
 for col_names, num_buckets, seed in part._hash_partitions:
@@ -429,12 +430,36 @@ cdef class Client:
 c.add_hash_partitions(v, num_buckets, seed)
 else:
 c.add_hash_partitions(v, num_buckets)
+
 # Apply range partitioning
 if part._range_partition_cols is not None:
 v.clear()
 for n in part._range_partition_cols:
 v.push_back(tobytes(n))
 c.set_range_partition_columns(v)
+if part._range_partitions_with_custom_hash_schemas:
+for p in part._range_partitions_with_custom_hash_schemas:
+if not isinstance(p.lower_bound, PartialRow):
+lower_bound = schema.new_row(p.lower_bound)
+else:
+lower_bound = p.lower_bound
+lower_bound._own = 0
+if not isinstance(p.upper_bound, PartialRow):
+upper_bound = schema.new_row(p.upper_bound)
+else:
+upper_bound = p.upper_bound
+upper_bound._own = 0
+range_partition = new KuduRangePartition(
+lower_bound.row,
+upper_bound.row,
+p.lower_bound_type,
+p.upper_bound_type)
+for col_names, num_buckets, seed in p.hash_dimensions:
+v.clear()
+for n in col_names:
+v.push_back(tobytes(n))
+range_partition.add_hash_partitions(v, num_buckets, 
seed if seed else 0)
+c.add_custom_range_partition(range_partition)
 if part._range_partitions:
 for partition in part._range_partitions:
 if not isinstance(partition[0], PartialRow):
@@ -1208,6 +1233,53 @@ cdef class Column:
 
 return result
 
+
+class RangePartition(object):
+"""
+Argument to Client.add_custom_range_partition(...) to contain information
+on the range bounds and range-specific hash schema.
+"""
+def __init__(self,
+ lower_bound=None,
+ upper_bound=None,
+ lower_bound_type='inclusive',
+ upper_bound_type='exclusive'):
+"""
+Parameters
+--
+lower_bound : PartialRow/list/tuple/dict
+upper_bound : PartialRow/list/tuple/dict
+lower_bound_type : {'inclusive', 'exclusive'} or constants
+  kudu.EXCLUSIVE_BOUND and kudu.INCLUSIVE_BOUND
+upper_bound_type : {'inclusive', 'exclusive'} or constants
+  kudu.EXCLUSIVE_BOUND and kudu.INCLUSIVE_BOUND
+"""
+self.lower_bound = lower_bound
+self.upper_bound = upper_bound
+self.lower_bound_type = 
_check_convert_range_bound_type(lower_bound_type)
+self.upper_bound_type = 
_check_convert_range_bound_type(upper_bound_type)
+self.hash_dimensions = []
+
+def add_hash_partitions(self, column_names, num_buckets, seed=None):
+"""
+Adds a dimension with the specified parameters to the custom hash 
schema
+for this range partit

[kudu] branch master updated: KUDU-2671 add tests to check for number of hash buckets

2022-07-21 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 55d711d78 KUDU-2671 add tests to check for number of hash buckets
55d711d78 is described below

commit 55d711d78a21c76718b06dede79de2e9b9d5e8e8
Author: Alexey Serbin 
AuthorDate: Fri Jul 15 11:43:00 2022 -0700

KUDU-2671 add tests to check for number of hash buckets

This patch adds a few test scenarios to cover the handling of invalid
number of hash buckets for range-specific hash schemas in AlterTable
RPCs.

Change-Id: I6bdd728b5dea7fa864648e167a1a76b07c706e8f
Reviewed-on: http://gerrit.cloudera.org:8080/18739
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 .../java/org/apache/kudu/client/TestKuduTable.java | 49 +++
 src/kudu/client/flex_partitioning_client-test.cc   | 47 +++
 src/kudu/master/master-test.cc | 95 ++
 3 files changed, 191 insertions(+)

diff --git 
a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java 
b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
index 80101d806..28c4ac66c 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
@@ -1700,6 +1700,55 @@ public class TestKuduTable {
 }
   }
 
+  @Test(timeout = 10)
+  public void testAlterTableAddRangeCustomHashSchemaWrongBucketsNumber() 
throws Exception {
+final List columns = ImmutableList.of(
+new ColumnSchema.ColumnSchemaBuilder("key", 
Type.INT32).key(true).build(),
+new ColumnSchema.ColumnSchemaBuilder("value", Type.STRING).build());
+final Schema schema = new Schema(columns);
+
+CreateTableOptions options = getBasicCreateTableOptions();
+// Add table-wide schema for the table.
+options.addHashPartitions(ImmutableList.of("key"), 2, 0);
+// Add a range partition with table-wide hash schema.
+{
+  PartialRow lower = schema.newPartialRow();
+  lower.addInt(0, -100);
+  PartialRow upper = schema.newPartialRow();
+  upper.addInt(0, 0);
+  options.addRangePartition(lower, upper);
+}
+
+client.createTable(tableName, schema, options);
+
+PartialRow lower = schema.newPartialRow();
+lower.addInt(0, 0);
+PartialRow upper = schema.newPartialRow();
+upper.addInt(0, 100);
+
+// Try to add range with a single hash bucket -- it should not be possible.
+for (int hashBucketNum = -1; hashBucketNum < 2; ++hashBucketNum) {
+  try {
+RangePartitionWithCustomHashSchema range =
+new RangePartitionWithCustomHashSchema(
+lower,
+upper,
+RangePartitionBound.INCLUSIVE_BOUND,
+RangePartitionBound.EXCLUSIVE_BOUND);
+range.addHashPartitions(ImmutableList.of("key"), hashBucketNum, 0);
+
+client.alterTable(tableName, new 
AlterTableOptions().addRangePartition(range));
+fail(String.format("should not be able to add a partition with " +
+"invalid range-specific hash schema of %d hash buckets", 
hashBucketNum));
+  } catch (KuduException ex) {
+final String errmsg = ex.getMessage();
+assertTrue(errmsg, ex.getStatus().isInvalidArgument());
+assertTrue(String.format("%d hash buckets: %s", hashBucketNum, errmsg),
+errmsg.matches("must have at least two hash buckets"));
+  }
+}
+  }
+
   @Test(timeout = 10)
   @KuduTestHarness.MasterServerConfig(flags = {
   "--enable_per_range_hash_schemas=false",
diff --git a/src/kudu/client/flex_partitioning_client-test.cc 
b/src/kudu/client/flex_partitioning_client-test.cc
index 77e67d990..1d3909940 100644
--- a/src/kudu/client/flex_partitioning_client-test.cc
+++ b/src/kudu/client/flex_partitioning_client-test.cc
@@ -2036,5 +2036,52 @@ TEST_F(FlexPartitioningScanTest, MaxKeyValue) {
   }
 }
 
+// Try adding range partition with custom hash schema where the number of
+// hash buckets is invalid.
+TEST_F(FlexPartitioningAlterTableTest, AddRangeWithWrongHashBucketsNumber) {
+  constexpr const char* const kCol0 = "c0";
+  constexpr const char* const kCol1 = "c1";
+  constexpr const char* const kErrMsg =
+  "at least two buckets are required to establish hash partitioning";
+  constexpr const char* const kTableName = 
"AddRangeWithWrongHashBucketsNumber";
+
+  KuduSchema schema;
+  {
+KuduSchemaBuilder b;
+b.AddColumn(kCol0)->Type(KuduColumnSchema::INT32)->NotNull()->PrimaryKey();
+b.AddColumn(kCol1)->Type(KuduColumnSchema::STRING)->Nullable();
+ASSERT_OK(b.Build());
+  }
+
+  unique_ptr tabl

[kudu] branch master updated: [client] KUDU-2671 move KuduRangePartition out of KuduTableCreator

2022-07-21 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 6fdd91905 [client] KUDU-2671 move KuduRangePartition out of 
KuduTableCreator
6fdd91905 is described below

commit 6fdd9190592fe47c2202e75edd2773832bf29968
Author: Alexey Serbin 
AuthorDate: Wed Jul 20 17:27:40 2022 -0700

[client] KUDU-2671 move KuduRangePartition out of KuduTableCreator

This patch moves the KuduRangePartition class out of the
KuduTableCreator class.  The reasoning behind this change is two-fold:
  * make the API of KuduTableAlterer more consistent with the added
AddRangePartition(KuduRangePartition* partition) method
  * avoid issues with embedded classes while cythonizing C++ client API
while adding support for range-specific hash schemas

This change might break ABI compatibility, but we haven't announced
support for range-specific hash schemas in Kudu 1.16 since it was not
ready yet.  Any Kudu C++ client application that started using that
experimental API (which is very unlikely) should be recompiled with
updated headers and the kudu_client library.

Change-Id: I5af14e7e802baca496e13e05860d66685914dd29
Reviewed-on: http://gerrit.cloudera.org:8080/18765
Reviewed-by: Abhishek Chennaka 
Reviewed-by: Mahesh Reddy 
Tested-by: Alexey Serbin 
Reviewed-by: Attila Bukor 
---
 src/kudu/client/client.cc|  32 +++---
 src/kudu/client/client.h | 131 ---
 src/kudu/client/flex_partitioning_client-test.cc |  12 +--
 src/kudu/client/scan_token-test.cc   |  18 ++--
 src/kudu/client/table_alterer-internal.h |   2 +-
 src/kudu/client/table_creator-internal.cc|   8 +-
 src/kudu/client/table_creator-internal.h |  10 +-
 7 files changed, 103 insertions(+), 110 deletions(-)

diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 7adc33800..d31214371 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -1048,28 +1048,26 @@ Status KuduTableCreator::Create() {
   return Status::OK();
 }
 
-KuduTableCreator::KuduRangePartition::KuduRangePartition(
+KuduRangePartition::KuduRangePartition(
 KuduPartialRow* lower_bound,
 KuduPartialRow* upper_bound,
-RangePartitionBound lower_bound_type,
-RangePartitionBound upper_bound_type)
+KuduTableCreator::RangePartitionBound lower_bound_type,
+KuduTableCreator::RangePartitionBound upper_bound_type)
 : data_(new Data(lower_bound, upper_bound, lower_bound_type, 
upper_bound_type)) {
 }
 
-KuduTableCreator::KuduRangePartition::~KuduRangePartition() {
+KuduRangePartition::~KuduRangePartition() {
   delete data_;
 }
 
-Status KuduTableCreator::KuduRangePartition::add_hash_partitions(
+Status KuduRangePartition::add_hash_partitions(
 const vector& columns,
 int32_t num_buckets,
 int32_t seed) {
   if (seed < 0) {
-// TODO(aserbin): change the signature of
-//KuduRangePartition::add_hash_partitions() to use uint32_t
-//for the 'seed' parameter while it's still possible since
-//the client API hasn't been released yet
-return Status::InvalidArgument("hash seed must non-negative");
+// int32_t, not uint32_t for seed is used to be "compatible" with the type
+// of the 'seed' parameter for KuduTableCreator::add_hash_partitions().
+return Status::InvalidArgument("hash seed must be non-negative");
   }
   return data_->add_hash_partitions(columns, num_buckets, seed);
 }
@@ -1543,9 +1541,8 @@ KuduTableAlterer* 
KuduTableAlterer::AddRangePartitionWithDimension(
 
   Data::Step s { AlterTableRequestPB::ADD_RANGE_PARTITION,
  nullptr,
- std::unique_ptr(
- new KuduTableCreator::KuduRangePartition(
- lower_bound, upper_bound, lower_bound_type, 
upper_bound_type)),
+ std::unique_ptr(new KuduRangePartition(
+ lower_bound, upper_bound, lower_bound_type, 
upper_bound_type)),
  dimension_label.empty() ? nullopt : 
make_optional(dimension_label) };
   data_->steps_.emplace_back(std::move(s));
   data_->has_alter_partitioning_steps = true;
@@ -1553,7 +1550,7 @@ KuduTableAlterer* 
KuduTableAlterer::AddRangePartitionWithDimension(
 }
 
 KuduTableAlterer* KuduTableAlterer::AddRangePartition(
-KuduTableCreator::KuduRangePartition* partition) {
+KuduRangePartition* partition) {
   CHECK(partition);
   if (partition->data_->lower_bound_ == nullptr || 
partition->data_->upper_bound_  == nullptr) {
 data_->status_ = Status::InvalidArgument("range partition bounds may not 
be null");
@@ -1572,7 +1569,7 @@ K

[kudu] branch master updated: [client] remove C++11 extensions from exported header

2022-07-21 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 9aa17addd [client] remove C++11 extensions from exported header
9aa17addd is described below

commit 9aa17addd132956a2a09bf522aa5d565a4098775
Author: Alexey Serbin 
AuthorDate: Wed Jul 20 15:56:51 2022 -0700

[client] remove C++11 extensions from exported header

Changelist 708fdff19 included updates to the exported C++ header
write_op.h.  The C++ client API is still kept C++98 compliant,
so C++11 extensions should not be present.

This patch reverts the changes in src/kudu/client/write_op.h
that 708fdff19 introduced.

This is a follow-up to 708fdff19073dbe9dc4eb9492551443aa38111f9.

Change-Id: I042ccaa76e424475dd2347505a611cdcab5c21d3
Reviewed-on: http://gerrit.cloudera.org:8080/18764
Tested-by: Alexey Serbin 
Reviewed-by: Attila Bukor 
---
 src/kudu/client/write_op.h | 43 ++-
 1 file changed, 22 insertions(+), 21 deletions(-)

diff --git a/src/kudu/client/write_op.h b/src/kudu/client/write_op.h
index e5b76488e..a1e9e30a6 100644
--- a/src/kudu/client/write_op.h
+++ b/src/kudu/client/write_op.h
@@ -30,6 +30,7 @@
 
 #ifdef KUDU_HEADERS_NO_STUBS
 #include "kudu/gutil/macros.h"
+#include "kudu/gutil/port.h"
 #else
 #include "kudu/client/stubs.h"
 #endif
@@ -140,16 +141,16 @@ class KUDU_EXPORT KuduWriteOperation {
 ///   columns which do not have default values.
 class KUDU_EXPORT KuduInsert : public KuduWriteOperation {
  public:
-  ~KuduInsert() override;
+  virtual ~KuduInsert();
 
   /// @copydoc KuduWriteOperation::ToString()
-  std::string ToString() const override { return "INSERT " + row_.ToString(); }
+  virtual std::string ToString() const OVERRIDE { return "INSERT " + 
row_.ToString(); }
 
  protected:
   /// @cond PROTECTED_MEMBERS_DOCUMENTED
 
   /// @copydoc KuduWriteOperation::type()
-  Type type() const override {
+  virtual Type type() const OVERRIDE {
 return INSERT;
   }
 
@@ -167,16 +168,16 @@ class KUDU_EXPORT KuduInsert : public KuduWriteOperation {
 ///   columns which do not have default values.
 class KUDU_EXPORT KuduInsertIgnore : public KuduWriteOperation {
  public:
-  ~KuduInsertIgnore() override;
+  virtual ~KuduInsertIgnore();
 
   /// @copydoc KuduWriteOperation::ToString()
-  std::string ToString() const override { return "INSERT IGNORE " + 
row_.ToString(); }
+  virtual std::string ToString() const OVERRIDE { return "INSERT IGNORE " + 
row_.ToString(); }
 
  protected:
   /// @cond PROTECTED_MEMBERS_DOCUMENTED
 
   /// @copydoc KuduWriteOperation::type()
-  Type type() const override {
+  virtual Type type() const OVERRIDE {
 return INSERT_IGNORE;
   }
 
@@ -193,16 +194,16 @@ class KUDU_EXPORT KuduInsertIgnore : public 
KuduWriteOperation {
 /// See KuduInsert for more details.
 class KUDU_EXPORT KuduUpsert : public KuduWriteOperation {
  public:
-  ~KuduUpsert() override;
+  virtual ~KuduUpsert();
 
   /// @copydoc KuduWriteOperation::ToString()
-  std::string ToString() const override { return "UPSERT " + row_.ToString(); }
+  virtual std::string ToString() const OVERRIDE { return "UPSERT " + 
row_.ToString(); }
 
  protected:
   /// @cond PROTECTED_MEMBERS_DOCUMENTED
 
   /// @copydoc KuduWriteOperation::type()
-  Type type() const override {
+  virtual Type type() const OVERRIDE {
 return UPSERT;
   }
 
@@ -220,16 +221,16 @@ class KUDU_EXPORT KuduUpsert : public KuduWriteOperation {
 ///   in the schema to be set in the embedded KuduPartialRow object.
 class KUDU_EXPORT KuduUpdate : public KuduWriteOperation {
  public:
-  ~KuduUpdate() override;
+  virtual ~KuduUpdate();
 
   /// @copydoc KuduWriteOperation::ToString()
-  std::string ToString() const override { return "UPDATE " + row_.ToString(); }
+  virtual std::string ToString() const OVERRIDE { return "UPDATE " + 
row_.ToString(); }
 
  protected:
   /// @cond PROTECTED_MEMBERS_DOCUMENTED
 
   /// @copydoc KuduWriteOperation::type()
-  Type type() const override {
+  virtual Type type() const OVERRIDE {
 return UPDATE;
   }
 
@@ -246,16 +247,16 @@ class KUDU_EXPORT KuduUpdate : public KuduWriteOperation {
 ///   in the schema to be set in the embedded KuduPartialRow object.
 class KUDU_EXPORT KuduUpdateIgnore : public KuduWriteOperation {
 public:
-  ~KuduUpdateIgnore() override;
+  virtual ~KuduUpdateIgnore();
 
   /// @copydoc KuduWriteOperation::ToString()
-  std::string ToString() const override { return "UPDATE IGNORE " + 
row_.ToString(); }
+  virtual std::string ToString() const OVERRIDE { return "UPDATE IGNORE " + 
row_.ToString(); }
 
 protected:
   /// @cond PROTECTED_MEMBERS_DOCUMENTED
 
   /// @copydoc KuduWriteOper

[kudu] branch master updated: KUDU-2671 check for duplicate columns in hash schema when adding range

2022-07-15 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new dc4031f69 KUDU-2671 check for duplicate columns in hash schema when 
adding range
dc4031f69 is described below

commit dc4031f693382df08c0fab1d0c5ac6bc3c203c35
Author: Alexey Serbin 
AuthorDate: Wed Jul 13 19:33:02 2022 -0700

KUDU-2671 check for duplicate columns in hash schema when adding range

This patch adds validation for range-specific hash schemas when
adding new range partitions.  Without this patch, invalid requests
to add range partitions with duplicate columns across dimensions
of the custom hash schema would be accepted, but tablets could not be
created, resulting in timed-out IsAlterTableDone() RPC.  The patch also
contains new test scenarios for both C++ and Java Kudu clients, making
sure the corresponding error is reported back to the client.  I verified
that the new test scenarios failed as expected if commenting out
the newly added hash schema validation code.

This patch also fixes a few typos in test scenarios from master-test.cc
since the new verification code exposed those mistakes.

Change-Id: Iefe6a97028ae12585ac5496ac8608448ffacd95e
Reviewed-on: http://gerrit.cloudera.org:8080/18728
Reviewed-by: Mahesh Reddy 
Tested-by: Alexey Serbin 
Reviewed-by: Abhishek Chennaka 
Reviewed-by: Alexey Serbin 
---
 .../org/apache/kudu/client/TestAlterTable.java |  62 ++
 src/kudu/client/flex_partitioning_client-test.cc   | 106 ++-
 src/kudu/common/partition.h|   8 +-
 src/kudu/master/catalog_manager.cc |   1 +
 src/kudu/master/master-test.cc | 209 +++--
 5 files changed, 276 insertions(+), 110 deletions(-)

diff --git 
a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java 
b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
index 2c4908020..7e722e0de 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
@@ -819,6 +819,68 @@ public class TestAlterTable {
 }
   }
 
+  @Test(timeout = 10)
+  public void testAlterTryAddRangeWithCustomHashSchemaDuplicateColumns()
+  throws Exception {
+ArrayList columns = new ArrayList<>(2);
+columns.add(new ColumnSchema.ColumnSchemaBuilder("c0", Type.INT32)
+.nullable(false)
+.key(true)
+.build());
+columns.add(new ColumnSchema.ColumnSchemaBuilder("c1", Type.INT32)
+.nullable(false)
+.key(true)
+.build());
+final Schema schema = new Schema(columns);
+
+CreateTableOptions createOptions =
+new CreateTableOptions()
+.setRangePartitionColumns(ImmutableList.of("c0"))
+.addHashPartitions(ImmutableList.of("c0"), 2, 0)
+.addHashPartitions(ImmutableList.of("c1"), 3, 0)
+.setNumReplicas(1);
+
+// Add range partition with table-wide hash schema.
+{
+  PartialRow lower = schema.newPartialRow();
+  lower.addInt("c0", -100);
+  PartialRow upper = schema.newPartialRow();
+  upper.addInt("c0", 0);
+  createOptions.addRangePartition(lower, upper);
+}
+
+client.createTable(tableName, schema, createOptions);
+
+// Try adding a range partition with custom hash schema having multiple
+// hash dimensions and conflicting on columns used for hash function:
+// different dimensions should not intersect on the set of columns
+// used for hashing.
+{
+  PartialRow lower = schema.newPartialRow();
+  lower.addInt("c0", 0);
+  PartialRow upper = schema.newPartialRow();
+  upper.addInt("c0", 100);
+  RangePartitionWithCustomHashSchema range =
+  new RangePartitionWithCustomHashSchema(
+  lower,
+  upper,
+  RangePartitionBound.INCLUSIVE_BOUND,
+  RangePartitionBound.EXCLUSIVE_BOUND);
+  range.addHashPartitions(ImmutableList.of("c0"), 3, 0);
+  range.addHashPartitions(ImmutableList.of("c0"), 3, 0);
+  try {
+client.alterTable(tableName, new 
AlterTableOptions().addRangePartition(range));
+fail("shouldn't be able to add a range with custom hash schema " +
+"having duplicate hash columns across different dimensions");
+  } catch (KuduException ex) {
+final String errmsg = ex.getMessage();
+assertTrue(errmsg, ex.getStatus().isInvalidArgument());
+assertTrue(errmsg, errmsg.matches(
+"hash bucket schema components must not contain column

[kudu] branch master updated: KUDU-2671 Make WebUI compatible with custom hash schema

2022-07-14 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new e8ce3a9bf KUDU-2671  Make WebUI compatible with custom hash schema
e8ce3a9bf is described below

commit e8ce3a9bff9d3a0c00af7e71592e552c33ce5b24
Author: Abhishek Chennaka 
AuthorDate: Fri Jul 8 01:59:58 2022 -0400

KUDU-2671  Make WebUI compatible with custom hash schema

This patch updates the /table?id= page in the Kudu master
WebUI to show custom hash schemas in the sections of:

1. Partition Schema
The custom hash schema if present for a particular range is displayed
right beside the range schema. Different dimensions of the hash
schema are comma separated.

2. Detail
There are new columns to identify if a particular partition has
custom or table wide hash schema, display the hash schema and the hash
partition id of the partition.

The Kudu tablet server WebUI's pages /tablets and
/tablet?id= are also tested to reflect the custom hash
schema or table wide hash schema accordingly.

Below are the screenshots of the WebUI after the changes
Master WebUI:
https://i.imgur.com/O4ra4JA.png
Tablet server WebUI:
https://i.imgur.com/BxdfsYt.png
https://i.imgur.com/l2wA08Q.png

Change-Id: Ic8b8d90f70c39f13b838e858c870e08dacbdfcd3
Reviewed-on: http://gerrit.cloudera.org:8080/18712
Reviewed-by: Alexey Serbin 
Tested-by: Kudu Jenkins
---
 src/kudu/client/flex_partitioning_client-test.cc | 108 +
 src/kudu/common/partition-test.cc|  10 +-
 src/kudu/common/partition.cc | 101 +++-
 src/kudu/common/partition.h  |   8 +
 src/kudu/master/master-test.cc   | 196 ++-
 src/kudu/master/master_path_handlers.cc  |   6 +-
 6 files changed, 413 insertions(+), 16 deletions(-)

diff --git a/src/kudu/client/flex_partitioning_client-test.cc 
b/src/kudu/client/flex_partitioning_client-test.cc
index 8a56050d1..c8af1b26d 100644
--- a/src/kudu/client/flex_partitioning_client-test.cc
+++ b/src/kudu/client/flex_partitioning_client-test.cc
@@ -37,6 +37,7 @@
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/stl_util.h"
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/catalog_manager.h"
 #include "kudu/master/master.h"
 #include "kudu/master/mini_master.h"
@@ -47,6 +48,8 @@
 #include "kudu/tserver/tablet_server.h"
 #include "kudu/tserver/ts_tablet_manager.h"
 #include "kudu/util/metrics.h"
+#include "kudu/util/curl_util.h"
+#include "kudu/util/faststring.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
@@ -55,6 +58,7 @@
 
 DECLARE_bool(enable_per_range_hash_schemas);
 DECLARE_int32(heartbeat_interval_ms);
+DECLARE_string(webserver_doc_root);
 
 METRIC_DECLARE_counter(scans_started);
 
@@ -63,9 +67,13 @@ using kudu::client::KuduValue;
 using kudu::cluster::InternalMiniCluster;
 using kudu::cluster::InternalMiniClusterOptions;
 using kudu::master::CatalogManager;
+using kudu::master::TableInfo;
+using kudu::master::TabletInfo;
+using kudu::tablet::TabletReplica;
 using std::string;
 using std::unique_ptr;
 using std::vector;
+using strings::Substitute;
 
 static constexpr const char* const kKeyColumn = "key";
 static constexpr const char* const kIntValColumn = "int_val";
@@ -97,6 +105,10 @@ class FlexPartitioningTest : public KuduTest {
 // Reduce the TS<->Master heartbeat interval to speed up testing.
 FLAGS_heartbeat_interval_ms = 10;
 
+// Ensure the static pages are not available as tests are written based
+// on this value of the flag
+FLAGS_webserver_doc_root = "";
+
 // Start minicluster and wait for tablet servers to connect to master.
 cluster_.reset(new InternalMiniCluster(env_, 
InternalMiniClusterOptions()));
 ASSERT_OK(cluster_->Start());
@@ -546,6 +558,102 @@ TEST_F(FlexPartitioningCreateTableTest, 
DefaultAndCustomHashSchemas) {
   }
 }
 
+TEST_F(FlexPartitioningCreateTableTest, TabletServerWebUI) {
+  // Create a table with the following partitions:
+  //
+  //hash bucket
+  //   key0   1   2   3
+  // ---
+  //  <111x:{key} x:{key} -   -
+  // 111-222  x:{key} x:{key} x:{key} -
+  // 222-333  x:{key} x:{key} x:{key} x:{key}
+  constexpr const char* const kTableName = "TabletServerWebUI";
+
+  unique_ptr table_creator(client_->NewTableCreator());
+  tabl

[kudu] branch master updated: [refactor] Fix some lint and tidy errors

2022-07-13 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 708fdff19 [refactor] Fix some lint and tidy errors
708fdff19 is described below

commit 708fdff19073dbe9dc4eb9492551443aa38111f9
Author: Yingchun Lai 
AuthorDate: Wed Jul 13 23:56:02 2022 +0800

[refactor] Fix some lint and tidy errors

This patch has no functional changes, it fixes some lint and
tidy errors, it's the first step to implement immutable column
feature, aiming to reduce the diff code when we review that
patch in https://gerrit.cloudera.org/c/18241/.

Change-Id: I89b0f3b9ace02f380ce1193bc1facf2aaf135096
Reviewed-on: http://gerrit.cloudera.org:8080/18724
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 src/kudu/client/write_op.h  | 43 ++---
 src/kudu/common/row.h   |  1 +
 src/kudu/common/row_changelist.cc   |  8 ++
 src/kudu/common/row_changelist.h|  4 +--
 src/kudu/common/schema-test.cc  | 14 +-
 src/kudu/tablet/delta_applier.cc| 12 
 src/kudu/tablet/delta_applier.h | 30 ++--
 src/kudu/tablet/delta_iterator_merger.cc| 16 ---
 src/kudu/tablet/delta_iterator_merger.h |  2 +-
 src/kudu/tablet/delta_store.h   |  6 ++--
 src/kudu/tablet/deltafile.cc|  2 +-
 src/kudu/tablet/deltafile.h |  2 +-
 src/kudu/tablet/deltamemstore-test.cc   |  4 +--
 src/kudu/tablet/deltamemstore.cc|  2 +-
 src/kudu/tablet/deltamemstore.h |  2 +-
 src/kudu/tablet/ops/op.h|  3 +-
 src/kudu/tablet/rowset.cc   |  1 -
 src/kudu/tablet/rowset.h|  2 +-
 src/kudu/tablet/tablet-decoder-eval-test.cc |  6 ++--
 src/kudu/tablet/tablet.cc   |  7 ++---
 20 files changed, 78 insertions(+), 89 deletions(-)

diff --git a/src/kudu/client/write_op.h b/src/kudu/client/write_op.h
index a1e9e30a6..e5b76488e 100644
--- a/src/kudu/client/write_op.h
+++ b/src/kudu/client/write_op.h
@@ -30,7 +30,6 @@
 
 #ifdef KUDU_HEADERS_NO_STUBS
 #include "kudu/gutil/macros.h"
-#include "kudu/gutil/port.h"
 #else
 #include "kudu/client/stubs.h"
 #endif
@@ -141,16 +140,16 @@ class KUDU_EXPORT KuduWriteOperation {
 ///   columns which do not have default values.
 class KUDU_EXPORT KuduInsert : public KuduWriteOperation {
  public:
-  virtual ~KuduInsert();
+  ~KuduInsert() override;
 
   /// @copydoc KuduWriteOperation::ToString()
-  virtual std::string ToString() const OVERRIDE { return "INSERT " + 
row_.ToString(); }
+  std::string ToString() const override { return "INSERT " + row_.ToString(); }
 
  protected:
   /// @cond PROTECTED_MEMBERS_DOCUMENTED
 
   /// @copydoc KuduWriteOperation::type()
-  virtual Type type() const OVERRIDE {
+  Type type() const override {
 return INSERT;
   }
 
@@ -168,16 +167,16 @@ class KUDU_EXPORT KuduInsert : public KuduWriteOperation {
 ///   columns which do not have default values.
 class KUDU_EXPORT KuduInsertIgnore : public KuduWriteOperation {
  public:
-  virtual ~KuduInsertIgnore();
+  ~KuduInsertIgnore() override;
 
   /// @copydoc KuduWriteOperation::ToString()
-  virtual std::string ToString() const OVERRIDE { return "INSERT IGNORE " + 
row_.ToString(); }
+  std::string ToString() const override { return "INSERT IGNORE " + 
row_.ToString(); }
 
  protected:
   /// @cond PROTECTED_MEMBERS_DOCUMENTED
 
   /// @copydoc KuduWriteOperation::type()
-  virtual Type type() const OVERRIDE {
+  Type type() const override {
 return INSERT_IGNORE;
   }
 
@@ -194,16 +193,16 @@ class KUDU_EXPORT KuduInsertIgnore : public 
KuduWriteOperation {
 /// See KuduInsert for more details.
 class KUDU_EXPORT KuduUpsert : public KuduWriteOperation {
  public:
-  virtual ~KuduUpsert();
+  ~KuduUpsert() override;
 
   /// @copydoc KuduWriteOperation::ToString()
-  virtual std::string ToString() const OVERRIDE { return "UPSERT " + 
row_.ToString(); }
+  std::string ToString() const override { return "UPSERT " + row_.ToString(); }
 
  protected:
   /// @cond PROTECTED_MEMBERS_DOCUMENTED
 
   /// @copydoc KuduWriteOperation::type()
-  virtual Type type() const OVERRIDE {
+  Type type() const override {
 return UPSERT;
   }
 
@@ -221,16 +220,16 @@ class KUDU_EXPORT KuduUpsert : public KuduWriteOperation {
 ///   in the schema to be set in the embedded KuduPartialRow object.
 class KUDU_EXPORT KuduUpdate : public KuduWriteOperation {
  public:
-  virtual ~KuduUpdate();
+  ~KuduUpdate() override;
 
   /// @copydoc KuduWriteOperation::ToString()
-  virtual std::string ToString() const OVERRIDE { return "UPDATE " + 
row_.ToString

[kudu] branch master updated: [client-test] more robust MinReplicationFactor scenario

2022-07-13 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new b60b6fd77 [client-test] more robust MinReplicationFactor scenario
b60b6fd77 is described below

commit b60b6fd7751b3b67156f460fb46dabd2db00d9d7
Author: Alexey Serbin 
AuthorDate: Mon Jul 11 19:01:32 2022 -0700

[client-test] more robust MinReplicationFactor scenario

This patch updates the ReplicationFactorLimitsTest.MinReplicationFactor
test scenario to make it more robust.  With this update, test failures
like below shouldn't happen anymore:

  src/kudu/client/client-test.cc:8722: Failure
  Value of: s.ToString()
  Expected: has substring "not enough live tablet servers to create a table 
with the requested replication factor 3; 1 tablet servers are alive"
Actual: "Invalid argument: Error creating table foobar on the master: 
not enough live tablet servers to create a table with the requested replication 
factor 3; 0 tablet servers are alive"

Change-Id: I509f04067f28be566f655050f1b29f38654df24b
Reviewed-on: http://gerrit.cloudera.org:8080/18719
Reviewed-by: Mahesh Reddy 
Tested-by: Kudu Jenkins
Reviewed-by: Yingchun Lai 
---
 src/kudu/client/client-test.cc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index 25c0961e6..97a38618c 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -8695,7 +8695,7 @@ TEST_F(ReplicationFactorLimitsTest, MinReplicationFactor) 
{
 const vector> cases = {
   {2, "illegal replication factor 2: replication factor must be odd"},
   {3, "not enough live tablet servers to create a table with the requested 
"
-  "replication factor 3; 1 tablet servers are alive"},
+  "replication factor 3"},
 };
 
 for (auto i = 1; i < cluster_->num_tablet_servers(); ++i) {



[kudu] branch master updated: KUDU-2671 Remove commas from multi-level custom hash schema printing

2022-07-12 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 317ff2a17 KUDU-2671 Remove commas from multi-level custom hash schema 
printing
317ff2a17 is described below

commit 317ff2a1763211d399e2c6c3afa47cad766d5a03
Author: Kurt Deschler 
AuthorDate: Mon Jul 11 21:29:41 2022 -0500

KUDU-2671 Remove commas from multi-level custom hash schema printing

This patch removes commas separating hash dimensions in the formatted
output of a range-specific hash schema.  An extra test scenario is
added to spot regressions in the formatted output and cover the
functionality of creating and altering tables with multi-dimensional
hash schemas.

Change-Id: I424790b62fb4a323f59d4d33901faae57cf39ad0
Reviewed-on: http://gerrit.cloudera.org:8080/18720
Tested-by: Kudu Jenkins
Reviewed-by: Kurt Deschler 
Reviewed-by: Alexey Serbin 
---
 .../java/org/apache/kudu/client/Partition.java |   6 -
 .../java/org/apache/kudu/client/TestKuduTable.java | 234 +
 2 files changed, 234 insertions(+), 6 deletions(-)

diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/Partition.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/Partition.java
index 1fce1e2b5..7faaf8a91 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/Partition.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/Partition.java
@@ -300,13 +300,7 @@ public class Partition implements Comparable {
 if (showHashInfo) {
   List hashSchema =
   partitionSchema.getHashSchemaForRange(rangeKeyStart);
-  boolean firstHashDimension = true;
   for (PartitionSchema.HashBucketSchema hashDimension : hashSchema) {
-if (firstHashDimension) {
-  firstHashDimension = false;
-} else {
-  sb.append(',');
-}
 sb.append(" HASH(");
 boolean firstId = true;
 for (Integer id : hashDimension.getColumnIds()) {
diff --git 
a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java 
b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
index 40a94c52f..0d4a08028 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
@@ -45,6 +45,7 @@ import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import org.junit.Before;
 import org.junit.Rule;
@@ -57,6 +58,7 @@ import org.apache.kudu.Schema;
 import org.apache.kudu.Type;
 import org.apache.kudu.test.ClientTestUtil;
 import org.apache.kudu.test.KuduTestHarness;
+import org.apache.kudu.util.Pair;
 
 public class TestKuduTable {
   private static final Logger LOG = 
LoggerFactory.getLogger(TestKuduTable.class);
@@ -1993,6 +1995,238 @@ public class TestKuduTable {
 }
   }
 
+  @Test(timeout = 10)
+  @KuduTestHarness.MasterServerConfig(flags = {
+  "--enable_per_range_hash_schemas=true",
+  })
+  public void testAlterTableAddRangePartitionCustomHashSchemaMultiDimensional()
+  throws Exception {
+final List columns = ImmutableList.of(
+new ColumnSchema.ColumnSchemaBuilder("c0", 
Type.INT32).key(true).build(),
+new ColumnSchema.ColumnSchemaBuilder("c1", 
Type.INT64).key(true).build(),
+new ColumnSchema.ColumnSchemaBuilder("value", 
Type.STRING).nullable(true).build());
+final Schema schema = new Schema(columns);
+
+CreateTableOptions opt = new CreateTableOptions();
+opt.setRangePartitionColumns(ImmutableList.of("c0"));
+// Define table-wide schema for the table.
+opt.addHashPartitions(ImmutableList.of("c0"), 2, 0);
+opt.addHashPartitions(ImmutableList.of("c1"), 3, 0);
+
+// Add a range partition with table-wide hash schema.
+{
+  PartialRow lower = schema.newPartialRow();
+  lower.addInt(0, -100);
+  PartialRow upper = schema.newPartialRow();
+  upper.addInt(0, 100);
+  opt.addRangePartition(lower, upper);
+}
+
+client.createTable(tableName, schema, opt);
+
+assertEquals(
+ImmutableList.of("-100 <= VALUES < 100"),
+client.openTable(tableName).getFormattedRangePartitions(1));
+assertEquals(
+ImmutableList.of("-100 <= VALUES < 100 HASH(c0) PARTITIONS 2 HASH(c1) 
PARTITIONS 3"),
+
client.openTable(tableName).getFormattedRangePartitionsWithHashSchema(1));
+
+{
+  PartialRow lower = schema.newPartialRow();
+  lower.addInt(0, 100);
+  PartialRow upper = schema.newPartialRow();
+  upper.addInt(0, 200);
+
+  Ra

[kudu] branch master updated: KUDU-2671: Extra test coverage for KuduScanTokens.

2022-07-12 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 8c20126d8 KUDU-2671: Extra test coverage for KuduScanTokens.
8c20126d8 is described below

commit 8c20126d8a648f3fed15b4f41527089947d80d11
Author: Mahesh Reddy 
AuthorDate: Tue Jul 12 16:49:26 2022 -0700

KUDU-2671: Extra test coverage for KuduScanTokens.

This patch adds extra test coverage for scans using
KuduScanTokens. Specifically, the test case is on a
table with custom hash schemas and non covering ranges.

Change-Id: I0dec5c713940c1f8d66416c6a287b93e250d90e6
Reviewed-on: http://gerrit.cloudera.org:8080/18721
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 src/kudu/client/client.h   |   4 -
 src/kudu/client/scan_token-test.cc | 214 +++--
 2 files changed, 208 insertions(+), 10 deletions(-)

diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 418dfe261..5fabee2ac 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -3114,10 +3114,6 @@ class KUDU_EXPORT KuduScanner {
 ///
 /// Scan token locality information can be inspected using the
 /// KuduScanToken::tablet() function.
-///
-/// Scan tokens are not yet compatible for tables that contain range-specific
-/// hash schemas. To be clear, the existing use case of tables with all ranges
-/// using the table wide hash schema is functional as expected.
 class KUDU_EXPORT KuduScanToken {
  public:
 
diff --git a/src/kudu/client/scan_token-test.cc 
b/src/kudu/client/scan_token-test.cc
index f38dae6ae..d0a6f5377 100644
--- a/src/kudu/client/scan_token-test.cc
+++ b/src/kudu/client/scan_token-test.cc
@@ -595,16 +595,15 @@ TEST_F(ScanTokenTest, 
ScanTokensWithCustomHashSchemasPerRange) {
   // Create table
   shared_ptr table;
   {
-unique_ptr lower_bound(schema.NewRow());
-unique_ptr upper_bound(schema.NewRow());
 unique_ptr table_creator(client_->NewTableCreator());
 table_creator->table_name("table");
 table_creator->num_replicas(1);
 table_creator->schema();
-
 table_creator->add_hash_partitions({ "col" }, 2);
 
 {
+  unique_ptr lower_bound(schema.NewRow());
+  unique_ptr upper_bound(schema.NewRow());
   ASSERT_OK(lower_bound->SetInt64("col", 0));
   ASSERT_OK(upper_bound->SetInt64("col", 100));
   unique_ptr range_partition(
@@ -615,14 +614,14 @@ TEST_F(ScanTokenTest, 
ScanTokensWithCustomHashSchemasPerRange) {
 }
 
 {
-  lower_bound.reset(schema.NewRow());
-  upper_bound.reset(schema.NewRow());
+  unique_ptr lower_bound(schema.NewRow());
+  unique_ptr upper_bound(schema.NewRow());
   ASSERT_OK(lower_bound->SetInt64("col", 100));
   ASSERT_OK(upper_bound->SetInt64("col", 200));
   unique_ptr range_partition(
   new KuduTableCreator::KuduRangePartition(lower_bound.release(),
upper_bound.release()));
-  range_partition->add_hash_partitions({ "col"}, 2);
+  range_partition->add_hash_partitions({ "col" }, 2);
   table_creator->add_custom_range_partition(range_partition.release());
 }
 
@@ -732,6 +731,209 @@ TEST_F(ScanTokenTest, 
ScanTokensWithCustomHashSchemasPerRange) {
   }
 }
 
+TEST_F(ScanTokenTest, TestScanTokensWithCustomHashSchemasPerNonCoveringRange) {
+  FLAGS_enable_per_range_hash_schemas = true;
+  KuduSchema schema;
+  // Create schema
+  {
+KuduSchemaBuilder builder;
+
builder.AddColumn("col")->NotNull()->Type(KuduColumnSchema::INT64)->PrimaryKey();
+ASSERT_OK(builder.Build());
+  }
+
+  // Create table
+  shared_ptr table;
+  {
+unique_ptr table_creator(client_->NewTableCreator());
+table_creator->table_name("table");
+table_creator->num_replicas(1);
+table_creator->schema();
+table_creator->add_hash_partitions({ "col" }, 2);
+
+{
+  unique_ptr lower_bound(schema.NewRow());
+  unique_ptr upper_bound(schema.NewRow());
+  ASSERT_OK(lower_bound->SetInt64("col", 0));
+  ASSERT_OK(upper_bound->SetInt64("col", 100));
+  unique_ptr range_partition(
+  new KuduTableCreator::KuduRangePartition(lower_bound.release(), 
upper_bound.release()));
+  range_partition->add_hash_partitions({ "col" }, 4);
+  table_creator->add_custom_range_partition(range_partition.release());
+}
+
+{
+  unique_ptr lower_bound(schema.NewRow());
+  unique_ptr upper_bound(schema.NewRow());
+  ASSERT_OK(lower_bound->SetInt64("col", 200));
+  ASSERT_OK(upper_bound->SetInt64("col", 300));
+  unique_ptr range_parti

[kudu] branch master updated: KUDU-2671 forward-looking provision for AddRangePartition

2022-07-11 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new e0f96b9c8 KUDU-2671 forward-looking provision for AddRangePartition
e0f96b9c8 is described below

commit e0f96b9c838e33b93690a76f771d1eeaf3a99222
Author: Alexey Serbin 
AuthorDate: Thu Jul 7 17:03:55 2022 -0700

KUDU-2671 forward-looking provision for AddRangePartition

The way how the information on range-specific hash schema is specified
in AlterTableRequestPB::AddRangePartition introduced by [1] assumes
there should not be an empty custom hash schema for a range when the
table-wide hash schema isn't empty.  As of now, the assumption holds
true since there is an artificial restriction on the variability of the
number of dimensions in per-range hash schemas in a table introduced
by changelist [2].  However, once the restriction introduced in [2] is
removed, the current type of the AddRangePartition::custom_hash_schema
deprives the code to tell between the case of a range-specific hash
schema with zero hash dimensions (a.k.a. empty hash schema, i.e. no hash
bucketing at all) and the case of table-wide hash schema for a newly
added range partition.

This patch fixes the deficiency, so now it's possible to call
has_custom_hash_schema() and hasCustomHashSchema() on AddRangePartition
object in C++ and Java code correspondingly, not relying on the
emptiness of the repeated field representing the set of hash dimensions
for the range-specific hash schema.

This patch would break backwards compatibility if there were a version
of Kudu released with the change introduced in changlist [1], but that's
not the case.  So, it was possible to simply change the type of the
AddRangePartition::custom_hash_schema field.

[1] 
https://github.com/apache/kudu/commit/11db3f28b36d92ce1515bcaace51a3586838abcb
[2] 
https://github.com/apache/kudu/commit/6998193e69eeda497f912d1d806470c95b591ad4

Change-Id: I30f654443c7f51a76dea9d980588b399b06c2dd1
Reviewed-on: http://gerrit.cloudera.org:8080/18713
Tested-by: Alexey Serbin 
Reviewed-by: Mahesh Reddy 
Reviewed-by: Abhishek Chennaka 
Reviewed-by: Alexey Serbin 
---
 .../java/org/apache/kudu/client/AlterTableOptions.java |  8 ++--
 src/kudu/client/table_alterer-internal.cc  |  2 +-
 src/kudu/master/catalog_manager.cc |  7 ---
 src/kudu/master/master-test.cc |  8 +---
 src/kudu/master/master.proto   | 14 --
 5 files changed, 24 insertions(+), 15 deletions(-)

diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
index 1603d0801..e5e8f193f 100644
--- 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
+++ 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
@@ -385,12 +385,8 @@ public class AlterTableOptions {
 new Operation.OperationsEncoder().encodeLowerAndUpperBounds(
 range.getLowerBound(), range.getUpperBound(),
 range.getLowerBoundType(), range.getUpperBoundType()));
-for (org.apache.kudu.Common.PartitionSchemaPB.HashBucketSchemaPB 
hashSchema :
-range.toPB().getHashSchemaList()) {
-  Common.PartitionSchemaPB.HashBucketSchemaPB.Builder hbs =
-  rangeBuilder.addCustomHashSchemaBuilder();
-  hbs.mergeFrom(hashSchema);
-}
+rangeBuilder.getCustomHashSchemaBuilder().addAllHashSchema(
+range.toPB().getHashSchemaList());
 step.setAddRangePartition(rangeBuilder);
 if (!pb.hasSchema()) {
   pb.setSchema(ProtobufHelper.schemaToPb(range.getLowerBound().getSchema(),
diff --git a/src/kudu/client/table_alterer-internal.cc 
b/src/kudu/client/table_alterer-internal.cc
index 8e43ebfe2..68a8b8def 100644
--- a/src/kudu/client/table_alterer-internal.cc
+++ b/src/kudu/client/table_alterer-internal.cc
@@ -183,7 +183,7 @@ Status 
KuduTableAlterer::Data::ToRequest(AlterTableRequestPB* req) {
 
 for (const auto& hash_dimension : partition_data->hash_schema_) {
   auto* custom_hash_schema_pb = 
pb_step->mutable_add_range_partition()->
-  add_custom_hash_schema();
+  mutable_custom_hash_schema()->add_hash_schema();
   for (const auto& column_name : hash_dimension.column_names) {
 custom_hash_schema_pb->add_columns()->set_name(column_name);
   }
diff --git a/src/kudu/master/catalog_manager.cc 
b/src/kudu/master/catalog_manager.cc
index 17411b1c3..534405aa6 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -

[kudu] branch master updated: KUDU-2671 show custom hash schema in formatRangePartition

2022-07-11 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 2696d5d04 KUDU-2671 show custom hash schema in formatRangePartition
2696d5d04 is described below

commit 2696d5d044a44f35abf906ac371eebd627c98d08
Author: Kurt Deschler 
AuthorDate: Fri Jul 1 21:33:58 2022 -0500

KUDU-2671 show custom hash schema in formatRangePartition

This patch introduces a new method targeted for Impala for formatted
output of table's range partitions along with hash their schemas.
Hash schema information is appended to each range partition.

Several test scenarios in TestKuduTable are updated to provide coverage
for the functionality of the newly introduced method
KuduTable.getFormattedRangePartitionsWithHashSchema().

Change-Id: Ia1a014ad820b2d73ea09fe3f67baefe9d0acb7aa
Reviewed-on: http://gerrit.cloudera.org:8080/18693
Tested-by: Alexey Serbin 
Reviewed-by: Kurt Deschler 
Reviewed-by: Alexey Serbin 
---
 .../java/org/apache/kudu/client/KuduTable.java | 22 ++-
 .../java/org/apache/kudu/client/Partition.java | 27 -
 .../java/org/apache/kudu/client/TestKuduTable.java | 46 ++
 3 files changed, 76 insertions(+), 19 deletions(-)

diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/KuduTable.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/KuduTable.java
index b6fee17cf..c5a7c2636 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/KuduTable.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/KuduTable.java
@@ -303,7 +303,27 @@ public class KuduTable {
 List rangePartitions = getRangePartitions(timeout);
 List formattedPartitions = new ArrayList<>();
 for (Partition partition : rangePartitions) {
-  formattedPartitions.add(partition.formatRangePartition(this));
+  formattedPartitions.add(partition.formatRangePartition(this, false));
+}
+return formattedPartitions;
+  }
+
+  /**
+   * Retrieves a formatted representation of this table's range partitions 
along
+   * with hash schema output for each range. The range partitions are returned
+   * in sorted order by value and contain no duplicates.
+   *
+   * @param timeout the timeout of the operation
+   * @return a list of the formatted range partitions with hash schema for each
+   */
+  @InterfaceAudience.LimitedPrivate("Impala")
+  @InterfaceStability.Unstable
+  public List getFormattedRangePartitionsWithHashSchema(long timeout)
+  throws Exception {
+List rangePartitions = getRangePartitions(timeout);
+List formattedPartitions = new ArrayList<>();
+for (Partition partition : rangePartitions) {
+  formattedPartitions.add(partition.formatRangePartition(this, true));
 }
 return formattedPartitions;
   }
diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/Partition.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/Partition.java
index 7f9a80f57..1fce1e2b5 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/Partition.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/Partition.java
@@ -220,9 +220,10 @@ public class Partition implements Comparable {
* Formats the range partition into a string suitable for debug printing.
*
* @param table that this partition belongs to
+   * @param showHashInfo whether to output hash schema info per range
* @return a string containing a formatted representation of the range 
partition
*/
-  String formatRangePartition(KuduTable table) {
+  String formatRangePartition(KuduTable table, boolean showHashInfo) {
 Schema schema = table.getSchema();
 PartitionSchema partitionSchema = table.getPartitionSchema();
 PartitionSchema.RangeSchema rangeSchema = partitionSchema.getRangeSchema();
@@ -296,6 +297,30 @@ public class Partition implements Comparable {
   }
 }
 
+if (showHashInfo) {
+  List hashSchema =
+  partitionSchema.getHashSchemaForRange(rangeKeyStart);
+  boolean firstHashDimension = true;
+  for (PartitionSchema.HashBucketSchema hashDimension : hashSchema) {
+if (firstHashDimension) {
+  firstHashDimension = false;
+} else {
+  sb.append(',');
+}
+sb.append(" HASH(");
+boolean firstId = true;
+for (Integer id : hashDimension.getColumnIds()) {
+  if (firstId) {
+firstId = false;
+  } else {
+sb.append(',');
+  }
+  
sb.append(schema.getColumnByIndex(schema.getColumnIndex(id)).getName());
+}
+sb.append(") PARTITIONS ");
+sb.append(hashDimension.getNumBuckets());
+  }
+}
 return sb.toString();
   }
 }
diff --git 
a/java/kudu-c

[kudu] 01/03: KUDU-2671 fix handling ranges with table-wide hash schemas

2022-07-11 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 1609ecc72487a3237ccc464e3fa6f82b3d173e17
Author: Alexey Serbin 
AuthorDate: Sat Jul 2 21:40:38 2022 -0700

KUDU-2671 fix handling ranges with table-wide hash schemas

This patch fixes a bug/typo introduced recently: adding a range with
the table-wide hash schema would be a no-op when
--enable_per_range_hash_schemas was set 'true'.

I added new test scenarios to cover the regression and verified
that, as expected, they would fail without the fix.

This is a follow-up to f55dd22bfea4beee99d72891efbbc67307b19d1e.

Change-Id: I908e9654ed856920d483ce63a546946cbac0a641
Reviewed-on: http://gerrit.cloudera.org:8080/18696
Tested-by: Alexey Serbin 
Reviewed-by: Mahesh Reddy 
Reviewed-by: Abhishek Chennaka 
Reviewed-by: Attila Bukor 
---
 src/kudu/client/flex_partitioning_client-test.cc |  84 +
 src/kudu/master/catalog_manager.cc   |  27 ++-
 src/kudu/master/master-test.cc   | 219 ---
 3 files changed, 299 insertions(+), 31 deletions(-)

diff --git a/src/kudu/client/flex_partitioning_client-test.cc 
b/src/kudu/client/flex_partitioning_client-test.cc
index 04013900c..35a10a54f 100644
--- a/src/kudu/client/flex_partitioning_client-test.cc
+++ b/src/kudu/client/flex_partitioning_client-test.cc
@@ -1675,5 +1675,89 @@ TEST_F(FlexPartitioningAlterTableTest, 
UnsupportedRangeSpecificHashSchema) {
   ASSERT_OK(client_->DeleteTable(kTableName));
 }
 
+// Make sure adding and dropping ranges with the table-wide hash schema works
+// as expected when --enable_per_range_hash_schemas=true.
+TEST_F(FlexPartitioningAlterTableTest, AddDropTableWideHashSchemaPartitions) {
+  FLAGS_enable_per_range_hash_schemas = true;
+
+  constexpr const char* const kTableName =
+  "AddDropTableWideHashSchemaPartitions";
+
+  unique_ptr table_creator(client_->NewTableCreator());
+  table_creator->table_name(kTableName)
+  .schema(_)
+  .num_replicas(1)
+  .add_hash_partitions({ kKeyColumn }, 2)
+  .set_range_partition_columns({ kKeyColumn });
+
+  // Add a range partition with the table-wide hash partitioning rules.
+  {
+unique_ptr lower(schema_.NewRow());
+ASSERT_OK(lower->SetInt32(kKeyColumn, 0));
+unique_ptr upper(schema_.NewRow());
+ASSERT_OK(upper->SetInt32(kKeyColumn, 111));
+table_creator->add_range_partition(lower.release(), upper.release());
+  }
+
+  ASSERT_OK(table_creator->Create());
+
+  ASSERT_OK(InsertTestRows(kTableName, 0, 111));
+  NO_FATALS(CheckTableRowsNum(kTableName, 111));
+
+  // To have mix of ranges, add a single range with custom hash schema.
+  {
+unique_ptr alterer(client_->NewTableAlterer(kTableName));
+auto p = CreateRangePartition(-111, 0);
+ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 5, 1));
+alterer->AddRangePartition(p.release());
+ASSERT_OK(alterer->Alter());
+  }
+
+  ASSERT_OK(InsertTestRows(kTableName, -111, 0));
+  NO_FATALS(CheckTableRowsNum(kTableName, 222));
+
+  // Add one more range partition with the table-wide hash schema.
+  {
+unique_ptr alterer(client_->NewTableAlterer(kTableName));
+
+unique_ptr lower(schema_.NewRow());
+ASSERT_OK(lower->SetInt32(kKeyColumn, 111));
+unique_ptr upper(schema_.NewRow());
+ASSERT_OK(upper->SetInt32(kKeyColumn, 222));
+alterer->AddRangePartition(lower.release(), upper.release());
+
+ASSERT_OK(alterer->Alter());
+  }
+
+  ASSERT_OK(InsertTestRows(kTableName, 111, 222));
+  NO_FATALS(CheckTableRowsNum(kTableName, 333));
+
+  // Drop the ranges with the table-wide hash partitions.
+  {
+unique_ptr alterer(client_->NewTableAlterer(kTableName));
+
+{
+  unique_ptr lower(schema_.NewRow());
+  ASSERT_OK(lower->SetInt32(kKeyColumn, 111));
+  unique_ptr upper(schema_.NewRow());
+  ASSERT_OK(upper->SetInt32(kKeyColumn, 222));
+  alterer->DropRangePartition(lower.release(), upper.release());
+}
+{
+  unique_ptr lower(schema_.NewRow());
+  ASSERT_OK(lower->SetInt32(kKeyColumn, 0));
+  unique_ptr upper(schema_.NewRow());
+  ASSERT_OK(upper->SetInt32(kKeyColumn, 111));
+  alterer->DropRangePartition(lower.release(), upper.release());
+}
+
+ASSERT_OK(alterer->Alter());
+  }
+
+  NO_FATALS(CheckTableRowsNum(kTableName, 111));
+
+  ASSERT_OK(client_->DeleteTable(kTableName));
+}
+
 } // namespace client
 } // namespace kudu
diff --git a/src/kudu/master/catalog_manager.cc 
b/src/kudu/master/catalog_manager.cc
index 78fc4ec75..17411b1c3 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -2677,16 +2677,17 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
 vector part

[kudu] branch master updated (aba975b2c -> 23908bdbe)

2022-07-11 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


from aba975b2c [unittest] LogRollingITest offen failed at asan, because of 
glog' leak
 new 1609ecc72 KUDU-2671 fix handling ranges with table-wide hash schemas
 new 8c8f393a4 KUDU-2671 fix double-free in new scenario of scan_token-test
 new 23908bdbe KUDU-2671 fix consistency check in KuduScanner::NextBatch()

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/kudu/client/client.cc|   3 +-
 src/kudu/client/flex_partitioning_client-test.cc | 121 ++---
 src/kudu/client/scan_token-test.cc   |  23 +--
 src/kudu/client/scanner-internal.cc  |   1 +
 src/kudu/master/catalog_manager.cc   |  27 ++-
 src/kudu/master/master-test.cc   | 219 ---
 6 files changed, 328 insertions(+), 66 deletions(-)



[kudu] 03/03: KUDU-2671 fix consistency check in KuduScanner::NextBatch()

2022-07-11 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 23908bdbe753c25aa2a33796559ecf64f0197949
Author: Alexey Serbin 
AuthorDate: Fri Jul 8 17:01:34 2022 -0700

KUDU-2671 fix consistency check in KuduScanner::NextBatch()

KuduScanner::Data::OpenTablet() can return earlier when the metacache
returns a tablet covering the partition key range past the partition key
provided.  In such case, the 'proxy_' member of KuduScanner::Data isn't
set, and that's expected since the upper level code should continue with
the next tablet.  The new test sub-scenarios introduced in [1] triggered
the consistency check on the 'data_->proxy_' member in
KuduScanner::NextBatch(), so this patch addresses the issue.

Also, this patch re-enables a few test sub-scenarios introduced in
[1] and [2].

[1] 
https://github.com/apache/kudu/commit/b746978c71ce4a95b69d49c43d0ac852909a8b4e
[2] 
https://github.com/apache/kudu/commit/1889d4c44385fec5efeeb2d287d9ab7a3544dcfe

Change-Id: Icfbdfac46f35d4d143f37845d158795bf1793da7
Reviewed-on: http://gerrit.cloudera.org:8080/18715
Reviewed-by: Khazar Mammadli 
Tested-by: Alexey Serbin 
Reviewed-by: Mahesh Reddy 
Reviewed-by: Abhishek Chennaka 
Reviewed-by: Attila Bukor 
---
 src/kudu/client/client.cc|  3 +-
 src/kudu/client/flex_partitioning_client-test.cc | 37 +---
 src/kudu/client/scanner-internal.cc  |  1 +
 3 files changed, 17 insertions(+), 24 deletions(-)

diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 46b9a53da..7adc33800 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -2027,7 +2027,6 @@ Status 
KuduScanner::NextBatch(internal::ScanBatchDataInterface* batch_data) {
   // need to do some swapping of the response objects around to avoid
   // stomping on the memory the user is looking at.
   CHECK(data_->open_);
-  CHECK(data_->proxy_);
 
   batch_data->Clear();
 
@@ -2037,6 +2036,7 @@ Status 
KuduScanner::NextBatch(internal::ScanBatchDataInterface* batch_data) {
 
   if (data_->data_in_open_) {
 // We have data from a previous scan.
+CHECK(data_->proxy_);
 VLOG(2) << "Extracting data from " << data_->DebugString();
 data_->data_in_open_ = false;
 return batch_data->Reset(_->controller_,
@@ -2048,6 +2048,7 @@ Status 
KuduScanner::NextBatch(internal::ScanBatchDataInterface* batch_data) {
 
   if (data_->last_response_.has_more_results()) {
 // More data is available in this tablet.
+CHECK(data_->proxy_);
 VLOG(2) << "Continuing " << data_->DebugString();
 
 MonoTime batch_deadline = MonoTime::Now() + 
data_->configuration().timeout();
diff --git a/src/kudu/client/flex_partitioning_client-test.cc 
b/src/kudu/client/flex_partitioning_client-test.cc
index 35a10a54f..17ed4baa0 100644
--- a/src/kudu/client/flex_partitioning_client-test.cc
+++ b/src/kudu/client/flex_partitioning_client-test.cc
@@ -253,9 +253,9 @@ class FlexPartitioningTest : public KuduTest {
 }
 ASSERT_OK(scanner.Open());
 
-KuduScanBatch batch;
 int64_t count = 0;
 while (scanner.HasMoreRows()) {
+  KuduScanBatch batch;
   ASSERT_OK(scanner.NextBatch());
   count += batch.NumRows();
 }
@@ -300,7 +300,7 @@ class FlexPartitioningTest : public KuduTest {
 ASSERT_OK(scanner.AddConjunctPredicate(table->NewComparisonPredicate(
 col_name, KuduPredicate::GREATER_EQUAL, KuduValue::FromInt(lower;
 ASSERT_OK(scanner.AddConjunctPredicate(table->NewComparisonPredicate(
-col_name, KuduPredicate::LESS_EQUAL, KuduValue::FromInt(upper;
+col_name, KuduPredicate::LESS, KuduValue::FromInt(upper;
 
 ASSERT_OK(scanner.Open());
 KuduScanBatch batch;
@@ -444,19 +444,13 @@ TEST_F(FlexPartitioningCreateTableTest, 
DISABLED_SingleCustomRangeEmptyHashSchem
   // the partitions: first check the range of table-wide schema, then check
   // the ranges with custom hash schemas.
   ASSERT_OK(InsertTestRows(kTableName, -111, 0));
-  //NO_FATALS(CheckTableRowsNum(kTableName, 111));
+  NO_FATALS(CheckTableRowsNum(kTableName, 111));
   ASSERT_OK(InsertTestRows(kTableName, 111, 222));
-  // TODO(aserbin): uncomment the line below once PartitionPruner handles such
-  //cases properly
-  //NO_FATALS(CheckTableRowsNum(kTableName, 222));
+  NO_FATALS(CheckTableRowsNum(kTableName, 222));
 }
 
 // Create a table with mixed set of range partitions, using both table-wide and
 // custom hash bucket schemas.
-//
-// TODO(aserbin): add verification based on PartitionSchema provided by
-//KuduTable::partition_schema() once PartitionPruner
-//recognizes custom hash bucket schema per range

[kudu] branch master updated: [java] client update master locations cache

2022-07-06 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new a50aa5bfd [java] client update master locations cache
a50aa5bfd is described below

commit a50aa5bfdcebd9e842a357e47caa307e5db2bae0
Author: zhangyifan27 
AuthorDate: Fri Jul 1 20:31:24 2022 +0800

[java] client update master locations cache

Recently a master in our cluster was down because of network issues and 
somehow
the server didn't close the connections to some clients. Then these clients
keep trying to connect to the dead master but can't receive response until 
rpc
timeout, even when this server is up the client still send rpc through the 
old
channel and can't connect to the server and the new leader master. The only
solution is to restart clients.

This patch fixes the issue that java client can't invalidate stale 
locations of
the leader master. Maybe in this case we also need a better way to trigger
connection shutdown for an inactive channel.

Change-Id: Ia2877518866ac4c2d1dda6427ce57d08df48a864
Reviewed-on: http://gerrit.cloudera.org:8080/18703
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 .../kudu/client/ConnectToClusterResponse.java  |  7 ++
 .../apache/kudu/client/TableLocationsCache.java|  8 +--
 .../kudu/client/TestTableLocationsCache.java   | 27 ++
 src/kudu/master/master.proto   |  2 +-
 4 files changed, 41 insertions(+), 3 deletions(-)

diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java
 
b/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java
index 18421a664..9b965f718 100644
--- 
a/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java
+++ 
b/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java
@@ -34,6 +34,12 @@ class ConnectToClusterResponse {
   private static final ByteString FAKE_TABLET_ID = ByteString.copyFromUtf8(
   AsyncKuduClient.MASTER_TABLE_NAME_PLACEHOLDER);
 
+  /**
+   * If the client caches master locations, the entries should not live longer
+   * than this timeout. Defaults to one hour.
+   */
+  private static final int CACHE_TTL_MS = 60 * 60 * 1000;
+
   /** The host and port of the master that is currently leader */
   private final HostAndPort leaderHostAndPort;
   /** The response from that master */
@@ -73,6 +79,7 @@ class ConnectToClusterResponse {
 .addTsInfos(TSInfoPB.newBuilder()
 .addRpcAddresses(ProtobufHelper.hostAndPortToPB(leaderHostAndPort))
 .setPermanentUuid(ByteString.copyFromUtf8(fakeUuid)))
+.setTtlMillis(CACHE_TTL_MS)
 .build();
   }
 }
diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/TableLocationsCache.java
 
b/java/kudu-client/src/main/java/org/apache/kudu/client/TableLocationsCache.java
index a138dbd0a..788c1f997 100644
--- 
a/java/kudu-client/src/main/java/org/apache/kudu/client/TableLocationsCache.java
+++ 
b/java/kudu-client/src/main/java/org/apache/kudu/client/TableLocationsCache.java
@@ -62,7 +62,11 @@ class TableLocationsCache {
   rwl.readLock().lock();
   try {
 Preconditions.checkState(entries.size() <= 1);
-return entries.get(AsyncKuduClient.EMPTY_ARRAY);
+TableLocationsCache.Entry entry = 
entries.get(AsyncKuduClient.EMPTY_ARRAY);
+if (entry.isStale()) {
+  return null;
+}
+return entry;
   } finally {
 rwl.readLock().unlock();
   }
@@ -105,7 +109,7 @@ class TableLocationsCache {
 if (requestPartitionKey == null) {
   // Master lookup.
   Preconditions.checkArgument(tablets.size() == 1);
-  Entry entry = Entry.tablet(tablets.get(0), TimeUnit.DAYS.toMillis(1));
+  Entry entry = Entry.tablet(tablets.get(0), deadline);
 
   rwl.writeLock().lock();
   try {
diff --git 
a/java/kudu-client/src/test/java/org/apache/kudu/client/TestTableLocationsCache.java
 
b/java/kudu-client/src/test/java/org/apache/kudu/client/TestTableLocationsCache.java
index 0f0ce1a12..606952094 100644
--- 
a/java/kudu-client/src/test/java/org/apache/kudu/client/TestTableLocationsCache.java
+++ 
b/java/kudu-client/src/test/java/org/apache/kudu/client/TestTableLocationsCache.java
@@ -98,4 +98,31 @@ public class TestTableLocationsCache {
 cache.cacheTabletLocations(tablets, AsyncKuduClient.EMPTY_ARRAY, 1, 100);
 assertNotNull(cache.get(AsyncKuduClient.EMPTY_ARRAY));
   }
+
+  // Test for checking client will not get expired tablet locations.
+  @Test(timeout = 3)
+  public void testTTLForTableLocationsCache() {
+final int TTL_MS = 100;
+byte[] partitionKey = "start_key".getBytes(StandardCharsets.UTF_8);
+List tablets = Imm

[kudu] branch master updated: [Tools] Support to config hash bucket numbers when copy a table

2022-07-03 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 05903d962 [Tools] Support to config hash bucket numbers when copy a 
table
05903d962 is described below

commit 05903d96296f9d06423deaddcfbeb5f22461dc76
Author: xinghuayu007 <1450306...@qq.com>
AuthorDate: Thu Jun 9 11:09:16 2022 +0800

[Tools] Support to config hash bucket numbers when copy a table

When copying a table to another table, we can create the table
with the same schema. But we can not configure the number of hash bucket
in the new table. Why we need to config hash bucket number? The old table
might be configured with small number of hash buckets, but contained a lot
of data in it. When copying the table to new cluster, we want to add more
hash buckets to store. And there isn't a way to change the number of hash
buckets in the partition schema of an already existing table.

Change-Id: I1cec38e5ea09c66bfed20622b85033602da60d41
Reviewed-on: http://gerrit.cloudera.org:8080/18604
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 src/kudu/tools/kudu-tool-test.cc| 117 +---
 src/kudu/tools/table_scanner.cc |  46 +-
 src/kudu/tools/tool_action_table.cc |   1 +
 3 files changed, 155 insertions(+), 9 deletions(-)

diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 876a1854e..23601cedf 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -522,6 +522,7 @@ class ToolTest : public KuduTest {
 string columns;
 TableCopyMode mode;
 int32_t create_table_replication_factor;
+string create_table_hash_bucket_nums;
   };
 
   void RunCopyTableCheck(const RunCopyTableCheckArgs& args) {
@@ -569,9 +570,11 @@ class ToolTest : public KuduTest {
 
 // Execute copy command.
 string stdout;
-NO_FATALS(RunActionStdoutString(
+string stderr;
+Status s = RunActionStdoutStderrString(
 Substitute("table copy $0 $1 $2 -dst_table=$3 -predicates=$4 
-write_type=$5 "
-   "-create_table=$6 
-create_table_replication_factor=$7",
+   "-create_table=$6 
-create_table_replication_factor=$7 "
+   "-create_table_hash_bucket_nums=$8",
cluster_->master()->bound_rpc_addr().ToString(),
args.src_table_name,
cluster_->master()->bound_rpc_addr().ToString(),
@@ -579,8 +582,36 @@ class ToolTest : public KuduTest {
args.predicates_json,
write_type,
create_table,
-   args.create_table_replication_factor),
-));
+   args.create_table_replication_factor,
+   args.create_table_hash_bucket_nums),
+, );
+if (args.create_table_hash_bucket_nums == "10,aa") {
+  ASSERT_STR_CONTAINS(stderr, "cannot parse the number of hash buckets.");
+  return;
+}
+if (args.create_table_hash_bucket_nums == "10,20,30") {
+  ASSERT_STR_CONTAINS(stderr, "The count of hash bucket numbers must be 
equal to the "
+  "number of hash schema dimensions.");
+  return;
+}
+if (args.create_table_hash_bucket_nums == "10") {
+  ASSERT_STR_CONTAINS(stderr, "The count of hash bucket numbers must be 
equal to the "
+  "number of hash schema dimensions.");
+  return;
+}
+if (args.create_table_hash_bucket_nums == "10,1") {
+  ASSERT_STR_CONTAINS(stderr, "The number of hash buckets must not be less 
than 2.");
+  return;
+}
+if (args.create_table_hash_bucket_nums == "10,1") {
+  ASSERT_STR_CONTAINS(stderr, "The number of hash buckets must not be less 
than 2.");
+  return;
+}
+if (args.create_table_hash_bucket_nums == "10,50") {
+  ASSERT_STR_CONTAINS(stderr, "There are no hash partitions defined in 
this table.");
+  return;
+}
+ASSERT_TRUE(s.ok());
 
 // Check total count.
 int64_t total = max(args.max_value - args.min_value + 1, 0);
@@ -612,6 +643,32 @@ class ToolTest : public KuduTest {
 // Replication factor is different when explicitly set it to 3 
(default 1).
 if (args.create_table_replication_factor == 3 &&
 HasPrefixString(src_schema[i], "REPLICAS ")) continue;
+vector hash_bucket_nums = 
Split(args.create_table_hash_bucket_nums,
+   

[kudu] branch master updated: KUDU-2671: Follow up pruning patch.

2022-07-03 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new b746978c7 KUDU-2671: Follow up pruning patch.
b746978c7 is described below

commit b746978c71ce4a95b69d49c43d0ac852909a8b4e
Author: Mahesh Reddy 
AuthorDate: Fri Apr 1 01:37:52 2022 -0700

KUDU-2671: Follow up pruning patch.

This patch flattens the result set of the pruner into a one
dimensional container. The new container only contains the
partition key ranges and no longer stores the range bounds.

Currently, full scans using KuduScanner with no predicates are
functional. Scans with range predicates are also functional on
tables with covering ranges as well as on tables with non
covering ranges.

There are a few commented out test cases within
flex_partitioning_client-test. These test cases involve a scan
with range predicates that are both out of bounds. They fail
because the non covering range case is triggered in
scanner_internal and we return early from this function before
the proxy_ is set. Check(proxy_) is where the tests fails in
KuduScanner::NextBatch within client.cc.

Using KuduScanTokens to scan tables with range specific hash schemas
is not yet compatible. A follow up patch should address this deficiency.

The scan token tests with custom hash schemas are failing when
verifying the tablet info. It seems that the data_ field of the
KuduTablets isn't set.

Change-Id: I3a1bf5344c0ef856072d3ed102714dce5ba21060
Reviewed-on: http://gerrit.cloudera.org:8080/17879
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 src/kudu/client/client.h |   5 +
 src/kudu/client/flex_partitioning_client-test.cc | 356 +--
 src/kudu/client/scan_token-internal.cc   |   1 -
 src/kudu/client/scan_token-test.cc   | 203 -
 src/kudu/client/scanner-internal.cc  |   2 +-
 src/kudu/common/partition_pruner-test.cc |   1 -
 src/kudu/common/partition_pruner.cc  | 120 
 src/kudu/common/partition_pruner.h   |  16 +-
 8 files changed, 582 insertions(+), 122 deletions(-)

diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 296a0daf7..418dfe261 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -3078,6 +3078,7 @@ class KUDU_EXPORT KuduScanner {
   Status NextBatch(internal::ScanBatchDataInterface* batch);
 
   friend class KuduScanToken;
+  friend class FlexPartitioningTest;
   FRIEND_TEST(ClientTest, TestBlockScannerHijackingAttempts);
   FRIEND_TEST(ClientTest, TestScanCloseProxy);
   FRIEND_TEST(ClientTest, TestScanFaultTolerance);
@@ -3113,6 +3114,10 @@ class KUDU_EXPORT KuduScanner {
 ///
 /// Scan token locality information can be inspected using the
 /// KuduScanToken::tablet() function.
+///
+/// Scan tokens are not yet compatible for tables that contain range-specific
+/// hash schemas. To be clear, the existing use case of tables with all ranges
+/// using the table wide hash schema is functional as expected.
 class KUDU_EXPORT KuduScanToken {
  public:
 
diff --git a/src/kudu/client/flex_partitioning_client-test.cc 
b/src/kudu/client/flex_partitioning_client-test.cc
index 97c4a5375..04013900c 100644
--- a/src/kudu/client/flex_partitioning_client-test.cc
+++ b/src/kudu/client/flex_partitioning_client-test.cc
@@ -17,7 +17,6 @@
 
 #include 
 #include 
-#include 
 #include 
 #include 
 #include 
@@ -43,10 +42,13 @@
 #include "kudu/master/mini_master.h"
 #include "kudu/mini-cluster/internal_mini_cluster.h"
 #include "kudu/tablet/tablet_replica.h"
+#include "kudu/tablet/tablet.h"
 #include "kudu/tserver/mini_tablet_server.h"
 #include "kudu/tserver/tablet_server.h"
 #include "kudu/tserver/ts_tablet_manager.h"
+#include "kudu/util/metrics.h"
 #include "kudu/util/net/sockaddr.h"
+#include "kudu/util/slice.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
@@ -54,13 +56,13 @@
 DECLARE_bool(enable_per_range_hash_schemas);
 DECLARE_int32(heartbeat_interval_ms);
 
+METRIC_DECLARE_counter(scans_started);
+
 using kudu::client::sp::shared_ptr;
 using kudu::client::KuduValue;
 using kudu::cluster::InternalMiniCluster;
 using kudu::cluster::InternalMiniClusterOptions;
 using kudu::master::CatalogManager;
-using kudu::master::TabletInfo;
-using kudu::tablet::TabletReplica;
 using std::string;
 using std::unique_ptr;
 using std::vector;
@@ -234,12 +236,38 @@ class FlexPartitioningTest : public KuduTest {
 ASSERT_EQ(expected_count, count);
   }
 
-  void CheckLiveRowCount(const char* table_name,

[kudu] branch master updated: KUDU-2671 fix handling hash dimensions in ADD_RANGE_PARTITION

2022-07-02 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 75045fa10 KUDU-2671 fix handling hash dimensions in ADD_RANGE_PARTITION
75045fa10 is described below

commit 75045fa10619c5674ca9b6405332be97e43bfeba
Author: Alexey Serbin 
AuthorDate: Sat Jul 2 13:11:58 2022 -0700

KUDU-2671 fix handling hash dimensions in ADD_RANGE_PARTITION

This patch fixes a bug in handling hash dimensions when storing the
information on HashSchema for a newly added range in the system catalog.
I modfied one already existing test scenario in MasterTest to spot
regressions, verifying that before the fix the updated scenario
would fail as expected.

This is a follow-up to 6909ee4f800da192b72e59680916e5004527b6db.

Change-Id: If1fed52f9abd02a8aa2bc85f2692252d16965621
Reviewed-on: http://gerrit.cloudera.org:8080/18695
Reviewed-by: Attila Bukor 
Tested-by: Alexey Serbin 
---
 src/kudu/master/catalog_manager.cc |  3 +--
 src/kudu/master/master-test.cc | 42 ++
 2 files changed, 39 insertions(+), 6 deletions(-)

diff --git a/src/kudu/master/catalog_manager.cc 
b/src/kudu/master/catalog_manager.cc
index da017d828..78fc4ec75 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -2705,9 +2705,8 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
   auto* hash_dimension_pb = range->add_hash_schema();
   hash_dimension_pb->set_num_buckets(hash_dimension.num_buckets);
   hash_dimension_pb->set_seed(hash_dimension.seed);
-  auto* columns = hash_dimension_pb->add_columns();
   for (const auto& column_id : hash_dimension.column_ids) {
-columns->set_id(column_id);
+hash_dimension_pb->add_columns()->set_id(column_id);
   }
 }
 ++partition_schema_updates;
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index ff91b65db..03a9f7085 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -1006,7 +1006,7 @@ TEST_F(MasterTest, 
AlterTableAddAndDropRangeWithSpecificHashSchema) {
   constexpr const char* const kCol0 = "c_int32";
   constexpr const char* const kCol1 = "c_int64";
   const Schema kTableSchema({ColumnSchema(kCol0, INT32),
- ColumnSchema(kCol1, INT64)}, 1);
+ ColumnSchema(kCol1, INT64)}, 2);
   FLAGS_enable_per_range_hash_schemas = true;
   FLAGS_default_num_replicas = 1;
 
@@ -1040,7 +1040,7 @@ TEST_F(MasterTest, 
AlterTableAddAndDropRangeWithSpecificHashSchema) {
   }
 
   const auto& table_id = create_table_resp.table_id();
-  const HashSchema custom_hash_schema{{{kCol0}, 5, 1}};
+  const HashSchema custom_hash_schema{{{kCol0,kCol1}, 5, 1}};
 
   // Alter the table, adding a new range with custom hash schema.
   {
@@ -1060,6 +1060,7 @@ TEST_F(MasterTest, 
AlterTableAddAndDropRangeWithSpecificHashSchema) {
   ColumnSchemaPB* col1 = req.mutable_schema()->add_columns();
   col1->set_name(kCol1);
   col1->set_type(INT64);
+  col0->set_is_key(true);
 }
 
 AlterTableRequestPB::Step* step = req.add_alter_schema_steps();
@@ -1115,6 +1116,28 @@ TEST_F(MasterTest, 
AlterTableAddAndDropRangeWithSpecificHashSchema) {
   PartitionSchemaPB ps_pb;
   ASSERT_OK(partition_schema_retriever(_pb));
   ASSERT_EQ(1, ps_pb.custom_hash_schema_ranges_size());
+
+  // Check the hash schema parameters (i.e. columns and number of hash
+  // buckets) are stored and read back by the client as expected.
+  const auto& range = ps_pb.custom_hash_schema_ranges(0);
+  ASSERT_EQ(1, range.hash_schema_size());
+  const auto& hash_schema = range.hash_schema(0);
+
+  ASSERT_EQ(5, hash_schema.num_buckets());
+  ASSERT_EQ(1, hash_schema.seed());
+
+  ASSERT_EQ(2, hash_schema.columns_size());
+  const auto schema = kTableSchema.CopyWithColumnIds();
+
+  const auto ref_col_0_id = int32_t(schema.column_id(0));
+  const auto& col_0 = hash_schema.columns(0);
+  ASSERT_TRUE(col_0.has_id());
+  ASSERT_EQ(ref_col_0_id, col_0.id());
+
+  const auto ref_col_1_id = int32_t(schema.column_id(1));
+  const auto& col_1 = hash_schema.columns(1);
+  ASSERT_TRUE(col_1.has_id());
+  ASSERT_EQ(ref_col_1_id, col_1.id());
 }
   }
 
@@ -1148,6 +1171,8 @@ TEST_F(MasterTest, 
AlterTableAddAndDropRangeWithSpecificHashSchema) {
 resp.partition_schema(), received_schema, ));
 ASSERT_TRUE(ps.HasCustomHashSchemas());
 
+// Verify that PartitionSchema::FromPB() translated/retrieved the data
+// from PartitionSchemaPB as expected.
 const auto& table_wide_hash_schema = ps.

[kudu] branch master updated: KUDU-2671 mark --enable_per_range_hash_schemas as 'advanced'

2022-07-02 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 357dd89f7 KUDU-2671 mark --enable_per_range_hash_schemas as 'advanced'
357dd89f7 is described below

commit 357dd89f7acc645456eea1c46d233f94aaa4e150
Author: Alexey Serbin 
AuthorDate: Sat Jul 2 10:40:03 2022 -0700

KUDU-2671 mark --enable_per_range_hash_schemas as 'advanced'

Since the functionality for KUDU-2671 is almost there, this patch
removes the 'unsafe' tag from the newly introduced
--enable_per_range_hash_schemas flag which enables support for
range-specific hash schemas.  Now the flag is marked as 'advanced'
instead and also marked as 'runtime' as well.

Change-Id: Iecfebf908745d279bb7a276806c7c96b363ba8db
Reviewed-on: http://gerrit.cloudera.org:8080/18694
Reviewed-by: Attila Bukor 
Tested-by: Alexey Serbin 
---
 src/kudu/master/catalog_manager.cc | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/src/kudu/master/catalog_manager.cc 
b/src/kudu/master/catalog_manager.cc
index 2a0d88200..da017d828 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -355,7 +355,8 @@ TAG_FLAG(table_locations_cache_capacity_mb, advanced);
 
 DEFINE_bool(enable_per_range_hash_schemas, false,
 "Whether the ability to specify different hash schemas per range 
is enabled");
-TAG_FLAG(enable_per_range_hash_schemas, unsafe);
+TAG_FLAG(enable_per_range_hash_schemas, advanced);
+TAG_FLAG(enable_per_range_hash_schemas, runtime);
 
 DEFINE_bool(enable_table_write_limit, false,
 "Enable the table write limit. "



[kudu] branch master updated: [java] KUDU-2671 require RANGE_SPECIFIC_HASH_SCHEMA when necessary

2022-07-01 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new e696b8a1a [java] KUDU-2671 require RANGE_SPECIFIC_HASH_SCHEMA when 
necessary
e696b8a1a is described below

commit e696b8a1a4936d5703a9d5d26469ea997a0b2d90
Author: Alexey Serbin 
AuthorDate: Thu Jun 30 20:15:41 2022 -0700

[java] KUDU-2671 require RANGE_SPECIFIC_HASH_SCHEMA when necessary

With this patch, Kudu Java client requires the newly introduced feature
flag RANGE_SPECIFIC_HASH_SCHEMA to be present at the server side when
creating a table with range-specific hash schemas.  The flag is also
required when altering already existing table, adding a new range
partition with custom hash schema.

This patch also provides test coverage for the newly introduced
functionality.

This is a follow-up to https://gerrit.cloudera.org/#/c/18633

Change-Id: I7e6625277cc2fb63000999f0c74d9b3d929d5657
Reviewed-on: http://gerrit.cloudera.org:8080/18687
Reviewed-by: Attila Bukor 
Tested-by: Alexey Serbin 
---
 .../org/apache/kudu/client/AlterTableOptions.java  | 19 +++-
 .../org/apache/kudu/client/AlterTableRequest.java  |  8 ++--
 .../org/apache/kudu/client/CreateTableOptions.java |  7 ++-
 .../org/apache/kudu/client/CreateTableRequest.java |  6 +--
 .../org/apache/kudu/client/TestAlterTable.java | 51 ++
 .../java/org/apache/kudu/client/TestKuduTable.java | 40 +
 6 files changed, 121 insertions(+), 10 deletions(-)

diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
index a2e4f6476..1603d0801 100644
--- 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
+++ 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
@@ -22,9 +22,11 @@ import static org.apache.kudu.ColumnSchema.Encoding;
 import static org.apache.kudu.master.Master.AlterTableRequestPB;
 
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Map;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 import com.google.protobuf.ByteString;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
@@ -33,6 +35,7 @@ import org.apache.kudu.ColumnSchema;
 import org.apache.kudu.Common;
 import org.apache.kudu.Type;
 import org.apache.kudu.client.ProtobufHelper.SchemaPBConversionFlags;
+import org.apache.kudu.master.Master;
 
 /**
  * This builder must be used to alter a table. At least one change must be 
specified.
@@ -42,6 +45,7 @@ import 
org.apache.kudu.client.ProtobufHelper.SchemaPBConversionFlags;
 public class AlterTableOptions {
   private final AlterTableRequestPB.Builder pb = 
AlterTableRequestPB.newBuilder();
   private boolean wait = true;
+  private boolean isAddingRangeWithCustomHashSchema = false;
 
   /**
* Change a table's name.
@@ -393,6 +397,7 @@ public class AlterTableOptions {
   EnumSet.of(SchemaPBConversionFlags.SCHEMA_PB_WITHOUT_COMMENT,
   SchemaPBConversionFlags.SCHEMA_PB_WITHOUT_ID)));
 }
+isAddingRangeWithCustomHashSchema = true;
 return this;
   }
 
@@ -529,4 +534,16 @@ public class AlterTableOptions {
   boolean shouldWait() {
 return wait;
   }
-}
+
+  List getRequiredFeatureFlags() {
+if (!hasAddDropRangePartitions()) {
+  return ImmutableList.of();
+}
+if (!isAddingRangeWithCustomHashSchema) {
+  return 
ImmutableList.of(Master.MasterFeatures.RANGE_PARTITION_BOUNDS_VALUE);
+}
+return ImmutableList.of(
+Master.MasterFeatures.RANGE_PARTITION_BOUNDS_VALUE,
+Master.MasterFeatures.RANGE_SPECIFIC_HASH_SCHEMA_VALUE);
+  }
+}
\ No newline at end of file
diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableRequest.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableRequest.java
index c323c95e1..a27e01048 100644
--- 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableRequest.java
+++ 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableRequest.java
@@ -42,7 +42,7 @@ class AlterTableRequest extends KuduRpc {
   static final String ALTER_TABLE = "AlterTable";
   private final String name;
   private final AlterTableRequestPB.Builder builder;
-  private final List requiredFeatures;
+  private final List featureFlags;
 
   AlterTableRequest(KuduTable masterTable,
 String name,
@@ -52,9 +52,7 @@ class AlterTableRequest extends KuduRpc {
 super(masterTable, timer, timeoutMillis);
 this.name = name;
 this.builder = ato.getProtobuf();
-this.requiredFeatures = ato.hasAddDropRangePartitions() ?
-Immuta

[kudu] branch master updated: [java] KUDU-2671 support adding a range with custom hash schema

2022-07-01 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 248d9f8ce [java] KUDU-2671 support adding a range with custom hash 
schema
248d9f8ce is described below

commit 248d9f8cecd947d700f9e1135db17a30ea1e194a
Author: Alexey Serbin 
AuthorDate: Thu Jun 2 17:40:30 2022 -0700

[java] KUDU-2671 support adding a range with custom hash schema

With this patch, Kudu Java client now supports adding a range with
custom hash schema for a table.  The patch also contains test cases
to cover the new functionality.

This is a patch to complement [1] at the Kudu Java client side
([1] introduced corresponding changes at the Kudu C++ client).

[1] https://gerrit.cloudera.org/#/c/18663/

Change-Id: Ieaab7a79d4336de7ff6ec84b8c1806407e4fa44e
Reviewed-on: http://gerrit.cloudera.org:8080/18589
Tested-by: Alexey Serbin 
Reviewed-by: Mahesh Reddy 
Reviewed-by: Attila Bukor 
---
 .../org/apache/kudu/client/AlterTableOptions.java  |  33 +++
 .../org/apache/kudu/client/TestAlterTable.java | 262 +++
 .../java/org/apache/kudu/client/TestKuduTable.java | 279 -
 3 files changed, 567 insertions(+), 7 deletions(-)

diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
index 2f01d5798..a2e4f6476 100644
--- 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
+++ 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
@@ -363,6 +363,39 @@ public class AlterTableOptions {
 return this;
   }
 
+  /**
+   * Similar to the other addRangePartition() methods, but instead of adding a
+   * range with table-wide hash schema, this method adds a range with
+   * custom hash schema.
+   *
+   * @param range the range with custom hash schema
+   * @return this instance
+   */
+  public AlterTableOptions 
addRangePartition(RangePartitionWithCustomHashSchema range) {
+Preconditions.checkNotNull(range);
+AlterTableRequestPB.Step.Builder step = pb.addAlterSchemaStepsBuilder();
+step.setType(AlterTableRequestPB.StepType.ADD_RANGE_PARTITION);
+AlterTableRequestPB.AddRangePartition.Builder rangeBuilder =
+AlterTableRequestPB.AddRangePartition.newBuilder();
+rangeBuilder.setRangeBounds(
+new Operation.OperationsEncoder().encodeLowerAndUpperBounds(
+range.getLowerBound(), range.getUpperBound(),
+range.getLowerBoundType(), range.getUpperBoundType()));
+for (org.apache.kudu.Common.PartitionSchemaPB.HashBucketSchemaPB 
hashSchema :
+range.toPB().getHashSchemaList()) {
+  Common.PartitionSchemaPB.HashBucketSchemaPB.Builder hbs =
+  rangeBuilder.addCustomHashSchemaBuilder();
+  hbs.mergeFrom(hashSchema);
+}
+step.setAddRangePartition(rangeBuilder);
+if (!pb.hasSchema()) {
+  pb.setSchema(ProtobufHelper.schemaToPb(range.getLowerBound().getSchema(),
+  EnumSet.of(SchemaPBConversionFlags.SCHEMA_PB_WITHOUT_COMMENT,
+  SchemaPBConversionFlags.SCHEMA_PB_WITHOUT_ID)));
+}
+return this;
+  }
+
   /**
* Drop the range partition from the table with the specified inclusive 
lower bound and exclusive
* upper bound. The bounds must match exactly, and may not span multiple 
range partitions.
diff --git 
a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java 
b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
index 377428437..79ddc39c4 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
@@ -512,6 +512,268 @@ public class TestAlterTable {
 assertEquals(100, countRowsInTable(table));
   }
 
+  /**
+   * Test altering a table, adding range partitions with custom hash schema
+   * per range.
+   */
+  @Test(timeout = 10)
+  @KuduTestHarness.MasterServerConfig(flags = {
+  "--enable_per_range_hash_schemas=true",
+  })
+  public void testAlterAddRangeWithCustomHashSchema() throws Exception {
+ArrayList columns = new ArrayList<>(2);
+columns.add(new ColumnSchema.ColumnSchemaBuilder("c0", Type.INT32)
+.nullable(false)
+.key(true)
+.build());
+columns.add(new ColumnSchema.ColumnSchemaBuilder("c1", Type.INT32)
+.nullable(false)
+.build());
+final Schema schema = new Schema(columns);
+
+CreateTableOptions createOptions =
+new CreateTableOptions()
+.setRangePartitionColumns(ImmutableList.of("c0"))
+.addHashPartitions(ImmutableList.of("c0"), 2, 0)
+.setNumReplicas(1

[kudu] branch master updated: [java] KUDU-2671 support creating table with custom hash schema per range

2022-07-01 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 8b31a0ed4 [java] KUDU-2671 support creating table with custom hash 
schema per range
8b31a0ed4 is described below

commit 8b31a0ed48a8ee830c68dc74143c924f4b1f2116
Author: Alexey Serbin 
AuthorDate: Thu Dec 2 21:07:44 2021 -0800

[java] KUDU-2671 support creating table with custom hash schema per range

This patch introduces changes in the Kudu Java client API to make it
possible to create a Kudu table with custom hash bucket schemas per
range partition. Corresponding test coverage is present as well.

This is a patch to complement 586b79132 at the Kudu Java client side
(586b79132 introduced corresponding changes at the Kudu C++ client).

The appropriate Spark bindings haven't been updated yet -- that
to be done in a separate changelist.

Change-Id: I5ccf77ea2c39808520e76351d62571d449d10894
Reviewed-on: http://gerrit.cloudera.org:8080/18562
Tested-by: Alexey Serbin 
Reviewed-by: Zoltan Chovan 
Reviewed-by: Attila Bukor 
---
 .../org/apache/kudu/client/AsyncKuduClient.java|   4 +-
 .../org/apache/kudu/client/CreateTableOptions.java |  99 ++-
 .../java/org/apache/kudu/client/KeyEncoder.java|  21 +-
 .../java/org/apache/kudu/client/Operation.java |  24 +-
 .../org/apache/kudu/client/PartitionPruner.java| 312 +--
 .../org/apache/kudu/client/PartitionSchema.java| 163 +++-
 .../org/apache/kudu/client/ProtobufHelper.java |  53 +-
 .../org/apache/kudu/client/RangePartition.java |  66 ++
 .../client/RangePartitionWithCustomHashSchema.java |  87 ++
 .../org/apache/kudu/client/TestKeyEncoding.java|  59 +-
 .../org/apache/kudu/client/TestKuduClient.java | 103 +++
 .../java/org/apache/kudu/client/TestKuduTable.java | 902 +
 .../java/org/apache/kudu/client/TestOperation.java |  26 +-
 .../apache/kudu/client/TestPartitionPruner.java| 219 +
 14 files changed, 1969 insertions(+), 169 deletions(-)

diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
index 3a0780c14..1b1a690ca 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
@@ -633,8 +633,8 @@ public class AsyncKuduClient implements AutoCloseable {
 if (builder == null) {
   throw new IllegalArgumentException("CreateTableOptions may not be null");
 }
-if (!builder.getBuilder().getPartitionSchema().hasRangeSchema() &&
-builder.getBuilder().getPartitionSchema().getHashSchemaCount() == 0) {
+final Common.PartitionSchemaPB ps = 
builder.getBuilder().getPartitionSchema();
+if (!ps.hasRangeSchema() && ps.getHashSchemaCount() == 0) {
   throw new IllegalArgumentException("Table partitioning must be specified 
using " +
  "setRangePartitionColumns or 
addHashPartitions");
 
diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/CreateTableOptions.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/CreateTableOptions.java
index 6ac83dd62..e1d35360c 100644
--- 
a/java/kudu-client/src/main/java/org/apache/kudu/client/CreateTableOptions.java
+++ 
b/java/kudu-client/src/main/java/org/apache/kudu/client/CreateTableOptions.java
@@ -38,8 +38,11 @@ public class CreateTableOptions {
 
   private final List splitRows = Lists.newArrayList();
   private final List rangePartitions = Lists.newArrayList();
+  private final List customRangePartitions 
=
+  Lists.newArrayList(); // range partitions with custom hash schemas
   private Master.CreateTableRequestPB.Builder pb = 
Master.CreateTableRequestPB.newBuilder();
   private boolean wait = true;
+  private boolean isPbGenerationDone = false;
 
   /**
* Add a set of hash partitions to the table.
@@ -164,6 +167,23 @@ public class CreateTableOptions {
 return this;
   }
 
+  /**
+   * Add range partition with custom hash schema.
+   *
+   * @param rangePartition range partition with custom hash schema
+   * @return this CreateTableOptions object modified accordingly
+   */
+  public CreateTableOptions 
addRangePartition(RangePartitionWithCustomHashSchema rangePartition) {
+if (!splitRows.isEmpty()) {
+  throw new IllegalArgumentException(
+  "no range partitions with custom hash schema are allowed when using 
" +
+  "split rows to define range partitioning for a table");
+}
+customRangePartitions.add(rangePartition);
+
pb.getPartitionSchemaBuilder().addCustomHashSchemaRanges(rangePartition.toPB());
+return this;
+  }
+

[kudu] branch master updated: KUDU-2671 introduce RANGE_SPECIFIC_HASH_SCHEMA feature flag

2022-07-01 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 76f475aba KUDU-2671 introduce RANGE_SPECIFIC_HASH_SCHEMA feature flag
76f475aba is described below

commit 76f475abad0194464fd2e46383be7467f50aedd3
Author: Alexey Serbin 
AuthorDate: Thu Jun 16 19:15:43 2022 -0700

KUDU-2671 introduce RANGE_SPECIFIC_HASH_SCHEMA feature flag

This patch introduces a new RANGE_SPECIFIC_HASH_SCHEMA flag for master
to signal that a Kudu cluster is able to work with tables having
range-specific hash schemas (a.k.a. custom hash schemas per range).

In addition, now C++ client requires the new flag to be present at
the server side when creating a table having at least one range
partition with custom hash schema or when adding a new range partition
with custom hash schema.

The rationale for introducing the flag is the following: if there were
no RANGE_SPECIFIC_HASH_SCHEMA flag and a newer client were not requiring
the server to have such a flag, the client would not get an error while
trying to perform the following operations against tablet servers
of prior versions:
  * Creating a table having a range partition with custom hash schema
  * Adding a range partition with custom hash schema to existing table
That's because the information on custom hash schemas is provided via
newly added flags in corresponding protobuf structures, and the old
server would simply ignore the fields, assuming all the ranges to be
created have the table-wide hash schema.

A follow-up patch will add similar functionality for Kudu Java client.

Change-Id: I256d32003e869939e7aa581b21bbe1e77c1e3aba
Reviewed-on: http://gerrit.cloudera.org:8080/18633
Reviewed-by: Mahesh Reddy 
Reviewed-by: Abhishek Chennaka 
Tested-by: Alexey Serbin 
Reviewed-by: Attila Bukor 
---
 src/kudu/client/client-internal.cc   |  12 +-
 src/kudu/client/client-internal.h|   6 +-
 src/kudu/client/client.cc|  21 +++-
 src/kudu/client/flex_partitioning_client-test.cc | 138 +++
 src/kudu/client/table_alterer-internal.h |   3 +
 src/kudu/master/master.proto |   2 +
 src/kudu/master/master_service.cc|   3 +
 7 files changed, 175 insertions(+), 10 deletions(-)

diff --git a/src/kudu/client/client-internal.cc 
b/src/kudu/client/client-internal.cc
index 99934e28f..5c650599f 100644
--- a/src/kudu/client/client-internal.cc
+++ b/src/kudu/client/client-internal.cc
@@ -316,11 +316,15 @@ Status KuduClient::Data::CreateTable(KuduClient* client,
  const CreateTableRequestPB& req,
  CreateTableResponsePB* resp,
  const MonoTime& deadline,
- bool has_range_partition_bounds) {
+ bool has_range_partition_bounds,
+ bool has_range_specific_hash_schema) {
   vector features;
   if (has_range_partition_bounds) {
 features.push_back(MasterFeatures::RANGE_PARTITION_BOUNDS);
   }
+  if (has_range_specific_hash_schema) {
+features.push_back(MasterFeatures::RANGE_SPECIFIC_HASH_SCHEMA);
+  }
   Synchronizer sync;
   AsyncLeaderMasterRpc rpc(
   deadline, client, BackoffType::EXPONENTIAL, req, resp,
@@ -387,11 +391,15 @@ Status KuduClient::Data::AlterTable(KuduClient* client,
 const AlterTableRequestPB& req,
 AlterTableResponsePB* resp,
 const MonoTime& deadline,
-bool has_add_drop_partition) {
+bool has_add_drop_partition,
+bool adding_range_with_custom_hash_schema) 
{
   vector required_feature_flags;
   if (has_add_drop_partition) {
 
required_feature_flags.push_back(MasterFeatures::ADD_DROP_RANGE_PARTITIONS);
   }
+  if (adding_range_with_custom_hash_schema) {
+
required_feature_flags.push_back(MasterFeatures::RANGE_SPECIFIC_HASH_SCHEMA);
+  }
   Synchronizer sync;
   AsyncLeaderMasterRpc rpc(
   deadline, client, BackoffType::EXPONENTIAL, req, resp,
diff --git a/src/kudu/client/client-internal.h 
b/src/kudu/client/client-internal.h
index deab47d42..61661f73b 100644
--- a/src/kudu/client/client-internal.h
+++ b/src/kudu/client/client-internal.h
@@ -105,7 +105,8 @@ class KuduClient::Data {
 const master::CreateTableRequestPB& req,
 master::CreateTableResponsePB* resp,
 const MonoTime& deadline,
-

[kudu] branch master updated: [c++ client] KUDU-2671 Custom hash schema alter table support

2022-06-30 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 1889d4c44 [c++ client] KUDU-2671  Custom hash schema alter table 
support
1889d4c44 is described below

commit 1889d4c44385fec5efeeb2d287d9ab7a3544dcfe
Author: Abhishek Chennaka 
AuthorDate: Wed Jun 22 11:50:09 2022 -0400

[c++ client] KUDU-2671  Custom hash schema alter table support

This patch adds public methods to C++ client to alter a table and add
a new range partition with custom hash sub-partitioning. We make use
of the KuduTableCreator::KuduRangePartition() for this purpose. The
necessary changes are done in table_alterer-internal classes and
methods to use the above mentioned KuduRangePartition() to store
the table bounds information as well as custom hash schema
information.

Necessary tests are included in this patch which include adding and
dropping the ranges with custom hash schema by altering the table.
We also read and write the data into these partitions.

The pending work in this patch is to rebase on top of
https://gerrit.cloudera.org/#/c/17879/ and include test cases with
scans with predicates on these partitions.

Change-Id: Id4b1e306cca096d9479f06669cc22cc40d77fb42
Reviewed-on: http://gerrit.cloudera.org:8080/18663
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 src/kudu/client/client.cc|  47 --
 src/kudu/client/client.h |  22 +++
 src/kudu/client/flex_partitioning_client-test.cc | 202 +++
 src/kudu/client/table_alterer-internal.cc|  41 +++--
 src/kudu/client/table_alterer-internal.h |   8 +-
 5 files changed, 290 insertions(+), 30 deletions(-)

diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 281d1d9aa..1f3b86c56 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -1485,7 +1485,7 @@ KuduTableAlterer* KuduTableAlterer::SetComment(const 
string& new_comment) {
 
 KuduColumnSpec* KuduTableAlterer::AddColumn(const string& name) {
   Data::Step s = { AlterTableRequestPB::ADD_COLUMN,
-   new KuduColumnSpec(name), nullptr, nullptr };
+   new KuduColumnSpec(name), nullptr };
   auto* spec = s.spec;
   data_->steps_.emplace_back(std::move(s));
   return spec;
@@ -1493,7 +1493,7 @@ KuduColumnSpec* KuduTableAlterer::AddColumn(const string& 
name) {
 
 KuduColumnSpec* KuduTableAlterer::AlterColumn(const string& name) {
   Data::Step s = { AlterTableRequestPB::ALTER_COLUMN,
-   new KuduColumnSpec(name), nullptr, nullptr };
+   new KuduColumnSpec(name), nullptr };
   auto* spec = s.spec;
   data_->steps_.emplace_back(std::move(s));
   return spec;
@@ -1501,7 +1501,7 @@ KuduColumnSpec* KuduTableAlterer::AlterColumn(const 
string& name) {
 
 KuduTableAlterer* KuduTableAlterer::DropColumn(const string& name) {
   Data::Step s = { AlterTableRequestPB::DROP_COLUMN,
-   new KuduColumnSpec(name), nullptr, nullptr };
+   new KuduColumnSpec(name), nullptr };
   data_->steps_.emplace_back(std::move(s));
   return this;
 }
@@ -1539,16 +1539,42 @@ KuduTableAlterer* 
KuduTableAlterer::AddRangePartitionWithDimension(
 
   Data::Step s { AlterTableRequestPB::ADD_RANGE_PARTITION,
  nullptr,
- unique_ptr(lower_bound),
- unique_ptr(upper_bound),
- lower_bound_type,
- upper_bound_type,
+ std::unique_ptr(
+ new KuduTableCreator::KuduRangePartition(
+ lower_bound, upper_bound, lower_bound_type, 
upper_bound_type)),
  dimension_label.empty() ? nullopt : 
make_optional(dimension_label) };
   data_->steps_.emplace_back(std::move(s));
   data_->has_alter_partitioning_steps = true;
   return this;
 }
 
+KuduTableAlterer* KuduTableAlterer::AddRangePartition(
+KuduTableCreator::KuduRangePartition* partition) {
+  CHECK(partition);
+  if (partition->data_->lower_bound_ == nullptr || 
partition->data_->upper_bound_  == nullptr) {
+data_->status_ = Status::InvalidArgument("range partition bounds may not 
be null");
+return this;
+  }
+  if (partition->data_->lower_bound_->schema() != 
partition->data_->upper_bound_->schema()) {
+data_->status_ = Status::InvalidArgument("range partition bounds must have 
matching schemas");
+return this;
+  }
+  if (data_->schema_ == nullptr) {
+data_->schema_ = partition->data_->lower_bound_->schema();
+  } else if (partition->data_->lower_bound_->schema() != data_->schema_) {
+data_->status_ = Status::InvalidA

[kudu] branch master updated: KUDU-2671 introduce PartitionPruner::PrepareRangeSet()

2022-06-29 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 3c415ea2f KUDU-2671 introduce PartitionPruner::PrepareRangeSet()
3c415ea2f is described below

commit 3c415ea2f4418d60d94ff870154b3128100a7d77
Author: Alexey Serbin 
AuthorDate: Sat Jun 25 09:19:26 2022 -0700

KUDU-2671 introduce PartitionPruner::PrepareRangeSet()

Since PartitionSchema now provides and persists information only on
ranges with custom hash schemas [1], but PartitionPruner::Init()'s
logic assumed receiving information on all the existing ranges in case
of tables with range-specific hash schemas, it needed an update.

This patch does exactly so, adding a new PrepareRangeSet() method into
the PartitionPruner class.  The new method produces the preliminary
set of scanner ranges with proper hash schemas per each range using the
information on the table-wide hash schema and range-specific hash
schemas provided.  It splits the predicate-based range into sub-ranges
and assigns corresponding hash schemas to them.  In essence, the hash
schemas for the ranges with custom hash schemas are known, and the rest
of the sub-ranges have the table-wide hash schema.

This patch also contains unit test for the newly introduced method.

I updated TestHashSchemasPerRangeWithPartialPrimaryKeyRangePruning
and TestInListHashPruningPerRange scenarios of the PartitionPrunerTest
accordingly since now the number of initial ranges for pruning changed
even if the number of non-pruned ranges to scan stayed the same.

This is a follow-up to [1].

[1] https://gerrit.cloudera.org/#/c/18642/

Change-Id: I7f1903a444d47d30bbd7e119977cbb87bf1aa458
Reviewed-on: http://gerrit.cloudera.org:8080/18672
Tested-by: Alexey Serbin 
Reviewed-by: Attila Bukor 
---
 src/kudu/common/partition.cc |  11 +
 src/kudu/common/partition.h  |   5 +-
 src/kudu/common/partition_pruner-test.cc | 606 ++-
 src/kudu/common/partition_pruner.cc  | 176 ++---
 src/kudu/common/partition_pruner.h   |  24 ++
 5 files changed, 765 insertions(+), 57 deletions(-)

diff --git a/src/kudu/common/partition.cc b/src/kudu/common/partition.cc
index 852f0fe15..309071ef4 100644
--- a/src/kudu/common/partition.cc
+++ b/src/kudu/common/partition.cc
@@ -301,6 +301,17 @@ Status PartitionSchema::FromPB(
 }
   }
 
+  // Sort the ranges.
+  constexpr struct {
+bool operator()(const PartitionSchema::RangeWithHashSchema& lhs,
+const PartitionSchema::RangeWithHashSchema& rhs) const {
+  return lhs.lower < rhs.lower;
+}
+  } rangeLess;
+  sort(ranges_with_custom_hash_schemas.begin(),
+   ranges_with_custom_hash_schemas.end(),
+   rangeLess);
+
   auto& dict = partition_schema->hash_schema_idx_by_encoded_range_start_;
   for (auto it = ranges_with_custom_hash_schemas.cbegin();
it != ranges_with_custom_hash_schemas.cend(); ++it) {
diff --git a/src/kudu/common/partition.h b/src/kudu/common/partition.h
index b74ce566b..ef4b48d3a 100644
--- a/src/kudu/common/partition.h
+++ b/src/kudu/common/partition.h
@@ -462,6 +462,8 @@ class PartitionSchema {
 return hash_schema_;
   }
 
+  // Return all the known ranges that have custom hash schemas. The ranges are
+  // sorted by the lower bound in ascending order; the ranges do not intersect.
   const RangesWithHashSchemas& ranges_with_custom_hash_schemas() const {
 return ranges_with_custom_hash_schemas_;
   }
@@ -666,7 +668,8 @@ class PartitionSchema {
   HashSchema hash_schema_;
 
   // This contains only ranges with range-specific (i.e. different from
-  // the table-wide) hash schemas.
+  // the table-wide) hash schemas. This array is sorted by a range's lower 
bound
+  // in ascending order; the ranges do not intersect.
   RangesWithHashSchemas ranges_with_custom_hash_schemas_;
 
   // Encoded start of the range --> index of the hash bucket schemas for the
diff --git a/src/kudu/common/partition_pruner-test.cc 
b/src/kudu/common/partition_pruner-test.cc
index 85c3cef47..40c0ed398 100644
--- a/src/kudu/common/partition_pruner-test.cc
+++ b/src/kudu/common/partition_pruner-test.cc
@@ -20,6 +20,7 @@
 #include 
 #include 
 #include 
+#include 
 #include 
 #include 
 #include 
@@ -49,10 +50,12 @@ using std::get;
 using std::make_tuple;
 using std::nullopt;
 using std::optional;
+using std::ostream;
 using std::pair;
 using std::string;
 using std::tuple;
 using std::vector;
+using strings::Substitute;
 
 namespace kudu {
 
@@ -1283,7 +1286,7 @@ TEST_F(PartitionPrunerTest, 
TestHashSchemasPerRangeWithPartialPrimaryKeyRangePru
 3);
 
   PartitionSchemaPB pb;
-  CreatePartitionSchemaPB({"a", "b"}, {}, );

[kudu] branch master updated: KUDU-2671 update partition schema in catalog when dropping range

2022-06-29 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new f55dd22bf KUDU-2671 update partition schema in catalog when dropping 
range
f55dd22bf is described below

commit f55dd22bfea4beee99d72891efbbc67307b19d1e
Author: Alexey Serbin 
AuthorDate: Mon Jun 20 20:34:20 2022 -0700

KUDU-2671 update partition schema in catalog when dropping range

When dropping a range with custom hash schema from a table, it's
necessary to update the partition schema information stored in the
system catalog correspondingly.  That was missing in one of the previous
patches and this patch fixes the deficiency.

This patch also adds a test scenario to spot regressions, if any.  The
scenario was failing before the update in CatalogManager introduced
in this patch.

In addition, I updated the PartitionSchema class to store only the
information on ranges with custom hash schemas in the
ranges_with_custom_hash_schemas_ field.  I also added unit test
scenarios to cover the updated functionality of the PartitionSchema
class.

A follow-up patch is needed to refresh PartitionPruner's code since
the logic in PartitionPruner::Init() is expecting that the whole list
of table's ranges would returned by the
PartitionSchema::ranges_with_hash_schemas() method.

This is a follow-up to 250eb90bc0e1f4f472f44de8a23ce213595d5ee7.

Change-Id: Ib78afdd1a358751dca43f564c5d8e69191f165d4
Reviewed-on: http://gerrit.cloudera.org:8080/18642
Reviewed-by: Mahesh Reddy 
Reviewed-by: Abhishek Chennaka 
Tested-by: Alexey Serbin 
Reviewed-by: Attila Bukor 
---
 src/kudu/common/partition-test.cc| 465 ++-
 src/kudu/common/partition.cc | 161 ---
 src/kudu/common/partition.h  |  64 +++--
 src/kudu/common/partition_pruner-test.cc |  92 +++---
 src/kudu/common/partition_pruner.cc  |   4 +-
 src/kudu/master/catalog_manager.cc   | 123 +---
 src/kudu/master/catalog_manager.h|  15 +-
 src/kudu/master/master-test.cc   | 103 ++-
 8 files changed, 855 insertions(+), 172 deletions(-)

diff --git a/src/kudu/common/partition-test.cc 
b/src/kudu/common/partition-test.cc
index 3adacef56..1202f96ee 100644
--- a/src/kudu/common/partition-test.cc
+++ b/src/kudu/common/partition-test.cc
@@ -968,7 +968,9 @@ TEST_F(PartitionTest, VaryingHashSchemasPerRange) {
 ASSERT_OK(upper.SetStringCopy("a", "a4"));
 ASSERT_OK(upper.SetStringCopy("b", "b4"));
 AddRangePartitionWithSchema(
-schema, lower, upper, table_wide_hash_schema, _pb);
+schema, lower, upper,
+{ { { ColumnId(0), ColumnId(2) }, 3, 1 }, { { ColumnId(1) }, 2, 10 } },
+_pb);
   }
 
   { // [(a5, b5, _), (a6, _, c6))
@@ -984,14 +986,15 @@ TEST_F(PartitionTest, VaryingHashSchemasPerRange) {
   }
 
   PartitionSchema ps;
-  ASSERT_OK(PartitionSchema::FromPB(ps_pb, schema, ));
+  PartitionSchema::RangesWithHashSchemas ranges;
+  ASSERT_OK(PartitionSchema::FromPB(ps_pb, schema, , ));
   CheckSerializationFunctions(ps_pb, ps, schema);
 
   ASSERT_EQ("HASH (a, c) PARTITIONS 3, HASH (b) PARTITIONS 2, RANGE (a, b, c)",
 ps.DebugString(schema));
 
   vector partitions;
-  ASSERT_OK(ps.CreatePartitions(schema, ));
+  ASSERT_OK(ps.CreatePartitions(ranges, schema, ));
 
   ASSERT_EQ(16, partitions.size());
 
@@ -1175,13 +1178,14 @@ TEST_F(PartitionTest, CustomHashSchemasPerRangeOnly) {
   }
 
   PartitionSchema ps;
-  ASSERT_OK(PartitionSchema::FromPB(ps_pb, schema, ));
+  PartitionSchema::RangesWithHashSchemas ranges;
+  ASSERT_OK(PartitionSchema::FromPB(ps_pb, schema, , ));
   CheckSerializationFunctions(ps_pb, ps, schema);
 
   ASSERT_EQ("RANGE (a, b)", ps.DebugString(schema));
 
   vector partitions;
-  ASSERT_OK(ps.CreatePartitions(schema, ));
+  ASSERT_OK(ps.CreatePartitions(ranges, schema, ));
   ASSERT_EQ(2, partitions.size());
 
   {
@@ -1237,23 +1241,20 @@ TEST_F(PartitionTest, 
VaryingHashSchemasPerUnboundedRanges) {
 KuduPartialRow upper();
 ASSERT_OK(lower.SetStringCopy("a", "a4"));
 ASSERT_OK(lower.SetStringCopy("b", "b4"));
-PartitionSchema::HashSchema hash_schema_2_buckets_by_3 = {
-{{ColumnId(0)}, 2, 0},
-{{ColumnId(2)}, 3, 0}
-};
 AddRangePartitionWithSchema(
 schema, lower, upper,
 {{{ColumnId(0)}, 2, 0}, {{ColumnId(2)}, 3, 0}}, _pb);
   }
 
   PartitionSchema ps;
-  ASSERT_OK(PartitionSchema::FromPB(ps_pb, schema, ));
+  PartitionSchema::RangesWithHashSchemas ranges;
+  ASSERT_OK(PartitionSchema::FromPB(ps_pb, schema, , ));
   CheckSerializationFunctions(ps_pb, ps, schema);
 
   ASSERT_EQ("HASH (b) PARTITIO

[kudu] branch master updated: KUDU-2671 update Add{ExclusiveUpper,Lower}BoundPartitionKeyRaw

2022-06-21 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new ed5d5a11e KUDU-2671 update 
Add{ExclusiveUpper,Lower}BoundPartitionKeyRaw
ed5d5a11e is described below

commit ed5d5a11e54211c46560e3136bece6532e6d3b72
Author: Alexey Serbin 
AuthorDate: Mon Jun 20 12:00:17 2022 -0700

KUDU-2671 update Add{ExclusiveUpper,Lower}BoundPartitionKeyRaw

Since we don't allow for variable number of hash dimensions in
range-specific hash schemas for a table, this patch updates the code
in KuduScanner::AddExclusiveUpperBoundPartitionKeyRaw() and
KuduScanner::AddLowerBoundPartitionKeyRaw() to enable using them
for tables having custom hash schemas per range.

This is a follow-up to 6998193e69eeda497f912d1d806470c95b591ad4.

Change-Id: I1323e494733c9c082dedc12d941811f4175dbeab
Reviewed-on: http://gerrit.cloudera.org:8080/18640
Reviewed-by: Mahesh Reddy 
Tested-by: Kudu Jenkins
Reviewed-by: Abhishek Chennaka 
Reviewed-by: Attila Bukor 
---
 src/kudu/client/client.cc | 41 ++---
 1 file changed, 22 insertions(+), 19 deletions(-)

diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 7d11315f4..1e9c75a50 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -1783,30 +1783,33 @@ Status KuduScanner::AddExclusiveUpperBoundRaw(const 
Slice& key) {
 }
 
 Status KuduScanner::AddLowerBoundPartitionKeyRaw(const Slice& partition_key) {
-  // TODO(aserbin): use move semantics to pass PartitionKey arguments
-  if (const auto& table = GetKuduTable();
-  table->data_->partition_schema_.HasCustomHashSchemas()) {
-return Status::InvalidArgument(Substitute(
-"$0: cannot use AddLowerBoundPartitionKeyRaw() because "
-"the table has custom per-range hash schemas", table->name()));
-  }
+  // The number of hash dimensions in all hash schemas of a table is an
+  // invariant and checked throughout the code. With that, the table-wide hash
+  // schema is used as a proxy to find the number of hash dimensions to 
separate
+  // the hash-related prefix from the rest of the encoded partition key in the
+  // code below.
+  //
+  // TODO(KUDU-2671) update this code if allowing for different number of
+  // dimensions in range-specific hash schemas
   const auto& hash_schema = GetKuduTable()->partition_schema().hash_schema();
-  auto pkey = Partition::StringToPartitionKey(partition_key.ToString(),
-  hash_schema.size());
-  return data_->mutable_configuration()->AddLowerBoundPartitionKeyRaw(pkey);
+  return data_->mutable_configuration()->AddLowerBoundPartitionKeyRaw(
+  Partition::StringToPartitionKey(partition_key.ToString(),
+  hash_schema.size()));
 }
 
 Status KuduScanner::AddExclusiveUpperBoundPartitionKeyRaw(const Slice& 
partition_key) {
-  if (const auto& table = GetKuduTable();
-  table->data_->partition_schema_.HasCustomHashSchemas()) {
-return Status::InvalidArgument(Substitute(
-"$0: cannot use AddExclusiveUpperBoundPartitionKeyRaw() because "
-"the table has custom per-range hash schemas", table->name()));
-  }
+  // The number of hash dimensions in all hash schemas of a table is an
+  // invariant and checked throughout the code. With that, the table-wide hash
+  // schema is used as a proxy to find the number of hash dimensions to 
separate
+  // the hash-related prefix from the rest of the encoded partition key in the
+  // code below.
+  //
+  // TODO(KUDU-2671) update this code if allowing for different number of
+  // dimensions in range-specific hash schemas
   const auto& hash_schema = GetKuduTable()->partition_schema().hash_schema();
-  auto pkey = Partition::StringToPartitionKey(partition_key.ToString(),
-  hash_schema.size());
-  return data_->mutable_configuration()->AddUpperBoundPartitionKeyRaw(pkey);
+  return data_->mutable_configuration()->AddUpperBoundPartitionKeyRaw(
+  Partition::StringToPartitionKey(partition_key.ToString(),
+  hash_schema.size()));
 }
 
 Status KuduScanner::SetCacheBlocks(bool cache_blocks) {



[kudu] branch master updated: [rpc] report the names of the unsupported feature flags

2022-06-17 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 45a7857b4 [rpc] report the names of the unsupported feature flags
45a7857b4 is described below

commit 45a7857b494703ef5cf9da8671c75da88d5fc16d
Author: Alexey Serbin 
AuthorDate: Thu Jun 16 19:16:34 2022 -0700

[rpc] report the names of the unsupported feature flags

While introducing feature flags in the scope of KUDU-2671, I noticed
that missing feature flags are not reported in the error messages
(at least, I was looking at master_proxy_rpc.cc).  This patch addresses
that, making corresponding error messages more actionable.

As for test coverage, I updated a couple of test scenarios in
dynamic_multi_master-test.cc to provide initial coverage.
In addition, a follow-up patch will add a new test scenario into
flex_partitioning_client-test.cc.

Change-Id: Id9805dc3fb4ba0a734ca92198bc5dc6449f588b7
Reviewed-on: http://gerrit.cloudera.org:8080/18631
Tested-by: Kudu Jenkins
Reviewed-by: Attila Bukor 
---
 src/kudu/client/master_proxy_rpc.cc  | 11 +--
 src/kudu/client/txn_manager_proxy_rpc.cc | 16 ++--
 src/kudu/master/dynamic_multi_master-test.cc |  9 ++---
 src/kudu/tserver/heartbeater.cc  | 10 +-
 4 files changed, 34 insertions(+), 12 deletions(-)

diff --git a/src/kudu/client/master_proxy_rpc.cc 
b/src/kudu/client/master_proxy_rpc.cc
index da4bb1ed4..8b8a2b814 100644
--- a/src/kudu/client/master_proxy_rpc.cc
+++ b/src/kudu/client/master_proxy_rpc.cc
@@ -28,6 +28,7 @@
 #include "kudu/client/client-internal.h"
 #include "kudu/client/client.h"
 #include "kudu/common/wire_protocol.h"
+#include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/rpc/response_callback.h"
@@ -77,6 +78,7 @@ using master::ListTabletServersRequestPB;
 using master::ListTabletServersResponsePB;
 using master::MasterServiceProxy;
 using master::MasterErrorPB;
+using master::MasterFeatures_Name;
 using master::RemoveMasterRequestPB;
 using master::RemoveMasterResponsePB;
 using master::ReplaceTabletRequestPB;
@@ -259,8 +261,13 @@ bool AsyncLeaderMasterRpc::RetryOrReconnectIfNecessary(
   return true;
 }
 if (err->unsupported_feature_flags_size() > 0) {
-  s = Status::NotSupported(Substitute("Cluster does not support $0",
-  rpc_name_));
+  const auto features_str = JoinMapped(err->unsupported_feature_flags(),
+   [](uint32_t feature) {
+ return 
MasterFeatures_Name(feature);
+   }, ",");
+  s = Status::NotSupported(
+  Substitute("cluster does not support $0 with feature(s) $1",
+ rpc_name_, features_str));
 }
   }
 
diff --git a/src/kudu/client/txn_manager_proxy_rpc.cc 
b/src/kudu/client/txn_manager_proxy_rpc.cc
index ac7c03f7e..42a4bc59d 100644
--- a/src/kudu/client/txn_manager_proxy_rpc.cc
+++ b/src/kudu/client/txn_manager_proxy_rpc.cc
@@ -18,6 +18,7 @@
 #include "kudu/client/txn_manager_proxy_rpc.h"
 
 #include 
+#include 
 #include 
 #include 
 #include 
@@ -28,6 +29,7 @@
 #include "kudu/client/client-internal.h"
 #include "kudu/client/client.h"
 #include "kudu/common/wire_protocol.h"
+#include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/txn_manager.pb.h"
 #include "kudu/rpc/response_callback.h"
@@ -58,6 +60,7 @@ using kudu::transactions::KeepTransactionAliveRequestPB;
 using kudu::transactions::KeepTransactionAliveResponsePB;
 using kudu::transactions::TxnManagerServiceProxy;
 using kudu::transactions::TxnManagerErrorPB;
+using kudu::transactions::TxnManagerFeatures_Name;
 using std::string;
 using strings::Substitute;
 
@@ -207,13 +210,14 @@ bool AsyncRandomTxnManagerRpc::RetryIfNecessary(
   }
   return true;
 }
-// TODO(aserbin): report unsupported features in the error message if it
-//starts making sense: of course this code is forward
-//looking, but it's not clear how the detailed information
-//on missing features could help in making this error
-//message more actionable
 if (err->unsupported_feature_flags_size() > 0) {
-  s = Status::NotSupported("TxnManager is missing required features");
+  const auto required_features_str = JoinMapped(
+  err->unsuppo

[kudu] 02/02: KUDU-2671 update partition schema in catalog when adding range

2022-06-16 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 6909ee4f800da192b72e59680916e5004527b6db
Author: Alexey Serbin 
AuthorDate: Mon Jun 13 15:10:13 2022 -0700

KUDU-2671 update partition schema in catalog when adding range

When adding a range with custom hash schema to a table, it's necessary
to update the partition schema information stored in the system catalog
correspondingly.  That was missing in one of the previous patches and
this patch addresses the issue.

This patch also adds a test scenario to spot regressions, if any.  The
scenario was failing before the update in CatalogManager introduced
in this patch.  I also addressed nits pointed to by the TidyBot.

This is a follow-up to 250eb90bc0e1f4f472f44de8a23ce213595d5ee7.

Change-Id: I869458fb8bcb06801b54f2b4869e7826322563e0
Reviewed-on: http://gerrit.cloudera.org:8080/18615
Tested-by: Kudu Jenkins
Reviewed-by: Mahesh Reddy 
Reviewed-by: Attila Bukor 
---
 src/kudu/master/catalog_manager.cc |  48 +
 src/kudu/master/catalog_manager.h  |  24 ---
 src/kudu/master/master-test.cc | 137 -
 3 files changed, 181 insertions(+), 28 deletions(-)

diff --git a/src/kudu/master/catalog_manager.cc 
b/src/kudu/master/catalog_manager.cc
index 726499d56..d23d01e64 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -2518,10 +2518,11 @@ Status CatalogManager::DeleteTable(const 
DeleteTableRequestPB& req,
   return Status::OK();
 }
 
-Status CatalogManager::ApplyAlterSchemaSteps(const SysTablesEntryPB& 
current_pb,
- vector 
steps,
- Schema* new_schema,
- ColumnId* next_col_id) {
+Status CatalogManager::ApplyAlterSchemaSteps(
+const SysTablesEntryPB& current_pb,
+const vector& steps,
+Schema* new_schema,
+ColumnId* next_col_id) {
   const SchemaPB& current_schema_pb = current_pb.schema();
   Schema cur_schema;
   RETURN_NOT_OK(SchemaFromPB(current_schema_pb, _schema));
@@ -2601,20 +2602,20 @@ Status CatalogManager::ApplyAlterSchemaSteps(const 
SysTablesEntryPB& current_pb,
 }
 
 Status CatalogManager::ApplyAlterPartitioningSteps(
-const TableMetadataLock& l,
 const scoped_refptr& table,
 const Schema& client_schema,
-vector steps,
+const vector& steps,
+TableMetadataLock* l,
 vector>* tablets_to_add,
 vector>* tablets_to_drop) {
 
   // Get the table's schema as it's known to the catalog manager.
   Schema schema;
-  RETURN_NOT_OK(SchemaFromPB(l.data().pb.schema(), ));
+  RETURN_NOT_OK(SchemaFromPB(l->data().pb.schema(), ));
   // Build current PartitionSchema for the table.
   PartitionSchema partition_schema;
   RETURN_NOT_OK(PartitionSchema::FromPB(
-  l.data().pb.partition_schema(), schema, _schema));
+  l->data().pb.partition_schema(), schema, _schema));
   TableInfo::TabletInfoMap existing_tablets = table->tablet_map();
   TableInfo::TabletInfoMap new_tablets;
   auto abort_mutations = MakeScopedCleanup([_tablets]() {
@@ -2627,11 +2628,11 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
   for (const auto& step : steps) {
 CHECK(step.type() == AlterTableRequestPB::ADD_RANGE_PARTITION ||
   step.type() == AlterTableRequestPB::DROP_RANGE_PARTITION);
-const auto& range_bouds =
+const auto& range_bounds =
 step.type() == AlterTableRequestPB::ADD_RANGE_PARTITION
 ? step.add_range_partition().range_bounds()
 : step.drop_range_partition().range_bounds();
-RowOperationsPBDecoder decoder(_bouds, _schema, , 
nullptr);
+RowOperationsPBDecoder decoder(_bounds, _schema, , 
nullptr);
 vector ops;
 RETURN_NOT_OK(decoder.DecodeOperations());
 
@@ -2675,6 +2676,23 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
   }
   RETURN_NOT_OK(partition_schema.CreatePartitionsForRange(
   range_bound, hash_schema, schema, ));
+
+  // Add information on the new range with custom hash schema into the
+  // PartitionSchema for the table stored in the system catalog.
+  auto* p = l->mutable_data()->pb.mutable_partition_schema();
+  auto* range = p->add_custom_hash_schema_ranges();
+  RowOperationsPBEncoder encoder(range->mutable_range_bounds());
+  encoder.Add(RowOperationsPB::RANGE_LOWER_BOUND, *ops[0].split_row);
+  encoder.Add(RowOperationsPB::RANGE_UPPER_BOUND, *ops[1].split_row);
+  for (const auto& hash_dimension : hash_schema) {
+auto* hash_dimension_pb = range->add_hash_schema();
+hash_dimension_pb->set_num_buckets(hash_dimension.num_buckets);
+hash_dimension_pb->set_seed(hash_dim

[kudu] 01/02: KUDU-2671 more robust convention on specifying range bounds

2022-06-16 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 295b4903bc69fabb3cb36f618022d465c91954c7
Author: Alexey Serbin 
AuthorDate: Fri Jun 3 14:29:58 2022 -0700

KUDU-2671 more robust convention on specifying range bounds

This patch updates the code of the catalog manager to adhere to a more
robust convention on specifying the information for the range partition
boundaries when creating a table with custom hash schemas per range.

Prior to this patch, the catalog manager required both the
CreateTableRequestPB::split_rows_range_bounds and the
CreateTableRequestPB::partition_schema::custom_hash_schema_ranges fields
to have the same number of elements, assuming the former had the ranges
exactly corresponding to the latter, where the latter would also had
information on hash schema for each range correspondingly.  In addition
to duplicating the information unnecessarily, that approach was also
a bit brittle from the standpoint of keeping good API practices.

This patch updates the code to use a new convention: if there is at
least one range partition with custom hash schema in CreateTable RPC,
all the information on range boundaries and hash schemas should be
presented only via one field:
CreateTableRequestPB::partition_schema::custom_hash_schema_ranges.
That's better than the previous convention because:
  * it's more robust as explained above
  * it naturally follows the restriction of not allowing the split
rows along with range partitions with custom hash schemas

Also, I updated already existing tests and added extra test scenarios
to cover the updated functionality.

Change-Id: I14073e72178e6bb85bae719ad377c5bb05f8dd55
Reviewed-on: http://gerrit.cloudera.org:8080/18590
Tested-by: Alexey Serbin 
Reviewed-by: Mahesh Reddy 
Reviewed-by: Attila Bukor 
---
 src/kudu/client/client.cc  | 14 ++--
 .../integration-tests/table_locations-itest.cc | 72 +--
 src/kudu/master/catalog_manager.cc | 21 +++---
 src/kudu/master/master-test.cc | 84 ++
 4 files changed, 101 insertions(+), 90 deletions(-)

diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 61349527a..7d11315f4 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -971,24 +971,26 @@ Status KuduTableCreator::Create() {
   return Status::InvalidArgument("range bounds must not be null");
 }
 
-RowOperationsPB_Type lower_bound_type =
+const RowOperationsPB_Type lower_bound_type =
 range->lower_bound_type_ == KuduTableCreator::INCLUSIVE_BOUND
 ? RowOperationsPB::RANGE_LOWER_BOUND
 : RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND;
 
-RowOperationsPB_Type upper_bound_type =
+const RowOperationsPB_Type upper_bound_type =
 range->upper_bound_type_ == KuduTableCreator::EXCLUSIVE_BOUND
 ? RowOperationsPB::RANGE_UPPER_BOUND
 : RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND;
 
-splits_encoder.Add(lower_bound_type, *range->lower_bound_);
-splits_encoder.Add(upper_bound_type, *range->upper_bound_);
-
-if (has_range_with_custom_hash_schema) {
+if (!has_range_with_custom_hash_schema) {
+  splits_encoder.Add(lower_bound_type, *range->lower_bound_);
+  splits_encoder.Add(upper_bound_type, *range->upper_bound_);
+} else {
   auto* range_pb = partition_schema->add_custom_hash_schema_ranges();
   RowOperationsPBEncoder encoder(range_pb->mutable_range_bounds());
   encoder.Add(lower_bound_type, *range->lower_bound_);
   encoder.Add(upper_bound_type, *range->upper_bound_);
+  // Now, after adding the information range bounds, add the information
+  // on hash schema for the range.
   if (range->is_table_wide_hash_schema_) {
 // With the presence of a range with custom hash schema when the
 // table-wide hash schema is used for this particular range, also add 
an
diff --git a/src/kudu/integration-tests/table_locations-itest.cc 
b/src/kudu/integration-tests/table_locations-itest.cc
index a274cdeb7..babf660cf 100644
--- a/src/kudu/integration-tests/table_locations-itest.cc
+++ b/src/kudu/integration-tests/table_locations-itest.cc
@@ -162,12 +162,18 @@ class TableLocationsTest : public KuduTest {
   };
   typedef vector HashSchema;
 
+  struct RangeWithHashSchema {
+KuduPartialRow lower;
+KuduPartialRow upper;
+HashSchema hash_schema;
+  };
+
   Status CreateTable(
   const string& table_name,
   const Schema& schema,
   const vector& split_rows = {},
   const vector>& bounds = {},
-  const vector& range_hash_schemas = {},
+  const vector& ran

[kudu] branch master updated (9091f31cf -> 6909ee4f8)

2022-06-16 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


from 9091f31cf KUDU-2671 refactor PartitionSchema::CreatePartitions()
 new 295b4903b KUDU-2671 more robust convention on specifying range bounds
 new 6909ee4f8 KUDU-2671 update partition schema in catalog when adding 
range

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/kudu/client/client.cc  |  14 +-
 .../integration-tests/table_locations-itest.cc |  72 +++
 src/kudu/master/catalog_manager.cc |  69 ---
 src/kudu/master/catalog_manager.h  |  24 ++-
 src/kudu/master/master-test.cc | 221 ++---
 5 files changed, 282 insertions(+), 118 deletions(-)



[kudu] branch master updated: KUDU-2671 refactor PartitionSchema::CreatePartitions()

2022-06-14 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 9091f31cf KUDU-2671 refactor PartitionSchema::CreatePartitions()
9091f31cf is described below

commit 9091f31cfc5c102b6a734fa20b015735251392d4
Author: Alexey Serbin 
AuthorDate: Tue May 31 20:18:51 2022 -0700

KUDU-2671 refactor PartitionSchema::CreatePartitions()

This patch refactors the code of the PartitionSchema::CreatePartitions()
method and the code around its invocations, addressing a few TODOs.
As a consequence, this patch fixes a bug in the CreateTable path: prior
to this patch, a client could submit a CreateTableRequestPB request such
that the order of ranges in CreateTableRequestPB::split_rows_range_bounds
and CreateTableRequestPB::partition_schema::custom_hash_schema_ranges
were different, and the result table would be created with wrong hash
schemas.

For example:

  requested:
{{range_boundary_a0, range_boundary_a1}, hash_schema_a}
{{range_boundary_b0, range_boundary_b1}, hash_schema_b}

  created:
{{range_boundary_a0, range_boundary_a1}, hash_schema_b}
{{range_boundary_b0, range_boundary_b1}, hash_schema_a}

I'm planning to add corresponding test scenario to catch the regressions
in a separate changelist for ease of reviewing and tracking the changes.

However, I updated the existing test scenarios affected by this change.

Change-Id: I5bac1f8ee349577e2f912530a28776415ed0a5b0
Reviewed-on: http://gerrit.cloudera.org:8080/18582
Tested-by: Alexey Serbin 
Reviewed-by: Abhishek Chennaka 
Reviewed-by: Attila Bukor 
---
 src/kudu/common/partition-test.cc  | 571 +
 src/kudu/common/partition.cc   | 285 +-
 src/kudu/common/partition.h|  46 +-
 src/kudu/common/partition_pruner-test.cc   |  76 +--
 src/kudu/common/scan_spec-test.cc  |  18 +-
 .../integration-tests/ts_tablet_manager-itest.cc   |   2 +-
 src/kudu/master/catalog_manager.cc |  98 ++--
 src/kudu/master/master-test.cc |   4 +-
 src/kudu/master/sys_catalog.cc |   2 +-
 src/kudu/tablet/tablet-harness.h   |   2 +-
 src/kudu/tserver/tablet_server-test.cc |   2 +-
 11 files changed, 531 insertions(+), 575 deletions(-)

diff --git a/src/kudu/common/partition-test.cc 
b/src/kudu/common/partition-test.cc
index 259ba28c3..d74936936 100644
--- a/src/kudu/common/partition-test.cc
+++ b/src/kudu/common/partition-test.cc
@@ -19,9 +19,7 @@
 
 #include 
 #include 
-#include 
 #include 
-#include 
 #include 
 #include 
 
@@ -112,7 +110,7 @@ void CheckCreateRangePartitions(const 
vector, optional partitions;
-  ASSERT_OK(partition_schema.CreatePartitions(splits, bounds, {}, schema, 
));
+  ASSERT_OK(partition_schema.CreatePartitions(splits, bounds, schema, 
));
   ASSERT_EQ(expected_partition_ranges.size(), partitions.size());
 
   for (int i = 0; i < partitions.size(); i++) {
@@ -188,7 +186,7 @@ TEST_F(PartitionTest, TestCompoundRangeKeyEncoding) {
   }
 
   vector partitions;
-  ASSERT_OK(partition_schema.CreatePartitions(splits, bounds, {}, schema, 
));
+  ASSERT_OK(partition_schema.CreatePartitions(splits, bounds, schema, 
));
   ASSERT_EQ(4, partitions.size());
 
   EXPECT_TRUE(partitions[0].hash_buckets().empty());
@@ -475,7 +473,7 @@ TEST_F(PartitionTest, TestCreateHashPartitions) {
 
   vector partitions;
   ASSERT_OK(
-  partition_schema.CreatePartitions(vector(), {}, {}, 
schema, ));
+  partition_schema.CreatePartitions(vector(), {}, schema, 
));
   ASSERT_EQ(3, partitions.size());
 
   EXPECT_EQ(0, partitions[0].hash_buckets()[0]);
@@ -564,7 +562,7 @@ TEST_F(PartitionTest, TestCreatePartitions) {
   // Split keys need not be passed in sorted order.
   vector split_rows = { split_b, split_a };
   vector partitions;
-  ASSERT_OK(partition_schema.CreatePartitions(split_rows, {}, {}, schema, 
));
+  ASSERT_OK(partition_schema.CreatePartitions(split_rows, {}, schema, 
));
   ASSERT_EQ(12, partitions.size());
 
   EXPECT_EQ(0, partitions[0].hash_buckets()[0]);
@@ -912,31 +910,43 @@ void CheckSerializationFunctions(const PartitionSchemaPB& 
pb,
   ASSERT_EQ(partition_schema, partition_schema1);
 }
 
+void AddRangePartitionWithSchema(
+const Schema& schema,
+const KuduPartialRow& lower,
+const KuduPartialRow& upper,
+const PartitionSchema::HashSchema& range_hash_schema,
+PartitionSchemaPB* pb) {
+  auto* range = pb->add_custom_hash_schema_ranges();
+  RowOperationsPBEncoder encoder(range->mutable_range_bounds());
+  encoder.Add(RowOperationsPB::RANGE_LOWER_BOUND, lower);
+  encoder.Add(RowOperationsPB::RANGE_UPPER_BOUN

[kudu] branch master updated: [tests] make the startup page tests more robust

2022-06-13 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 236a58914 [tests] make the startup page tests more robust
236a58914 is described below

commit 236a589140f46b7a0138c2b26620f596139f2777
Author: Alexey Serbin 
AuthorDate: Thu Jun 9 18:17:48 2022 -0700

[tests] make the startup page tests more robust

I noticed that the startup page tests reliably fail on my test machine.
It turned out that there was KUDU_HOME variable set in the environment,
so the embedded webserver rendered the pages according to the Mustache
mappings found under $KUDU_HOME/www, while the test scenarios expect all
pages to be pre-rendered.

This patch updates corresponding test scenarios to reliably work even if
the KUDU_HOME environment is set and points to the directory with
the Mustache mapping files.

Change-Id: I2e56cd49eb2e00c5666e4214c7ca4246386e8b9e
Reviewed-on: http://gerrit.cloudera.org:8080/18606
Tested-by: Kudu Jenkins
Reviewed-by: Abhishek Chennaka 
Reviewed-by: Attila Bukor 
---
 src/kudu/master/master-test.cc | 118 -
 src/kudu/tserver/tablet_server-test.cc |  28 +---
 2 files changed, 93 insertions(+), 53 deletions(-)

diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index 5e13ed68c..d69003f56 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -141,6 +141,7 @@ DECLARE_int64(live_row_count_for_testing);
 DECLARE_int64(on_disk_size_for_testing);
 DECLARE_string(location_mapping_cmd);
 DECLARE_string(log_filename);
+DECLARE_string(webserver_doc_root);
 
 
METRIC_DECLARE_histogram(handler_latency_kudu_master_MasterService_GetTableSchema);
 
@@ -392,50 +393,6 @@ TEST_F(MasterTest, 
TestResetBlockCacheMetricsInSameProcess) {
   });
 }
 
-TEST_F(MasterTest, TestStartupWebPage) {
-  EasyCurl c;
-  faststring buf;
-  string addr = mini_master_->bound_http_addr().ToString();
-  mini_master_->Shutdown();
-  std::atomic run_status_reader = false;
-  thread read_startup_page([&] {
-EasyCurl thread_c;
-faststring thread_buf;
-while (!run_status_reader) {
-  SleepFor(MonoDelta::FromMilliseconds(10));
-  if (!(thread_c.FetchURL(strings::Substitute("http://$0/startup;, addr), 
_buf)).ok()) {
-continue;
-  }
-  ASSERT_STR_MATCHES(thread_buf.ToString(), "\"init_status\":(100|0)( 
|,)");
-  ASSERT_STR_MATCHES(thread_buf.ToString(), 
"\"read_filesystem_status\":(100|0)( |,)");
-  ASSERT_STR_MATCHES(thread_buf.ToString(), 
"\"read_instance_metadatafiles_status\""
-":(100|0)( |,)");
-  ASSERT_STR_MATCHES(thread_buf.ToString(), 
"\"read_data_directories_status\":"
-"([0-9]|[1-9][0-9]|100)( 
|,)");
-  ASSERT_STR_MATCHES(thread_buf.ToString(), 
"\"initialize_master_catalog_status\":"
-"([0-9]|[1-9][0-9]|100)( 
|,)");
-  ASSERT_STR_MATCHES(thread_buf.ToString(), 
"\"start_rpc_server_status\":(100|0)( |,)");
-}
-  });
-  SCOPED_CLEANUP({
-run_status_reader = true;
-read_startup_page.join();
-  });
-
-  ASSERT_OK(mini_master_->Restart());
-  ASSERT_OK(mini_master_->WaitForCatalogManagerInit());
-  run_status_reader = true;
-
-  // After all the steps have been completed, ensure every startup step has 
100 percent status
-  ASSERT_OK(c.FetchURL(strings::Substitute("http://$0/startup;, addr), ));
-  ASSERT_STR_CONTAINS(buf.ToString(), "\"init_status\":100");
-  ASSERT_STR_CONTAINS(buf.ToString(), "\"read_filesystem_status\":100");
-  ASSERT_STR_CONTAINS(buf.ToString(), 
"\"read_instance_metadatafiles_status\":100");
-  ASSERT_STR_CONTAINS(buf.ToString(), "\"read_data_directories_status\":100");
-  ASSERT_STR_CONTAINS(buf.ToString(), 
"\"initialize_master_catalog_status\":100");
-  ASSERT_STR_CONTAINS(buf.ToString(), "\"start_rpc_server_status\":100");
-}
-
 TEST_F(MasterTest, TestRegisterAndHeartbeat) {
   const char* const kTsUUID = "my-ts-uuid";
 
@@ -2819,8 +2776,79 @@ TEST_P(AuthzTokenMasterTest, TestGenerateAuthzTokens) {
 ASSERT_EQ(supports_authz, resp.has_authz_token());
   }
 }
-
 INSTANTIATE_TEST_SUITE_P(SupportsAuthzTokens, AuthzTokenMasterTest, 
::testing::Bool());
 
+class MasterStartupTest : public KuduTest {
+ protected:
+  void SetUp() override {
+KuduTest::SetUp();
+
+// The embedded webserver renders the contents of the generated pages
+   

[kudu] branch master updated: KUDU-3354: Deflake DisableWriteWhenExceedingQuotaTest

2022-06-10 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 687bef897 KUDU-3354: Deflake DisableWriteWhenExceedingQuotaTest
687bef897 is described below

commit 687bef89764640746deb27b3279e936acae4446e
Author: zhangyifan27 
AuthorDate: Thu Jun 2 22:07:32 2022 +0800

KUDU-3354: Deflake DisableWriteWhenExceedingQuotaTest

I noticed that the TestDisableWritePrivilegeWhenExceedingSizeQuota
test failed some times in latest code review items because of test
timeout. From the test log we can see maintenance manager tends to
schedule a high score compact operation of some rowsets but do
nothing in the end because 'Compaction resulted in no output rows
(all input rows were GCed!)'. This leads to failures to schedule
flush operations and to hit the size limit until run out of time.

I think this weird high perf score most likely has something to do
with not explicitly initializing the 'value_' of RowsetInfo after
some debugging. The 'value_' can't be updated in FinalizeCDFVector()
either, because in the test scenario it has nearly one key in a
rowset, and the 'total_width' of the rowset is 0.

I run the test 10 times and see no failure with this fix, compared
to 5 failures without the fix but with a DCHECK[1] that indicates
the compaction error occurred in the test case.

[1] https://gerrit.cloudera.org/c/18584/1/src/kudu/tablet/tablet.cc#2135

Change-Id: Id257a1ce04fb2043d79641f770d03393ec5b844b
Reviewed-on: http://gerrit.cloudera.org:8080/18584
Reviewed-by: Yingchun Lai 
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin 
---
 src/kudu/tablet/rowset_info.cc | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/src/kudu/tablet/rowset_info.cc b/src/kudu/tablet/rowset_info.cc
index 37d20167b..661de69e7 100644
--- a/src/kudu/tablet/rowset_info.cc
+++ b/src/kudu/tablet/rowset_info.cc
@@ -451,6 +451,8 @@ void RowSetInfo::SplitKeyRange(const RowSetTree& tree,
 RowSetInfo::RowSetInfo(RowSet* rs, double init_cdf)
 : cdf_min_key_(init_cdf),
   cdf_max_key_(init_cdf),
+  value_(0.0),
+  density_(0.0),
   extra_(new ExtraData()) {
   extra_->rowset = rs;
   extra_->base_and_redos_size_bytes = rs->OnDiskBaseDataSizeWithRedos();



[kudu] branch master updated: [master] KUDU-2671: Range specific hashing during table alter op.

2022-06-09 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 250eb90bc [master] KUDU-2671: Range specific hashing during table 
alter op.
250eb90bc is described below

commit 250eb90bc0e1f4f472f44de8a23ce213595d5ee7
Author: Abhishek Chennaka 
AuthorDate: Tue May 10 10:09:31 2022 -0400

[master] KUDU-2671: Range specific hashing during table alter op.

This commit has the changes needed on the master side to
support this functionality. A basic test is added to test
the functionality as well.

Change-Id: Iea9e3317d172c9ae76662c44b21fca9a4819930a
Reviewed-on: http://gerrit.cloudera.org:8080/18515
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 src/kudu/master/catalog_manager.cc |  16 -
 src/kudu/master/master-test.cc | 128 ++---
 2 files changed, 134 insertions(+), 10 deletions(-)

diff --git a/src/kudu/master/catalog_manager.cc 
b/src/kudu/master/catalog_manager.cc
index 2928466f4..ab3380f23 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -2618,7 +2618,6 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
   PartitionSchema partition_schema;
   RETURN_NOT_OK(PartitionSchema::FromPB(
   l.data().pb.partition_schema(), schema, _schema));
-
   TableInfo::TabletInfoMap existing_tablets = table->tablet_map();
   TableInfo::TabletInfoMap new_tablets;
   auto abort_mutations = MakeScopedCleanup([_tablets]() {
@@ -2627,9 +2626,22 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
 }
   });
 
+  vector range_hash_schemas;
   for (const auto& step : steps) {
 vector ops;
 if (step.type() == AlterTableRequestPB::ADD_RANGE_PARTITION) {
+  if (FLAGS_enable_per_range_hash_schemas &&
+  step.add_range_partition().custom_hash_schema_size() > 0) {
+const Schema schema = client_schema.CopyWithColumnIds();
+PartitionSchema::HashSchema hash_schema;
+RETURN_NOT_OK(PartitionSchema::ExtractHashSchemaFromPB(
+schema, step.add_range_partition().custom_hash_schema(), 
_schema));
+if (partition_schema.hash_schema().size() != hash_schema.size()) {
+  return Status::NotSupported(
+  "varying number of hash dimensions per range is not yet 
supported");
+}
+range_hash_schemas.emplace_back(std::move(hash_schema));
+  }
   RowOperationsPBDecoder 
decoder(_range_partition().range_bounds(),
  _schema, , nullptr);
   RETURN_NOT_OK(decoder.DecodeOperations());
@@ -2666,7 +2678,7 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
 
 vector partitions;
 RETURN_NOT_OK(partition_schema.CreatePartitions(
-{}, {{ *ops[0].split_row, *ops[1].split_row }}, {}, schema, 
));
+{}, {{ *ops[0].split_row, *ops[1].split_row }}, range_hash_schemas, 
schema, ));
 switch (step.type()) {
   case AlterTableRequestPB::ADD_RANGE_PARTITION: {
 for (const Partition& partition : partitions) {
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index db345476c..5e13ed68c 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -197,6 +197,7 @@ class MasterTest : public KuduTest {
  const vector>& 
bounds = {},
  const vector& range_hash_schemas = {});
 
+
   Status CreateTable(const string& name,
  const Schema& schema,
  const optional& type,
@@ -204,7 +205,9 @@ class MasterTest : public KuduTest {
  const optional& comment,
  const vector& split_rows,
  const vector>& 
bounds,
- const vector& range_hash_schemas);
+ const vector& range_hash_schemas,
+ const HashSchema& table_wide_hash_schema,
+ CreateTableResponsePB* resp);
 
   shared_ptr client_messenger_;
   unique_ptr mini_master_;
@@ -223,8 +226,9 @@ Status MasterTest::CreateTable(const string& name,
   KuduPartialRow split2();
   RETURN_NOT_OK(split2.SetInt32("key", 20));
 
+  CreateTableResponsePB resp;
   return CreateTable(
-  name, schema, type, owner, comment, { split1, split2 }, {}, {});
+  name, schema, type, owner, comment, { split1, split2 }, {}, {}, {}, 
);
 }
 
 Status MasterTest::CreateTable(
@@ -233,8 +237,9 @@ Status MasterTest::CreateTable(
 const vector& split_rows,
 const vector>& bounds,
 const vector& range_hash_schemas) {
+  CreateTableResponsePB resp;
   return CreateTable(
-name, schema, none, none, none, split_rows, bounds, 
range_hash_schemas);
+name,

[kudu] branch master updated: [kudu-tool-test] more robust ConnectionNegotiationTimeoutOption

2022-06-07 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new fae44610a [kudu-tool-test] more robust 
ConnectionNegotiationTimeoutOption
fae44610a is described below

commit fae44610a8af0d26463f4342847a4e37c7808f91
Author: Alexey Serbin 
AuthorDate: Mon Jun 6 21:46:52 2022 -0700

[kudu-tool-test] more robust ConnectionNegotiationTimeoutOption

This patch updates the expected message on connection negotiation
timeout in the ToolTest.ConnectionNegotiationTimeoutOption scenario,
making it more robust.

Change-Id: I9a34fc03f67c2ae3b5ff6b4f746dd87d79634c52
Reviewed-on: http://gerrit.cloudera.org:8080/18594
Reviewed-by: Mahesh Reddy 
Tested-by: Alexey Serbin 
Reviewed-by: Attila Bukor 
---
 src/kudu/tools/kudu-tool-test.cc | 7 +--
 1 file changed, 5 insertions(+), 2 deletions(-)

diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index c77fe502d..2f8329f35 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -7784,8 +7784,11 @@ TEST_F(ToolTest, ConnectionNegotiationTimeoutOption) {
 auto s = RunActionStderrString(Substitute(kPattern, rpc_addr, 10, 11), 
);
 ASSERT_TRUE(s.IsRuntimeError());
 ASSERT_STR_CONTAINS(msg, Substitute(
-"Timed out: Client connection negotiation failed: client connection to 
"
-"$0: received 0 of 4 requested bytes", rpc_addr));
+"Timed out: Client connection negotiation failed: "
+"client connection to $0", rpc_addr));
+ASSERT_STR_MATCHES(msg,
+"(Timeout exceeded waiting to connect)|"
+"(received|sent) 0 of .* requested bytes");
   }
 
   {



[kudu] branch master updated (b6aaf2b71 -> 9699f5885)

2022-06-02 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


from b6aaf2b71 KUDU-3368 Encrypt file keys with server keys
 new abe2a73cd KUDU-3373 Key provider interface
 new 9699f5885 [common] add DCHECK to spot bugs in using Schema

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/kudu/common/schema.h   | 24 ++
 src/kudu/fs/fs_manager.cc  | 26 ---
 src/kudu/fs/fs_manager.h   |  8 
 src/kudu/mini-cluster/external_mini_cluster.cc | 11 +++--
 src/kudu/mini-cluster/external_mini_cluster.h  |  6 +++
 src/kudu/server/CMakeLists.txt |  3 ++
 .../default_key_provider-test.cc}  | 30 ++--
 src/kudu/server/default_key_provider.h | 53 ++
 .../test/test_pass.h => server/key_provider.h} | 19 
 src/kudu/tools/tool_action_common.cc   |  4 +-
 src/kudu/util/env.h|  3 +-
 src/kudu/util/env_posix.cc |  2 +-
 src/kudu/util/test_util.cc |  2 +-
 13 files changed, 147 insertions(+), 44 deletions(-)
 copy src/kudu/{util/user-test.cc => server/default_key_provider-test.cc} (65%)
 create mode 100644 src/kudu/server/default_key_provider.h
 copy src/kudu/{security/test/test_pass.h => server/key_provider.h} (66%)



[kudu] 02/02: [common] add DCHECK to spot bugs in using Schema

2022-06-02 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 9699f5885ca18a1760d5d150b3ba82424d8948b3
Author: Alexey Serbin 
AuthorDate: Mon Feb 14 13:56:40 2022 -0800

[common] add DCHECK to spot bugs in using Schema

While reviewing recently posted patch [1], I was looking at the code in
src/kudu/common/schema.h and decided to add more DCHECK-like asserts
to help catching bugs in the implementation of the Schema class itself
and spotting errors at the call sites where Schema objects are used.

This patch doesn't contain any functional changes.

[1] https://gerrit.cloudera.org/#/c/18213/

Change-Id: I44b73ef06924af6556a7cd3da4a7eec20d12aefc
Reviewed-on: http://gerrit.cloudera.org:8080/18231
Reviewed-by: Andrew Wong 
Reviewed-by: Mahesh Reddy 
Tested-by: Alexey Serbin 
---
 src/kudu/common/schema.h | 24 +++-
 1 file changed, 15 insertions(+), 9 deletions(-)

diff --git a/src/kudu/common/schema.h b/src/kudu/common/schema.h
index 6662319ae..be93c9aab 100644
--- a/src/kudu/common/schema.h
+++ b/src/kudu/common/schema.h
@@ -534,6 +534,7 @@ class Schema {
   // Return the number of bytes needed to represent
   // only the key portion of this schema.
   size_t key_byte_size() const {
+DCHECK(initialized());
 return col_offsets_[num_key_columns_];
   }
 
@@ -545,6 +546,7 @@ class Schema {
 // division-by-a-constant gets optimized into multiplication,
 // the multiplication instruction has a significantly higher latency
 // than the simple load.
+DCHECK_EQ(cols_.size(), name_to_index_.size());
 return name_to_index_.size();
   }
 
@@ -597,12 +599,11 @@ class Schema {
   // Return the column index corresponding to the given column,
   // or kColumnNotFound if the column is not in this schema.
   int find_column(const StringPiece col_name) const {
-auto iter = name_to_index_.find(col_name);
+const auto iter = name_to_index_.find(col_name);
 if (PREDICT_FALSE(iter == name_to_index_.end())) {
   return kColumnNotFound;
-} else {
-  return (*iter).second;
 }
+return iter->second;
   }
 
   // Returns true if the schema contains nullable columns
@@ -622,8 +623,9 @@ class Schema {
 
   // Returns the list of primary key column IDs.
   std::vector get_key_column_ids() const {
-return std::vector(
-col_ids_.begin(), col_ids_.begin() + num_key_columns_);
+DCHECK_LE(num_key_columns_, col_ids_.size());
+return std::vector(col_ids_.begin(),
+ col_ids_.begin() + num_key_columns_);
   }
 
   // Return true if this Schema is initialized and valid.
@@ -717,15 +719,19 @@ class Schema {
 
   // Return the projection of this schema which contains only
   // the key columns.
-  // TODO: this should take a Schema* out-parameter to avoid an
+  //
+  // TODO(todd): this should take a Schema* out-parameter to avoid an
   // extra copy of the ColumnSchemas.
-  // TODO this should probably be cached since the key projection
+  //
+  // TODO(dralves): this should probably be cached since the key projection
   // is not supposed to change, for a single schema.
   Schema CreateKeyProjection() const {
+DCHECK_LE(num_key_columns_, cols_.size());
 std::vector key_cols(cols_.begin(),
-  cols_.begin() + num_key_columns_);
+   cols_.begin() + num_key_columns_);
 std::vector col_ids;
 if (!col_ids_.empty()) {
+  DCHECK_LE(num_key_columns_, col_ids_.size());
   col_ids.assign(col_ids_.begin(), col_ids_.begin() + num_key_columns_);
 }
 
@@ -869,7 +875,7 @@ class Schema {
 const bool use_column_ids = base_schema.has_column_ids() && 
has_column_ids();
 
 int proj_idx = 0;
-for (int i = 0; i < num_columns(); ++i) {
+for (size_t i = 0; i < num_columns(); ++i) {
   const ColumnSchema& col_schema = cols_[i];
 
   // try to lookup the column by ID if present or just by name.



[kudu] 01/02: KUDU-3373 Key provider interface

2022-06-02 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit abe2a73cdbe6438e34f825594d66aec53a329840
Author: Attila Bukor 
AuthorDate: Thu May 26 17:17:37 2022 +0200

KUDU-3373 Key provider interface

Kudu's server keys need to be encrypted on the servers, otherwise its
broken, as an attacker who can access Kudu's disks, can easily steal the
server keys used to encrypt the file keys. The cluster key, which will
be used to encrypt/decrypt the server keys, will live outside the
cluster. This commit introduces a key provider interface to
encrypt/decrypt server keys, with a reference (test-only) implementation
which uses memfrob() (a GNU C function that XORs an array with 42). A
follow-up commit will introduce a production-ready implementation that
uses Apache Ranger KMS to provide the keys.

Change-Id: Ie6ccc05fb991f0fd5cbcd8a49f5b23286d1094ac
Reviewed-on: http://gerrit.cloudera.org:8080/18568
Reviewed-by: Alexey Serbin 
Tested-by: Attila Bukor 
Reviewed-by: Zoltan Chovan 
---
 src/kudu/fs/fs_manager.cc  | 26 ++---
 src/kudu/fs/fs_manager.h   |  8 
 src/kudu/mini-cluster/external_mini_cluster.cc | 11 --
 src/kudu/mini-cluster/external_mini_cluster.h  |  6 +++
 src/kudu/server/CMakeLists.txt |  3 ++
 src/kudu/server/default_key_provider-test.cc   | 48 +++
 src/kudu/server/default_key_provider.h | 53 ++
 src/kudu/server/key_provider.h | 41 
 src/kudu/tools/tool_action_common.cc   |  4 +-
 src/kudu/util/env.h|  3 +-
 src/kudu/util/env_posix.cc |  2 +-
 src/kudu/util/test_util.cc |  2 +-
 12 files changed, 193 insertions(+), 14 deletions(-)

diff --git a/src/kudu/fs/fs_manager.cc b/src/kudu/fs/fs_manager.cc
index 56315592d..425fb428a 100644
--- a/src/kudu/fs/fs_manager.cc
+++ b/src/kudu/fs/fs_manager.cc
@@ -49,6 +49,8 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
 #include "kudu/gutil/walltime.h"
+#include "kudu/server/default_key_provider.h"
+#include "kudu/server/key_provider.h"
 #include "kudu/util/env_util.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/metrics.h"
@@ -144,6 +146,7 @@ using kudu::fs::ReadableBlock;
 using kudu::fs::UpdateInstanceBehavior;
 using kudu::fs::WritableBlock;
 using kudu::pb_util::SecureDebugString;
+using kudu::security::DefaultKeyProvider;
 using std::ostream;
 using std::string;
 using std::unique_ptr;
@@ -193,6 +196,9 @@ FsManager::FsManager(Env* env, FsManagerOpts opts)
 meta_on_xfs_(false) {
   DCHECK(opts_.update_instances == UpdateInstanceBehavior::DONT_UPDATE ||
  !opts_.read_only) << "FsManager can only be for updated if not in 
read-only mode";
+  if (FLAGS_encrypt_data_at_rest) {
+key_provider_.reset(new DefaultKeyProvider());
+  }
 }
 
 FsManager::~FsManager() {}
@@ -457,10 +463,14 @@ Status FsManager::Open(FsReport* report, Timer* 
read_instance_metadata_files,
 read_instance_metadata_files->Stop();
   }
 
-  if (!server_key().empty()) {
-env_->SetEncryptionKey(server_key().length() * 4,
-   reinterpret_cast(
- strings::a2b_hex(server_key()).c_str()));
+  if (!server_key().empty() && key_provider_) {
+string server_key;
+RETURN_NOT_OK(key_provider_->DecryptServerKey(this->server_key(), 
_key));
+// 'server_key' is a hexadecimal string and SetEncryptionKey expects bits
+// (hex / 2 = bytes * 8 = bits).
+env_->SetEncryptionKey(reinterpret_cast(
+ strings::a2b_hex(server_key).c_str()),
+   server_key.length() * 4);
   }
 
   // Open the directory manager if it has not been opened already.
@@ -672,14 +682,18 @@ Status 
FsManager::CreateInstanceMetadata(boost::optional uuid,
 metadata->set_uuid(oid_generator_.Next());
   }
   if (server_key) {
-metadata->set_server_key(server_key.get());
+RETURN_NOT_OK(key_provider_->EncryptServerKey(server_key.get(),
+  
metadata->mutable_server_key()));
   } else if (FLAGS_encrypt_data_at_rest) {
 uint8_t key_bytes[32];
 int num_bytes = FLAGS_encryption_key_length / 8;
 DCHECK(num_bytes <= sizeof(key_bytes));
 OPENSSL_RET_NOT_OK(RAND_bytes(key_bytes, num_bytes),
"Failed to generate random key");
-strings::b2a_hex(key_bytes, metadata->mutable_server_key(), num_bytes);
+string plain_server_key;
+strings::b2a_hex(key_bytes, _server_key, num_bytes);
+RETURN_NOT_OK(key_provider_-

[kudu] branch master updated: KUDU-2671 fix the check for hash dimensions uniformity

2022-05-23 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new ce499cec3 KUDU-2671 fix the check for hash dimensions uniformity
ce499cec3 is described below

commit ce499cec329874db03317ad57a165959fb8893b4
Author: Alexey Serbin 
AuthorDate: Mon May 16 17:23:09 2022 -0700

KUDU-2671 fix the check for hash dimensions uniformity

Before this patch, it was possible to create a table without ranges
using the table-wide hash schema and ranges with custom hash schemas
even if the table-wide hash schema had number of hash dimensions
different from per-range custom hash schemas.

This patch fixes the bug.  I added a corresponding test scenario,
updated existing ones, and removed a duplicate one.

This is a follow-up to 6998193e69eeda497f912d1d806470c95b591ad4.

Change-Id: I8c1282973deba57269f6e962be77e27baa39b187
Reviewed-on: http://gerrit.cloudera.org:8080/18532
Reviewed-by: Mahesh Reddy 
Tested-by: Alexey Serbin 
Reviewed-by: Attila Bukor 
---
 src/kudu/client/flex_partitioning_client-test.cc | 89 
 src/kudu/master/catalog_manager.cc   |  5 +-
 2 files changed, 64 insertions(+), 30 deletions(-)

diff --git a/src/kudu/client/flex_partitioning_client-test.cc 
b/src/kudu/client/flex_partitioning_client-test.cc
index e8e97e4ba..0c72faa2e 100644
--- a/src/kudu/client/flex_partitioning_client-test.cc
+++ b/src/kudu/client/flex_partitioning_client-test.cc
@@ -270,29 +270,6 @@ class FlexPartitioningTest : public KuduTest {
 // specified when creating a table.
 class FlexPartitioningCreateTableTest : public FlexPartitioningTest {};
 
-// Create tables with range partitions using custom hash bucket schemas only.
-//
-// TODO(aserbin): turn the sub-scenarios with non-primary-key columns for
-//custom hash buckets into negative ones after proper
-//checks are added at the server side
-// TODO(aserbin): add verification based on PartitionSchema provided by
-//KuduTable::partition_schema() once PartitionPruner
-//recognized custom hash bucket schema for ranges
-TEST_F(FlexPartitioningCreateTableTest, CustomHashSchema) {
-  // One-level hash bucket structure: { 3, "key" }.
-  {
-constexpr const char* const kTableName = "3@key";
-RangePartitions partitions;
-partitions.emplace_back(CreateRangePartition(0, 100));
-auto& p = partitions.back();
-ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 3, 0));
-ASSERT_OK(CreateTable(kTableName, std::move(partitions)));
-NO_FATALS(CheckTabletCount(kTableName, 3));
-ASSERT_OK(InsertTestRows(kTableName, 0, 100));
-NO_FATALS(CheckTableRowsNum(kTableName, 100));
-  }
-}
-
 TEST_F(FlexPartitioningCreateTableTest, TableWideHashSchema) {
   // Create a table with the following partitions:
   //
@@ -359,6 +336,23 @@ TEST_F(FlexPartitioningCreateTableTest, 
EmptyTableWideHashSchema) {
   //NO_FATALS(CheckTableRowsNum(kTableName, 333));
 }
 
+// Create tables with range partitions using custom hash bucket schemas only.
+TEST_F(FlexPartitioningCreateTableTest, CustomHashSchemaDiffersFromTableWide) {
+  // Using one-level hash bucketing { 3, "key" } as the custom hash schema
+  // for the newly created range partition. Note that the table-wide hash 
schema
+  // is empty per FlexPartitioningTest::FlexPartitioningTest(), so the attempt
+  // to create such a table fails with the Status::NotSupported() status.
+  constexpr const char* const kTableName = "3@key";
+  RangePartitions partitions;
+  partitions.emplace_back(CreateRangePartition(0, 100));
+  auto& p = partitions.back();
+  ASSERT_OK(p->add_hash_partitions({ kKeyColumn }, 3, 0));
+  const auto s = CreateTable(kTableName, std::move(partitions));
+  ASSERT_TRUE(s.IsNotSupported()) << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(),
+  "varying number of hash dimensions per range is not yet supported");
+}
+
 // TODO(aserbin): re-enable this scenario once varying hash dimensions per 
range
 //are supported
 TEST_F(FlexPartitioningCreateTableTest, 
DISABLED_SingleCustomRangeEmptyHashSchema) {
@@ -579,6 +573,7 @@ std::ostream& operator<<(std::ostream& os,
 struct PerRangeTestParameters {
   bool allowed;
   TablePartitionParameters partition_params;
+  vector table_wide_hash_schema_params;
 };
 std::ostream& operator<<(std::ostream& os,
  const PerRangeTestParameters& params) {
@@ -619,6 +614,11 @@ class FlexPartitioningCreateTableParamTest :
   table_creator->add_custom_range_partition(p.release());
 }
 
+for (const auto& table_wide_params :

[kudu] branch master updated: [test] fix ASAN test failures in master-stress-test

2022-05-16 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new e2f0d8fb0 [test] fix ASAN test failures in master-stress-test
e2f0d8fb0 is described below

commit e2f0d8fb056d9834199ec2feaba30daefae7
Author: zhangyifan27 
AuthorDate: Thu May 5 20:30:10 2022 +0800

[test] fix ASAN test failures in master-stress-test

The MasterStressTest occasionally failed in ASAN builds [1] [2]. I also 
built
the test in ASAN mode on a CentOS 7.9 machine, ran it in slow mode 3 times 
and
all failed.

It seems that some RetryingTSRpcTask is still running after 'TableInfo' 
object
destroyed if we set 
--enable_metadata_cleanup_for_deleted_tables_and_tablets=true.
To fix heap-use-after-free errors, we should abort and wait all pending 
tasks
completed before we destroy a TabletInfo object. With this fix the ASAN test
can pass 10 times.

[1] http://jenkins.kudu.apache.org/job/kudu-gerrit/25414/BUILD_TYPE=ASAN/
[2] http://jenkins.kudu.apache.org/job/kudu-gerrit/25448/BUILD_TYPE=ASAN/

Change-Id: I4eff45fbf05644362169485cd32678509eab1b07
Reviewed-on: http://gerrit.cloudera.org:8080/18501
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin 
---
 src/kudu/master/catalog_manager.cc | 49 ++
 1 file changed, 28 insertions(+), 21 deletions(-)

diff --git a/src/kudu/master/catalog_manager.cc 
b/src/kudu/master/catalog_manager.cc
index 7af3873ba..a1cf87ac8 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -3845,14 +3845,17 @@ class PickLeaderReplica : public TSPicker {
 //
 // The target tablet server is refreshed before each RPC by consulting the 
provided
 // TSPicker implementation.
+// Each created RetryingTSRpcTask should be added to TableInfo::pending_tasks_ 
by
+// calling TableInfo::AddTask(), so 'table' must remain valid for the lifetime 
of
+// this class.
 class RetryingTSRpcTask : public MonitoredTask {
  public:
-  RetryingTSRpcTask(Master *master,
+  RetryingTSRpcTask(Master* master,
 unique_ptr replica_picker,
-scoped_refptr table)
+TableInfo* table)
 : master_(master),
   replica_picker_(std::move(replica_picker)),
-  table_(std::move(table)),
+  table_(table),
   start_ts_(MonoTime::Now()),
   deadline_(start_ts_ + 
MonoDelta::FromMilliseconds(FLAGS_unresponsive_ts_rpc_timeout_ms)),
   attempt_(0),
@@ -3876,7 +3879,7 @@ class RetryingTSRpcTask : public MonitoredTask {
 
   MonoTime start_timestamp() const override { return start_ts_; }
   MonoTime completion_timestamp() const override { return end_ts_; }
-  const scoped_refptr& table() const { return table_ ; }
+  TableInfo* table() const { return table_; }
 
  protected:
   // Send an RPC request and register a callback.
@@ -3919,7 +3922,8 @@ class RetryingTSRpcTask : public MonitoredTask {
 
   Master * const master_;
   const unique_ptr replica_picker_;
-  const scoped_refptr table_;
+  // RetryingTSRpcTask is owned by 'TableInfo', so the backpointer should be 
raw.
+  TableInfo* table_;
 
   MonoTime start_ts_;
   MonoTime end_ts_;
@@ -4091,7 +4095,7 @@ class RetrySpecificTSRpcTask : public RetryingTSRpcTask {
  public:
   RetrySpecificTSRpcTask(Master* master,
  const string& permanent_uuid,
- const scoped_refptr& table)
+ TableInfo* table)
 : RetryingTSRpcTask(master,
 unique_ptr(new 
PickSpecificUUID(permanent_uuid)),
 table),
@@ -4113,7 +4117,7 @@ class AsyncCreateReplica : public RetrySpecificTSRpcTask {
  const string& permanent_uuid,
  const scoped_refptr& tablet,
  const TabletMetadataLock& tablet_lock)
-: RetrySpecificTSRpcTask(master, permanent_uuid, tablet->table()),
+: RetrySpecificTSRpcTask(master, permanent_uuid, tablet->table().get()),
   tablet_id_(tablet->id()) {
 deadline_ = start_ts_ + 
MonoDelta::FromMilliseconds(FLAGS_tablet_creation_timeout_ms);
 
@@ -4179,17 +4183,17 @@ class AsyncCreateReplica : public 
RetrySpecificTSRpcTask {
 // Send a DeleteTablet() RPC request.
 class AsyncDeleteReplica : public RetrySpecificTSRpcTask {
  public:
-  AsyncDeleteReplica(
-  Master* master, const string& permanent_uuid,
-  const scoped_refptr& table, string tablet_id,
-  TabletDataState delete_type,
-  optional cas_config_opid_index_less_or_equal,
-  string reason)
+  AsyncDeleteReplica(Master* master,
+ const string& permanent_uuid,
+ TableInfo* table,
+ string tablet_id,
+ TabletDataStat

[kudu] branch master updated: KUDU-2671 implement decoding of RowOperationsPB in Java client

2022-05-11 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 1e19a6822 KUDU-2671 implement decoding of RowOperationsPB in Java 
client
1e19a6822 is described below

commit 1e19a68221fbdc4151cac077f6a147e415430c30
Author: Alexey Serbin 
AuthorDate: Fri Apr 29 22:01:08 2022 -0700

KUDU-2671 implement decoding of RowOperationsPB in Java client

This patch implements decoding of range partitioning information
out of RowOperationsPB.  A few tests added as well to cover the
new functionality.

In the context of supporting custom hash schemas per range in the
Kudu Java client, a few follow-up changelists need the newly introduced
functionality.

Change-Id: I4f69f89f6b9e47d79b83c2109d85a95288bec380
Reviewed-on: http://gerrit.cloudera.org:8080/18494
Reviewed-by: Attila Bukor 
Reviewed-by: Mahesh Reddy 
Tested-by: Alexey Serbin 
---
 .../src/main/java/org/apache/kudu/Type.java|  17 +-
 .../java/org/apache/kudu/client/KeyEncoder.java|  15 +-
 .../java/org/apache/kudu/client/Operation.java | 164 +-
 .../java/org/apache/kudu/client/TestOperation.java | 330 -
 4 files changed, 506 insertions(+), 20 deletions(-)

diff --git a/java/kudu-client/src/main/java/org/apache/kudu/Type.java 
b/java/kudu-client/src/main/java/org/apache/kudu/Type.java
index 5a346c39c..9d550b66c 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/Type.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/Type.java
@@ -163,8 +163,8 @@ public enum Type {
   case DOUBLE:
   case UNIXTIME_MICROS:
 return Longs.BYTES;
-  default: throw new IllegalArgumentException("The provided data type 
doesn't map" +
-  " to know any known one.");
+  default: throw new IllegalArgumentException(
+  "the provided data type doesn't map to any known one");
 }
   }
 
@@ -205,9 +205,8 @@ public enum Type {
   case DECIMAL128:
 return DECIMAL;
   default:
-throw new IllegalArgumentException("The provided data type doesn't 
map" +
-" to know any known one: " + 
type.getDescriptorForType().getFullName());
-
+throw new IllegalArgumentException("the provided data type doesn't map 
" +
+"to any known one: " + type.getDescriptorForType().getFullName());
 }
   }
 
@@ -228,4 +227,10 @@ public enum Type {
 throw new IllegalArgumentException("The provided name doesn't map to any 
known type: " + name);
   }
 
-}
+  /**
+   * @return true if this type has a pre-determined fixed size, false otherwise
+   */
+  public boolean isFixedSize() {
+return this != BINARY && this != STRING && this != VARCHAR;
+  }
+}
\ No newline at end of file
diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/KeyEncoder.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/KeyEncoder.java
index fd3ec6997..0e3500e22 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/KeyEncoder.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/KeyEncoder.java
@@ -145,15 +145,14 @@ class KeyEncoder {
   throw new IllegalStateException(String.format("Primary key column %s is 
not set",
 column.getName()));
 }
-final Type type = column.getType();
-if (type == Type.STRING || type == Type.BINARY ||
-type == Type.VARCHAR) {
-  encodeBinary(row.getVarLengthData().get(columnIdx), isLast, buf);
+if (column.getType().isFixedSize()) {
+  encodeSignedInt(
+  row.getRowAlloc(),
+  schema.getColumnOffset(columnIdx),
+  column.getTypeSize(),
+  buf);
 } else {
-  encodeSignedInt(row.getRowAlloc(),
-  schema.getColumnOffset(columnIdx),
-  column.getTypeSize(),
-  buf);
+  encodeBinary(row.getVarLengthData().get(columnIdx), isLast, buf);
 }
   }
 
diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/Operation.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/Operation.java
index 0ed9caca2..0b2b933fa 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/Operation.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/Operation.java
@@ -407,8 +407,9 @@ public abstract class Operation extends 
KuduRpc {
 ColumnSchema col = schema.getColumnByIndex(colIdx);
 // Keys should always be specified, maybe check?
 if (row.isSet(colIdx) && !row.isSetToNull(colIdx)) {
-  if (col.getType() == Type.STRING || col.getType() == Type.BINARY ||
-  col.getType() == Type.VARCHAR) {
+  if (col

[kudu] 01/03: [client] prohibit copying/assigning of ResourceMetrics

2022-05-10 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 50395ff74cb47e9ac126e4bca38de1d7e48511c3
Author: Alexey Serbin 
AuthorDate: Mon May 9 18:50:26 2022 -0700

[client] prohibit copying/assigning of ResourceMetrics

Since ResourceMetrics contains a raw pointer, it makes it dangerous
since its auto-generated copy constructor and assignment operator
are not private.  Somebody could write something like

  {
KuduScanner scanner();
{
  ResouceMetrics m = session->GetResourceMetrics();
  ...
}
// Continue scanning activity with the 'scanner'.
...
  }

  or

  {
KuduSession session(...);
{
  ResouceMetrics m = session->GetWriteOpMetrics();
  ...
}
// Continue writing activity with the 'session'.
...
  }

and hit a silent memory corruption issue since that's the use-after-free
condition for the KuduScanner and KuduSession instances when the data
behind the raw pointer in ResourceMetrics::data_ is accessed later on.

This patch breaks the ABI compatibility for the kudu_client C++ library,
but this would surface only if there is a dangerous code like above
in the Kudu application linked with the library.  It's much worse having
such a hidden memory corruption issue in an application than hitting
a build breakage due to a linkage error which is quite simple to fix.

This is a follow-up to ece7b5653998db318e4baa5d57f27ba3a836731d.

Change-Id: I602cc4e194a975752687d13d525e44043955a5cf
Reviewed-on: http://gerrit.cloudera.org:8080/18510
Tested-by: Alexey Serbin 
Reviewed-by: Riza Suminto 
Reviewed-by: Attila Bukor 
---
 src/kudu/client/resource_metrics.h | 8 ++--
 1 file changed, 6 insertions(+), 2 deletions(-)

diff --git a/src/kudu/client/resource_metrics.h 
b/src/kudu/client/resource_metrics.h
index dff31fedc..1b57dada3 100644
--- a/src/kudu/client/resource_metrics.h
+++ b/src/kudu/client/resource_metrics.h
@@ -17,8 +17,9 @@
 #ifndef KUDU_CLIENT_RESOURCE_METRICS_H
 #define KUDU_CLIENT_RESOURCE_METRICS_H
 
-// NOTE: using stdint.h instead of cstdint because this file is supposed
-//   to be processed by a compiler lacking C++11 support.
+// NOTE: using stdint.h instead of cstdint because this file might be
+//   processed by a compiler lacking C++11 support (Kudu C++ client API
+//   still keeps C++98 compatibility)
 #include 
 
 #include 
@@ -27,6 +28,7 @@
 #include "kudu/util/kudu_export.h"
 
 #ifdef KUDU_HEADERS_NO_STUBS
+#include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
 #else
 #include "kudu/client/stubs.h"
@@ -68,6 +70,8 @@ class KUDU_EXPORT ResourceMetrics {
   friend class KuduSession;
   class KUDU_NO_EXPORT Data;
   Data* data_;
+
+  DISALLOW_COPY_AND_ASSIGN(ResourceMetrics);
 };
 
 } // namespace client



[kudu] 02/03: [client] add DISALLOW_COPY_AND_ASSIGN() for a few classes

2022-05-10 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 948417a460587a655773ed4bb1869b4bc5aa83f8
Author: Alexey Serbin 
AuthorDate: Mon May 9 20:03:08 2022 -0700

[client] add DISALLOW_COPY_AND_ASSIGN() for a few classes

After making the copy constructor and the assignment operator for the
ResourceMetrics class private, I took a quick look at the other exported
classes in the Kudu C++ client API and added corresponding macro
where appropriate.

Strictly speaking, this change breaks the ABI compatibility for the
Kudu C++ client API, but a code that would use these privatized members
was unsafe anyways.  I guess it's easier to fix the linker error after
upgrading to a newer version of the Kudu client library than have
a hidden memory corruption problem in a Kudu C++ application.

Change-Id: I5369760db3040c0357517903dab6ff4e2acb7656
Reviewed-on: http://gerrit.cloudera.org:8080/18511
Tested-by: Kudu Jenkins
Reviewed-by: Riza Suminto 
Reviewed-by: Attila Bukor 
---
 src/kudu/client/client.h | 2 ++
 src/kudu/client/schema.h | 5 +
 2 files changed, 7 insertions(+)

diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index bf94a7b26..65980a075 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -3359,6 +3359,8 @@ class KUDU_EXPORT KuduPartitioner {
 
   explicit KuduPartitioner(Data* data);
   Data* data_; // Owned.
+
+  DISALLOW_COPY_AND_ASSIGN(KuduPartitioner);
 };
 
 
diff --git a/src/kudu/client/schema.h b/src/kudu/client/schema.h
index 6791e5674..2fae75907 100644
--- a/src/kudu/client/schema.h
+++ b/src/kudu/client/schema.h
@@ -29,6 +29,7 @@
 #ifdef KUDU_HEADERS_NO_STUBS
 #include 
 
+#include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
 #else
 #include "kudu/client/stubs.h"
@@ -552,6 +553,8 @@ class KUDU_EXPORT KuduColumnSpec {
 
   // Owned.
   Data* data_;
+
+  DISALLOW_COPY_AND_ASSIGN(KuduColumnSpec);
 };
 
 /// @brief Builder API for constructing a KuduSchema object.
@@ -613,6 +616,8 @@ class KUDU_EXPORT KuduSchemaBuilder {
 
   // Owned.
   Data* data_;
+
+  DISALLOW_COPY_AND_ASSIGN(KuduSchemaBuilder);
 };
 
 /// @brief A representation of a table's schema.



[kudu] branch master updated (2b2a19907 -> 09a6c8833)

2022-05-10 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


from 2b2a19907 [docs] simplify multi-master migration instructions
 new 50395ff74 [client] prohibit copying/assigning of ResourceMetrics
 new 948417a46 [client] add DISALLOW_COPY_AND_ASSIGN() for a few classes
 new 09a6c8833 KUDU-2671 relax requirement for partition range specification

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/kudu/client/client.h|  2 ++
 src/kudu/client/resource_metrics.h  |  8 ++--
 src/kudu/client/schema.h|  5 +
 src/kudu/master/catalog_manager-test.cc | 21 -
 src/kudu/master/catalog_manager.cc  | 30 +++---
 5 files changed, 52 insertions(+), 14 deletions(-)



[kudu] 03/03: KUDU-2671 relax requirement for partition range specification

2022-05-10 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 09a6c8833e085dd46c18acec611191e6374c73b5
Author: Alexey Serbin 
AuthorDate: Mon May 9 14:21:37 2022 -0700

KUDU-2671 relax requirement for partition range specification

As of now (see [1]), the number of hash dimensions in a per-range custom
hash schema must be the same for all the ranges in a table.  With that,
it's possible to relax the requirements on specifying partition key
ranges for a GetTableLocations RPC.  That allows legacy Kudu clients
to work with tables having custom hash schemas per range.  If necessary,
it's possible to make the requirements stricter by setting
--require_new_spec_for_custom_hash_schema_range_bound=true.

This is a follow-up to [1].

[1] 
https://github.com/apache/kudu/commit/6998193e69eeda497f912d1d806470c95b591ad4

Change-Id: Idfdae92f6baee5e8f197cba41c8e51120f9b4d58
Reviewed-on: http://gerrit.cloudera.org:8080/18509
Reviewed-by: Attila Bukor 
Tested-by: Alexey Serbin 
---
 src/kudu/master/catalog_manager-test.cc | 21 -
 src/kudu/master/catalog_manager.cc  | 30 +++---
 2 files changed, 39 insertions(+), 12 deletions(-)

diff --git a/src/kudu/master/catalog_manager-test.cc 
b/src/kudu/master/catalog_manager-test.cc
index 38db44473..96898656e 100644
--- a/src/kudu/master/catalog_manager-test.cc
+++ b/src/kudu/master/catalog_manager-test.cc
@@ -23,6 +23,7 @@
 #include 
 
 #include 
+#include 
 #include 
 
 #include "kudu/common/common.pb.h"
@@ -37,6 +38,8 @@
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
+DECLARE_bool(require_new_spec_for_custom_hash_schema_range_bound);
+
 using std::iota;
 using std::string;
 using std::vector;
@@ -134,8 +137,24 @@ TEST(TableInfoTest, 
GetTableLocationsLegacyCustomHashSchemas) {
   table->AddRemoveTablets({ tablet }, {});
 
   // Query by specifying the start of the partition via the partition_key_start
-  // field: it should fail since the table has a range with custom hash schema.
+  // field: it should pass even if the table has a range with custom hash 
schema
+  // since as of now all the range partitions must have the number of hash
+  // dimenstions fixed across all the ranges in a table.
+  {
+GetTableLocationsRequestPB req;
+req.set_max_returned_locations(1);
+req.mutable_table()->mutable_table_name()->assign(table_id);
+req.mutable_partition_key_start()->assign("a");
+vector> tablets_in_range;
+ASSERT_OK(table->GetTabletsInRange(, _in_range));
+ASSERT_EQ(1, tablets_in_range.size());
+  }
+
+  // Query by specifying the start of the partition via the partition_key_start
+  // field: it should fail since the table has a range with custom hash schema
+  // and --require_new_spec_for_custom_hash_schema_range_bound=true.
   {
+FLAGS_require_new_spec_for_custom_hash_schema_range_bound = true;
 GetTableLocationsRequestPB req;
 req.set_max_returned_locations(1);
 req.mutable_table()->mutable_table_name()->assign(table_id);
diff --git a/src/kudu/master/catalog_manager.cc 
b/src/kudu/master/catalog_manager.cc
index f16ae3d3a..90511d145 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -397,6 +397,13 @@ DEFINE_bool(enable_chunked_tablet_writes, true,
 TAG_FLAG(enable_chunked_tablet_writes, experimental);
 TAG_FLAG(enable_chunked_tablet_writes, runtime);
 
+DEFINE_bool(require_new_spec_for_custom_hash_schema_range_bound, false,
+"Whether to require the client to use newer signature to specify "
+"range bounds when working with a table having custom hash schema "
+"per range");
+TAG_FLAG(require_new_spec_for_custom_hash_schema_range_bound, experimental);
+TAG_FLAG(require_new_spec_for_custom_hash_schema_range_bound, runtime);
+
 DECLARE_bool(raft_prepare_replacement_before_eviction);
 DECLARE_int64(tsk_rotation_seconds);
 DECLARE_string(ranger_config_path);
@@ -807,7 +814,7 @@ void CatalogManagerBgTasks::Run() {
   catalog_manager_->ExtractDeletedTablesAndTablets(_tables, 
_tablets);
   Status s = Status::OK();
   // Clean up metadata for deleted tablets first and then clean up 
metadata for deleted
-  // tables. This is the reverse of the order in which we load them. 
So for any remaining
+  // tables. This is the reverse of the order in which we load them. 
So for any remaining
   // tablet, the metadata of the table to which it belongs must exist.
   const time_t now = time(nullptr);
   if (!deleted_tablets.empty()) {
@@ -6557,6 +6564,11 @@ Status TableInfo::GetTabletsInRange(
 const GetTableLocationsRequestPB* req,
 vector>* ret) co

[kudu] 02/02: [docs] simplify multi-master migration instructions

2022-05-10 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 2b2a199077a2eb6e813a4b33601477a25491049e
Author: Alexey Serbin 
AuthorDate: Tue May 3 16:58:33 2022 -0700

[docs] simplify multi-master migration instructions

This patch updates the instructions for the multi-master migration
procedure by removing an extra conditional step based on the choice
whether DNS aliases were configured or not.

Change-Id: Ie9c530c194b40d658c7764dd553c5b16806dca90
Reviewed-on: http://gerrit.cloudera.org:8080/18500
Tested-by: Kudu Jenkins
Reviewed-by: Attila Bukor 
---
 docs/administration.adoc | 63 +++-
 1 file changed, 30 insertions(+), 33 deletions(-)

diff --git a/docs/administration.adoc b/docs/administration.adoc
index b7c0c0306..7aad2731a 100644
--- a/docs/administration.adoc
+++ b/docs/administration.adoc
@@ -553,31 +553,14 @@ checked using the `kudu master get_flags` command.
 If not specified, supply `--master_addresses=:` to master's 
configuration
 and restart the single master.
 
-* Optional: configure a DNS alias for the master. The alias could be a DNS 
cname (if the machine
-  already has an A record in DNS), an A record (if the machine is only known 
by its IP address),
-  or an alias in /etc/hosts. The alias should be an abstract representation of 
the master (e.g.
-  `master-1`).
+* Optional: configure a DNS alias for the master. The alias is a DNS CNAME
+record.
 +
 WARNING: Without DNS aliases it is not possible to recover from permanent
 master failures without restarting the tablet servers in the cluster
 to pick up the replacement master node at different hostname. It is highly
 recommended to use DNS aliases for Kudu master nodes to avoid that.
 +
-. If you have Kudu tables that are accessed from Impala, you must update
-the master addresses in the Apache Hive Metastore (HMS) database.
-* If you set up the DNS aliases, run the following statement in `impala-shell`,
-replacing `master-1` and `master-2` with your actual aliases.
-+
-[source,sql]
-
-ALTER TABLE table_name
-SET TBLPROPERTIES
-('kudu.master_addresses' = 'master-1,master-2');
-
-+
-* If you do not have DNS aliases set up, see Step #7 in the Performing
-the migration section for updating HMS.
-+
 . Perform the following preparatory steps for each new master:
 * Choose an unused machine in the cluster. The master generates very little 
load
   so it can be collocated with other data services or load-generating 
processes,
@@ -586,7 +569,6 @@ the migration section for updating HMS.
   `kudu-master` packages should be installed), or via some other means.
 * Choose and record the directory where the master's data will live.
 * Choose and record the port the master should use for RPCs.
-* Optional: configure a DNS alias for the master (e.g. `master-2`, `master-3`, 
etc).
 
 [[perform-the-migration]]
  Perform the migration
@@ -626,26 +608,42 @@ port:: master's previously recorded RPC port number
 
 . Restart the existing masters one by one.
 . Start the new master.
-. Modify the value of the `tserver_master_addrs` configuration parameter for 
each
-tablet server. The new value must be a comma-separated list of masters where 
each entry is a string
-of the form `:`
-hostname:: master's previously recorded hostname or alias
-port:: master's previously recorded RPC port number
-. Restart all the tablet servers to pick up the new master configuration.
-. If you have Kudu tables that are accessed from Impala and you didn't set up
-DNS aliases, update the HMS database manually in the underlying database that
-provides the storage for HMS.
-* The following is an example SQL statement you should run in the HMS database:
+. Repeat the steps above to add the desired number of masters into the cluster
+(for example, in case of adding two extra masters `master-2` and `master-3`,
+the sequence of those steps should be run for `master-2` and then for
+`master-3`).
+. *After adding all the desired masters into the cluster*, modify the
+value of the `tserver_master_addrs` configuration parameter for each tablet
+server. The new value must be a comma-separated list of masters where each
+entry is a string of the form `:`
+hostname:: master's hostname
+port:: master's RPC port number
+. Restart all the tablet servers to pick up the new masters' configuration.
+. If you have Kudu tables that are accessed from Impala, update the master
+addresses in the Apache Hive Metastore (HMS) database.
+* The following is an example SQL statement you should run manually in the
+underlying database that provides the storage for HMS (e.g., PostgreSQL)
+after migrating from one to three masters in a Kudu cluster:
 +
 [source,sql]
 
 UPDATE TABLE_PARAMS
 SET PARAM_VALUE =
-  'master-1.example.com,master-2.example.com'
+  'master-1.example.com,master-2.example.com

[kudu] branch master updated (becef6270 -> 2b2a19907)

2022-05-10 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


from becef6270 [java] Fix a scan bug which will read repetitive rows.
 new 11db3f28b [master] KUDU-2671: Range specific hashing during table 
alter op.
 new 2b2a19907 [docs] simplify multi-master migration instructions

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 docs/administration.adoc | 63 +---
 src/kudu/master/master.proto |  3 +++
 2 files changed, 33 insertions(+), 33 deletions(-)



[kudu] 01/02: [master] KUDU-2671: Range specific hashing during table alter op.

2022-05-10 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 11db3f28b36d92ce1515bcaace51a3586838abcb
Author: Abhishek Chennaka 
AuthorDate: Tue May 3 12:28:21 2022 -0400

[master] KUDU-2671: Range specific hashing during table alter op.

This commit has the proto file, master.proto, updated to reflect
the changes needed for having custom hash partitioning while
adding a new range to an existing table.
The AlterTableRequestPB has the new field to accommodate the
new change.
The reason for pushing this patch with just this change is to
unblock client side work while the server side work is being
done.

Change-Id: Ifec5566ea8f3e49d00dcb6964b3d17c4be0504eb
Reviewed-on: http://gerrit.cloudera.org:8080/18485
Reviewed-by: Alexey Serbin 
Tested-by: Kudu Jenkins
---
 src/kudu/master/master.proto | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/src/kudu/master/master.proto b/src/kudu/master/master.proto
index bfccafe50..48b004721 100644
--- a/src/kudu/master/master.proto
+++ b/src/kudu/master/master.proto
@@ -732,6 +732,9 @@ message AlterTableRequestPB {
 // The dimension label for the tablet. Used for dimension-specific 
placement
 // of the tablet's replicas.
 optional string dimension_label = 2;
+
+// The custom hash partition schema for the range if specified.
+repeated PartitionSchemaPB.HashBucketSchemaPB custom_hash_schema = 3;
   }
   message DropRangePartition {
 // A set of row operations containing the lower and upper range bound for



[kudu] 02/02: [client] KUDU-3365: Expose INSERT/UPDATE metrics in the Java API

2022-05-05 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 03f266854a6ab7ff1732d380c722ba64c018ecc1
Author: Riza Suminto 
AuthorDate: Tue May 3 17:38:55 2022 -0700

[client] KUDU-3365: Expose INSERT/UPDATE metrics in the Java API

The work done in the scope of KUDU-3351 included the server-side changes
and corresponding changes in the Kudu C++ API to expose the metrics to
the client applications. This patch implement similar changes to expose
such metrics in Java client API.

Change-Id: I956eb0c0a2cadcf3491550630b861bb48462e8eb
Reviewed-on: http://gerrit.cloudera.org:8080/18489
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin 
---
 .../org/apache/kudu/client/AsyncKuduSession.java   | 12 +
 .../main/java/org/apache/kudu/client/Batch.java|  5 +++-
 .../java/org/apache/kudu/client/BatchResponse.java | 17 +++-
 .../java/org/apache/kudu/client/KuduSession.java   |  5 
 .../java/org/apache/kudu/client/Operation.java |  5 +++-
 .../org/apache/kudu/client/OperationResponse.java  | 17 +++-
 .../org/apache/kudu/client/ResourceMetrics.java| 28 +++-
 .../apache/kudu/client/SessionConfiguration.java   |  6 +
 .../org/apache/kudu/client/TestKuduSession.java| 30 ++
 9 files changed, 120 insertions(+), 5 deletions(-)

diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduSession.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduSession.java
index 07397330e..9c0ea6912 100644
--- 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduSession.java
+++ 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduSession.java
@@ -183,6 +183,11 @@ public class AsyncKuduSession implements 
SessionConfiguration {
   private boolean ignoreAllDuplicateRows = false;
   private boolean ignoreAllNotFoundRows = false;
 
+  /**
+   * Cumulative operation metrics since the beginning of the session.
+   */
+  private final ResourceMetrics writeOpMetrics = new ResourceMetrics();
+
   /**
* Package-private constructor meant to be used via AsyncKuduClient
* @param client client that creates this session
@@ -317,6 +322,11 @@ public class AsyncKuduSession implements 
SessionConfiguration {
 return errorCollector.getErrors();
   }
 
+  @Override
+  public ResourceMetrics getWriteOpMetrics() {
+return this.writeOpMetrics;
+  }
+
   /**
* Flushes the buffered operations and marks this session as closed.
* See the javadoc on {@link #flush()} on how to deal with exceptions coming 
out of this method.
@@ -452,6 +462,7 @@ public class AsyncKuduSession implements 
SessionConfiguration {
 // are visible should the callback interrogate the error collector.
 operationResponse.getOperation().callback(operationResponse);
   }
+  writeOpMetrics.update(response.getWriteOpMetrics());
 
   return response;
 }
@@ -624,6 +635,7 @@ public class AsyncKuduSession implements 
SessionConfiguration {
 return client.sendRpcToTablet(operation)
 .addCallbackDeferring(resp -> {
   client.updateLastPropagatedTimestamp(resp.getWriteTimestampRaw());
+  writeOpMetrics.update(resp.getWriteOpMetrics());
   return Deferred.fromResult(resp);
 })
 .addErrback(new SingleOperationErrCallback(operation));
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/Batch.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/Batch.java
index 0c5c578b6..a248802f8 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/Batch.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/Batch.java
@@ -184,12 +184,15 @@ class Batch extends KuduRpc {
   }
   errorsPB = filteredErrors;
 }
+ResourceMetrics metrics = builder.hasResourceMetrics() ?
+ResourceMetrics.fromResourceMetricsPB(builder.getResourceMetrics()) : 
null;
 BatchResponse response = new 
BatchResponse(timeoutTracker.getElapsedMillis(),
tsUUID,
builder.getTimestamp(),
errorsPB,
operations,
-   operationIndexes);
+   operationIndexes,
+   metrics);
 
 if (injectedError != null) {
   if (injectedlatencyMs > 0) {
diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/BatchResponse.java 
b/java/kudu-client/src/main/java/org/apache/kudu/client/BatchResponse.java
index 24078eb4c..f11f930b3 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/BatchResponse.java
+++ b/java/kudu-client/sr

[kudu] 01/02: [docs] update a few notes on multi-master migration

2022-05-05 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit d411dd1b41a47a5d9f1ee9b68911aa5952475896
Author: Alexey Serbin 
AuthorDate: Tue May 3 14:41:57 2022 -0700

[docs] update a few notes on multi-master migration

This patch updates the warning note for the multi-master migration
section (of course, that's for 1.15 and newer versions) to bring more
clarity on what components to be restarted if not using DNS aliases.
The prior '... bringing the cluster down for maintenance ...' was not
technically correct since it's possible to perform rolling restart of
tablet servers in the cluster.

In addition, the obsolete warning note about the multi-master
migration procedure being unsafe is removed.  The new multi-master
migration procedure (for 1.15 and newer versions) is universal and
safe when migrating from N to N+1 masters.

Change-Id: Ie707e8342adf7a5e557079a4121a50d009f7602f
Reviewed-on: http://gerrit.cloudera.org:8080/18487
Tested-by: Kudu Jenkins
Reviewed-by: Attila Bukor 
---
 docs/administration.adoc | 9 -
 1 file changed, 4 insertions(+), 5 deletions(-)

diff --git a/docs/administration.adoc b/docs/administration.adoc
index 2ba1525ca..b7c0c0306 100644
--- a/docs/administration.adoc
+++ b/docs/administration.adoc
@@ -528,9 +528,6 @@ quorum. The below steps do not need to be performed in 
these versions.
 NOTE: From Kudu version 1.15.0 onwards, the `kudu master add` command has been 
added that simplifies
 the orchestration to migrate an existing Kudu cluster to multiple masters.
 
-WARNING: The workflow is unsafe for adding new masters to an existing 
configuration that already has
-three or more masters. Do not use it for that purpose.
-
 WARNING: An even number of masters doesn't provide any benefit over having one 
fewer masters.  This
 guide should always be used for migrating to three masters.
 
@@ -561,8 +558,10 @@ and restart the single master.
   or an alias in /etc/hosts. The alias should be an abstract representation of 
the master (e.g.
   `master-1`).
 +
-WARNING: Without DNS aliases it is not possible to recover from permanent 
master failures without
-bringing the cluster down for maintenance, and as such, it is highly 
recommended.
+WARNING: Without DNS aliases it is not possible to recover from permanent
+master failures without restarting the tablet servers in the cluster
+to pick up the replacement master node at different hostname. It is highly
+recommended to use DNS aliases for Kudu master nodes to avoid that.
 +
 . If you have Kudu tables that are accessed from Impala, you must update
 the master addresses in the Apache Hive Metastore (HMS) database.



[kudu] branch master updated (9ff042b73 -> 03f266854)

2022-05-05 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


from 9ff042b73 Update RELEASING doc
 new d411dd1b4 [docs] update a few notes on multi-master migration
 new 03f266854 [client] KUDU-3365: Expose INSERT/UPDATE metrics in the Java 
API

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 docs/administration.adoc   |  9 +++
 .../org/apache/kudu/client/AsyncKuduSession.java   | 12 +
 .../main/java/org/apache/kudu/client/Batch.java|  5 +++-
 .../java/org/apache/kudu/client/BatchResponse.java | 17 +++-
 .../java/org/apache/kudu/client/KuduSession.java   |  5 
 .../java/org/apache/kudu/client/Operation.java |  5 +++-
 .../org/apache/kudu/client/OperationResponse.java  | 17 +++-
 .../org/apache/kudu/client/ResourceMetrics.java| 28 +++-
 .../apache/kudu/client/SessionConfiguration.java   |  6 +
 .../org/apache/kudu/client/TestKuduSession.java| 30 ++
 10 files changed, 124 insertions(+), 10 deletions(-)



[kudu] branch master updated: [client] fix toggling fault-tolerance for a scanner

2022-05-02 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 54d0f0b22 [client] fix toggling fault-tolerance for a scanner
54d0f0b22 is described below

commit 54d0f0b227b82a5ae5bb6095877858e6cbb7c9c8
Author: Alexey Serbin 
AuthorDate: Fri Apr 29 17:29:24 2022 -0700

[client] fix toggling fault-tolerance for a scanner

Change-Id: Ic50f9d62623d8b9ced7d854ce0e80a0faaa60ad3
Reviewed-on: http://gerrit.cloudera.org:8080/18462
Reviewed-by: Andrew Wong 
Tested-by: Alexey Serbin 
---
 .../kudu/client/AbstractKuduScannerBuilder.java|  2 +-
 .../java/org/apache/kudu/client/TestScanToken.java | 22 
 src/kudu/client/scan_configuration.cc  |  6 +++--
 src/kudu/client/scan_token-test.cc | 30 ++
 4 files changed, 57 insertions(+), 3 deletions(-)

diff --git 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AbstractKuduScannerBuilder.java
 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AbstractKuduScannerBuilder.java
index 0e6b2d7cc..2c9b4aa36 100644
--- 
a/java/kudu-client/src/main/java/org/apache/kudu/client/AbstractKuduScannerBuilder.java
+++ 
b/java/kudu-client/src/main/java/org/apache/kudu/client/AbstractKuduScannerBuilder.java
@@ -96,7 +96,7 @@ public abstract class AbstractKuduScannerBuilder
   public S setFaultTolerant(boolean isFaultTolerant) {
 this.isFaultTolerant = isFaultTolerant;
 if (isFaultTolerant) {
-  readMode = AsyncKuduScanner.ReadMode.READ_AT_SNAPSHOT;
+  this.readMode = AsyncKuduScanner.ReadMode.READ_AT_SNAPSHOT;
 }
 return (S) this;
   }
diff --git 
a/java/kudu-client/src/test/java/org/apache/kudu/client/TestScanToken.java 
b/java/kudu-client/src/test/java/org/apache/kudu/client/TestScanToken.java
index 3ec8f173b..4ed897085 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestScanToken.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestScanToken.java
@@ -27,6 +27,7 @@ import static 
org.apache.kudu.test.ClientTestUtil.loadDefaultTable;
 import static org.apache.kudu.test.MetricTestUtils.totalRequestCount;
 import static org.apache.kudu.test.MetricTestUtils.validateRequestCount;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -865,4 +866,25 @@ public class TestScanToken {
 KuduScanner scanner = tokens.get(0).intoScanner(newClient);
 assertEquals(1, countRowsInScan(scanner));
   }
+
+  @Test
+  public void testScannerBuilderFaultToleranceToggle() throws IOException {
+KuduTable table = createDefaultTable(client, testTableName);
+KuduScanner.KuduScannerBuilder scannerBuilder =
+new KuduScanner.KuduScannerBuilder(asyncClient, table);
+assertFalse(scannerBuilder.isFaultTolerant);
+assertEquals(AsyncKuduScanner.ReadMode.READ_LATEST, 
scannerBuilder.readMode);
+
+scannerBuilder.setFaultTolerant(true);
+assertTrue(scannerBuilder.isFaultTolerant);
+assertEquals(AsyncKuduScanner.ReadMode.READ_AT_SNAPSHOT, 
scannerBuilder.readMode);
+
+scannerBuilder.setFaultTolerant(false);
+assertFalse(scannerBuilder.isFaultTolerant);
+assertEquals(AsyncKuduScanner.ReadMode.READ_AT_SNAPSHOT, 
scannerBuilder.readMode);
+
+scannerBuilder.readMode(AsyncKuduScanner.ReadMode.READ_YOUR_WRITES);
+assertFalse(scannerBuilder.isFaultTolerant);
+assertEquals(AsyncKuduScanner.ReadMode.READ_YOUR_WRITES, 
scannerBuilder.readMode);
+  }
 }
diff --git a/src/kudu/client/scan_configuration.cc 
b/src/kudu/client/scan_configuration.cc
index 32bc78470..707efc297 100644
--- a/src/kudu/client/scan_configuration.cc
+++ b/src/kudu/client/scan_configuration.cc
@@ -167,8 +167,10 @@ Status 
ScanConfiguration::SetReadMode(KuduScanner::ReadMode read_mode) {
 }
 
 Status ScanConfiguration::SetFaultTolerant(bool fault_tolerant) {
-  RETURN_NOT_OK(SetReadMode(KuduScanner::READ_AT_SNAPSHOT));
-  is_fault_tolerant_ = true;
+  if (fault_tolerant) {
+RETURN_NOT_OK(SetReadMode(KuduScanner::READ_AT_SNAPSHOT));
+  }
+  is_fault_tolerant_ = fault_tolerant;
   return Status::OK();
 }
 
diff --git a/src/kudu/client/scan_token-test.cc 
b/src/kudu/client/scan_token-test.cc
index ac75b4503..77c00b99b 100644
--- a/src/kudu/client/scan_token-test.cc
+++ b/src/kudu/client/scan_token-test.cc
@@ -1060,5 +1060,35 @@ TEST_P(StaleScanTokensParamTest, DroppingFirstRange) {
 INSTANTIATE_TEST_SUITE_P(FirstRangeDropped, StaleScanTokensParamTest,
  testing::Range(FirstRangeChangeMode::BEGIN,
 FirstRangeChangeMode::END));
+
+TEST_F(ScanTokenTest, ToggleFaultToleranceForScanConfiguration) {
+  constexpr const char* const

[kudu] branch master updated: [client] KUDU-3351: Add ResourceMetricsPB into WriteResponsePB

2022-04-29 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 0ddcaaabc [client] KUDU-3351: Add ResourceMetricsPB into 
WriteResponsePB
0ddcaaabc is described below

commit 0ddcaaabc97c85a4715ae79ff5604feb9b342779
Author: Riza Suminto 
AuthorDate: Tue Apr 26 14:35:19 2022 -0700

[client] KUDU-3351: Add ResourceMetricsPB into WriteResponsePB

KUDU-1563 adds support for INSERT_IGNORE, UPDATE_IGNORE, and
DELETE_IGNORE. However, it was lacking the per-session metrics about how
many rows get ignored vs modified. This patch implements the per-session
metrics by introducing a new ResourceMetricsPB field into the
WriteResponsePB that's populated in every response sent back to the
client.

Change-Id: I9adefd64b0058c66274a00e1b12334653fcab2b3
Reviewed-on: http://gerrit.cloudera.org:8080/18451
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin 
---
 src/kudu/client/batcher.cc |  5 
 src/kudu/client/client-test.cc | 33 ++
 src/kudu/client/client.cc  |  4 +++
 src/kudu/client/client.h   |  3 ++
 src/kudu/client/resource_metrics.h |  1 +
 src/kudu/client/session-internal.cc| 23 ++-
 src/kudu/client/session-internal.h | 11 
 .../integration-tests/exactly_once_writes-itest.cc |  8 ++
 src/kudu/tablet/ops/write_op.cc| 23 ++-
 src/kudu/tablet/ops/write_op.h |  5 
 src/kudu/tserver/tserver.proto | 24 
 11 files changed, 138 insertions(+), 2 deletions(-)

diff --git a/src/kudu/client/batcher.cc b/src/kudu/client/batcher.cc
index 2db5033b0..cfde2eca3 100644
--- a/src/kudu/client/batcher.cc
+++ b/src/kudu/client/batcher.cc
@@ -966,6 +966,11 @@ void Batcher::ProcessWriteResponse(const WriteRpc& rpc,
 had_errors_ = true;
   }
 
+  // Collect metrics
+  if (sp::shared_ptr session = weak_session_.lock()) {
+session->data_->UpdateWriteOpMetrics(rpc.resp().resource_metrics());
+  }
+
   // Remove all the ops from the "in-flight" list. It's essential to do so
   // _after_ adding all errors into the collector, otherwise there might be
   // a race which manifests itself as described at KUDU-1743. Essentially,
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index a7c7e9835..8543fbdc2 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -2936,6 +2936,24 @@ static void DoTestVerifyRows(const 
shared_ptr& tbl, int num_rows) {
   }
 }
 
+static void DoVerifyMetrics(const KuduSession* session,
+int64_t successful_inserts,
+int64_t insert_ignore_errors,
+int64_t successful_upserts,
+int64_t successful_updates,
+int64_t update_ignore_errors,
+int64_t successful_deletes,
+int64_t delete_ignore_errors) {
+  auto metrics = session->GetWriteOpMetrics().Get();
+  ASSERT_EQ(successful_inserts, metrics["successful_inserts"]);
+  ASSERT_EQ(insert_ignore_errors, metrics["insert_ignore_errors"]);
+  ASSERT_EQ(successful_upserts, metrics["successful_upserts"]);
+  ASSERT_EQ(successful_updates, metrics["successful_updates"]);
+  ASSERT_EQ(update_ignore_errors, metrics["update_ignore_errors"]);
+  ASSERT_EQ(successful_deletes, metrics["successful_deletes"]);
+  ASSERT_EQ(delete_ignore_errors, metrics["delete_ignore_errors"]);
+}
+
 TEST_F(ClientTest, TestInsertIgnore) {
   shared_ptr session = client_->NewSession();
   session->SetTimeoutMillis(1);
@@ -2945,6 +2963,7 @@ TEST_F(ClientTest, TestInsertIgnore) {
 unique_ptr insert(BuildTestInsert(client_table_.get(), 1));
 ASSERT_OK(session->Apply(insert.release()));
 DoTestVerifyRows(client_table_, 1);
+DoVerifyMetrics(session.get(), 1, 0, 0, 0, 0, 0, 0);
   }
 
   {
@@ -2952,6 +2971,7 @@ TEST_F(ClientTest, TestInsertIgnore) {
 unique_ptr 
insert_ignore(BuildTestInsertIgnore(client_table_.get(), 1));
 ASSERT_OK(session->Apply(insert_ignore.release()));
 DoTestVerifyRows(client_table_, 1);
+DoVerifyMetrics(session.get(), 1, 1, 0, 0, 0, 0, 0);
   }
 
   {
@@ -2963,6 +2983,7 @@ TEST_F(ClientTest, TestInsertIgnore) {
 ASSERT_OK(insert_ignore->mutable_row()->SetInt32("non_null_with_default", 
999));
 ASSERT_OK(session->Apply(insert_ignore.release())); // returns ok but 
results in no change
 DoTestVerifyRows(client_table_, 1);
+DoVerifyMetrics(session.get(

[kudu] branch gh-pages updated: Add .asf.yaml

2022-04-29 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch gh-pages
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/gh-pages by this push:
 new 0d0df7b26 Add .asf.yaml
0d0df7b26 is described below

commit 0d0df7b26922056fffc0b35c925ccc866e783c29
Author: Attila Bukor 
AuthorDate: Wed Apr 13 09:22:09 2022 +0200

Add .asf.yaml

An .asf.yaml was committed to the kudu-site repo from which the site
is published some time ago, but this is deleted when regenerating the
site from kudu/gh-pages. Adding this file in kudu/gh-pages should
resolve this issue.

This file is a config file required by the ASF infra[1], without it,
the updated site wouldn't be published.

[1] 
https://cwiki.apache.org/confluence/display/INFRA/Git+-+.asf.yaml+features#Git.asf.yamlfeatures-WebsitedeploymentserviceforGitrepositories

Change-Id: I58d8f85e83ac2ad46d192abb58b748eb7bba6c98
Reviewed-on: http://gerrit.cloudera.org:8080/18410
Reviewed-by: Alexey Serbin 
Tested-by: Attila Bukor 
---
 .asf.yaml   | 2 ++
 _config.yml | 3 +++
 2 files changed, 5 insertions(+)

diff --git a/.asf.yaml b/.asf.yaml
new file mode 100644
index 0..6e33a0350
--- /dev/null
+++ b/.asf.yaml
@@ -0,0 +1,2 @@
+publish:
+whoami: asf-site
diff --git a/_config.yml b/_config.yml
index 0ef898c42..9ad364762 100644
--- a/_config.yml
+++ b/_config.yml
@@ -8,6 +8,9 @@ exclude:
   - tmp
   - build
   - thirdparty
+include:
+  - .asf.yaml
+  - .htaccess
 
 # When writing a blog post, you can use this special HTML comment to
 # designate where the post should be chopped in order to generate the



[kudu] branch master updated: [tools] Add '--create_table_replication_factor' flag for 'kudu table copy'

2022-04-27 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 50b1cc45f [tools] Add '--create_table_replication_factor' flag for 
'kudu table copy'
50b1cc45f is described below

commit 50b1cc45f9fde3deac5aa0fef216f4950246b2c9
Author: Yingchun Lai 
AuthorDate: Mon Apr 25 16:43:59 2022 +0800

[tools] Add '--create_table_replication_factor' flag for 'kudu table copy'

Now it's possible to specify the replication factor for the
destination table when copying a table.

Some usage scenarios, copy a table with RF=3 from a cluster with
multiple tservers to a cluster with only one tserver, we can set
--create_table_replication_factor=1.

Change-Id: I9a4eebdcf85b5ec3666e023194b8c06d66b0a683
Reviewed-on: http://gerrit.cloudera.org:8080/18446
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin 
---
 src/kudu/tools/kudu-tool-test.cc| 50 +
 src/kudu/tools/table_scanner.cc |  9 +--
 src/kudu/tools/tool_action_table.cc | 15 +++
 3 files changed, 62 insertions(+), 12 deletions(-)

diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index c87d39c55..94ed6dbfe 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -517,6 +517,7 @@ class ToolTest : public KuduTest {
 int64_t max_value;
 string columns;
 TableCopyMode mode;
+int32_t create_table_replication_factor;
   };
 
   void RunCopyTableCheck(const RunCopyTableCheckArgs& args) {
@@ -566,14 +567,15 @@ class ToolTest : public KuduTest {
 string stdout;
 NO_FATALS(RunActionStdoutString(
 Substitute("table copy $0 $1 $2 -dst_table=$3 -predicates=$4 
-write_type=$5 "
-   "-create_table=$6",
+   "-create_table=$6 
-create_table_replication_factor=$7",
cluster_->master()->bound_rpc_addr().ToString(),
args.src_table_name,
cluster_->master()->bound_rpc_addr().ToString(),
kDstTableName,
args.predicates_json,
write_type,
-   create_table),
+   create_table,
+   args.create_table_replication_factor),
 ));
 
 // Check total count.
@@ -599,10 +601,15 @@ class ToolTest : public KuduTest {
cluster_->master()->bound_rpc_addr().ToString(),
kDstTableName), _schema));
 
-  // Remove the first lines, which are the different table names.
-  src_schema.erase(src_schema.begin());
-  dst_schema.erase(dst_schema.begin());
-  ASSERT_EQ(src_schema, dst_schema);
+  ASSERT_EQ(src_schema.size(), dst_schema.size());
+  for (int i = 0; i < src_schema.size(); ++i) {
+// Table name is different.
+if (HasPrefixString(src_schema[i], "TABLE ")) continue;
+// Replication factor is different when explicitly set it to 3 
(default 1).
+if (args.create_table_replication_factor == 3 &&
+HasPrefixString(src_schema[i], "REPLICAS ")) continue;
+ASSERT_EQ(src_schema[i], dst_schema[i]);
+  }
 }
 
 // Check all values.
@@ -719,7 +726,13 @@ class ToolTestCopyTableParameterized :
  public:
   void SetUp() override {
 test_case_ = GetParam();
-NO_FATALS(StartExternalMiniCluster());
+ExternalMiniClusterOptions opts;
+if (test_case_ == kTestCopyTableSchemaOnly) {
+  // In kTestCopyTableSchemaOnly case, we may create table with RF=3,
+  // means 3 tservers needed at least.
+  opts.num_tablet_servers = 3;
+}
+NO_FATALS(StartExternalMiniCluster(opts));
 
 // Create the src table and write some data to it.
 TestWorkload ww(cluster_.get());
@@ -757,7 +770,8 @@ class ToolTestCopyTableParameterized :
1,
total_rows_,
kSimpleSchemaColumns,
-   TableCopyMode::INSERT_TO_EXIST_TABLE };
+   TableCopyMode::INSERT_TO_EXIST_TABLE,
+   -1 };
 switch (test_case_) {
   case kTestCopyTableDstTableExist:
 return { args };
@@ -767,9 +781,25 @@ class ToolTestCopyTableParameterized :
   case kTestCopyTableUpsert:
 args.mode = TableCopyMode::UPSERT_TO_EXIST_TABLE;
 return { args };
-  case kTestCopyTableSchemaOnly:
+  case kTestCopyTableSchemaOnly: {
 args.mode = TableCopyMode::COPY_SCHEMA_ONLY;
-return { args };
+vector multi_args;
+ 

[kudu] branch master updated: [unit test] KUDU-3361 Fix SocketTest.TestRecvReset

2022-04-26 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new d216e417f [unit test] KUDU-3361 Fix SocketTest.TestRecvReset
d216e417f is described below

commit d216e417f7fa7daef59ec7eb2c15aadfd4e1c563
Author: shenxingwuying 
AuthorDate: Wed Apr 20 17:57:43 2022 +0800

[unit test] KUDU-3361 Fix SocketTest.TestRecvReset

The case SocketTest.TestRecvReset is not steady, and can cause
jenkins -1, and need rerun it. And I found using a fixed timeout
parameter is not good enough.

eg: Set timeout 1ms coredump happened.
Check failure stack trace: ***
Aborted at 1650445865 (unix time) try "date -d @1650445865" if you are
using GNU date ***
PC: @ 0x7fd63d0092e7 __GI_raise
SIGABRT (@0x198309) received by PID 1671945 (TID 0x7fd640cb7980) from
PID 1671945; stack trace: ***
@   0x7fd63e8881d1 google::(anonymous namespace)::FailureSignalHandler()
@ 0x7fd63d390140 (unknown)
@ 0x7fd63d0092e7 __GI_raise
@ 0x7fd63d00a6c8 __GI_abort
@ 0x7fd63e878419 google::logging_fail()
@ 0x7fd63e87be9d google::LogMessage::Fail()
@ 0x7fd63e87daae google::LogMessage::SendToLog()
@ 0x7fd63e87ba20 google::LogMessage::Flush()
@ 0x7fd63e87e399 google::LogMessageFatal::~LogMessageFatal()
@   0x21ff23 kudu::SocketTest::ConnectToListeningServer()
@   0x21cc4a kudu::SocketTest::DoTestServerDisconnects()
@   0x21b29c kudu::SocketTest_TestRecvReset_Test::TestBody()
@ 0x7fd63e219f2e 
testing::internal::HandleExceptionsInMethodIfSupported<>()
@ 0x7fd63e20fa2b testing::Test::Run()
@ 0x7fd63e20fb95 testing::TestInfo::Run()
@ 0x7fd63e20fc85 testing::TestSuite::Run()
@ 0x7fd63e210144 testing::internal::UnitTestImpl::RunAllTests()
@ 0x7fd63e21a46e 
testing::internal::HandleExceptionsInMethodIfSupported<>()
@ 0x7fd63e210340 testing::UnitTest::Run()
@ 0x7fd640e933c1 RUN_ALL_TESTS()
@ 0x7fd640e92316 main
@ 0x7fd63cff5ac5 __libc_start_main
@   0x219029 (unknown)
Aborted (core dumped)

eg: Set timeout 50ms test failed.
Value of: s.message().ToString()
Expected: contains regular expression "recv error from
127.0.0.1:[0-9]+: Resource temporarily unavailable"
Actual: "recv error from unknown peer: Transport endpoint is not
connected" (of type std::string)
WARNING: Logging before InitGoogleLogging() is written to STDERR

Change-Id: Ie85f36532510db53c8ac167f9d20689da0c76d33
Reviewed-on: http://gerrit.cloudera.org:8080/18431
    Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin 
---
 src/kudu/util/net/socket-test.cc | 19 +++
 1 file changed, 15 insertions(+), 4 deletions(-)

diff --git a/src/kudu/util/net/socket-test.cc b/src/kudu/util/net/socket-test.cc
index dacae8512..e67f956fa 100644
--- a/src/kudu/util/net/socket-test.cc
+++ b/src/kudu/util/net/socket-test.cc
@@ -21,6 +21,7 @@
 
 #include 
 #include 
+#include 
 #include 
 #include 
 #include 
@@ -28,13 +29,16 @@
 #include 
 #include 
 
+#include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
+#include "kudu/util/countdown_latch.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
+#include "kudu/util/thread.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
@@ -122,19 +126,26 @@ class SocketTest : public KuduTest {
 
   void DoTestServerDisconnects(bool accept, const std::string ) {
 NO_FATALS(BindAndListen("0.0.0.0:0"));
-std::thread t([&]{
+
+CountDownLatch latch(1);
+scoped_refptr t;
+Status status = kudu::Thread::Create("pool", "worker", ([&]{
   if (accept) {
 Sockaddr new_addr;
 Socket sock;
 CHECK_OK(listener_.Accept(, _addr, 0));
 CHECK_OK(sock.Close());
   } else {
-SleepFor(MonoDelta::FromMilliseconds(200));
+while (!latch.WaitFor(MonoDelta::FromMilliseconds(10))) {}
 CHECK_OK(listener_.Close());
   }
-});
+}), );
+ASSERT_OK(status);
 SCOPED_CLEANUP({
-  t.join();
+  latch.CountDown();
+  if (t) {
+t->Join();
+  }
 });
 
 Socket client = ConnectToListeningServer();



[kudu] branch master updated: [unit test] Fix a unit test problem from code review.

2022-04-20 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 42fbe6b2e [unit test] Fix a unit test problem from code review.
42fbe6b2e is described below

commit 42fbe6b2e0085d934068e1687de1d3c936e89ed1
Author: shenxingwuying 
AuthorDate: Wed Apr 20 16:14:44 2022 +0800

[unit test] Fix a unit test problem from code review.

Internal MiniTabletServer's Restart() function would core dump,
because server_ may be not nullptr, should shutdown and then start.

Change-Id: Ib6a810799cdd9adec24944c4bea959103c775966
Reviewed-on: http://gerrit.cloudera.org:8080/18427
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 src/kudu/tserver/mini_tablet_server.cc | 1 +
 1 file changed, 1 insertion(+)

diff --git a/src/kudu/tserver/mini_tablet_server.cc 
b/src/kudu/tserver/mini_tablet_server.cc
index 43c76d900..40acd362a 100644
--- a/src/kudu/tserver/mini_tablet_server.cc
+++ b/src/kudu/tserver/mini_tablet_server.cc
@@ -126,6 +126,7 @@ void MiniTabletServer::Shutdown() {
 }
 
 Status MiniTabletServer::Restart() {
+  Shutdown();
   return Start();
 }
 



[kudu] branch master updated: [test] avoid coredump in socket-test when failing

2022-04-18 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new c71f8abbb [test] avoid coredump in socket-test when failing
c71f8abbb is described below

commit c71f8abbbebb6e8c003090c0bb5866b443b8955f
Author: Alexey Serbin 
AuthorDate: Mon Apr 18 12:26:28 2022 -0700

[test] avoid coredump in socket-test when failing

I noticed the socket-test crashed when SocketTest.TestRecvReset failed
with the stack trace below.  This patch updates the code to at least
avoid the crash in case of a failure.

  src/kudu/util/net/socket-test.cc:144: Failure
  Value of: s.message().ToString()
  Expected: contains regular expression "recv error from 127.0.0.1:[0-9]+: 
Resource temporarily unavailable"
Actual: "recv error from unknown peer: Transport endpoint is not 
connected"
  terminating
  *** Aborted at 1650307680 (unix time) try "date -d @1650307680" if you 
are using GNU date ***
  PC: @ 0x7f37ddbe0fb7 gsignal
  *** SIGABRT (@0x3e85198) received by PID 20888 (TID 0x7f37e10253c0) 
from PID 20888; stack trace: ***
  @   0x437b40 __tsan::CallUserSignalHandler()
  @   0x43a274 rtl_sigaction()
  @ 0x7f37de1bd980 (unknown) at ??:0
  @ 0x7f37ddbe0fb7 gsignal at ??:0
  @ 0x7f37ddbe2921 abort at ??:0
  @   0x43bfd7 __interceptor_abort
  @ 0x7f37de791a05 abort_message at ??:0
  @ 0x7f37de7763be demangling_terminate_handler() at ??:0
  @ 0x7f37de790b58 std::__terminate() at ??:0
  @ 0x7f37de790ae8 std::terminate() at ??:0
  @ 0x7f37dea76b81 std::__1::thread::~thread() at ??:0
  @   0x4dc097 kudu::SocketTest::DoTestServerDisconnects() at 
src/kudu/util/net/socket-test.cc:147
  @   0x4db093 kudu::SocketTest_TestRecvReset_Test::TestBody() 
at src/kudu/util/net/socket-test.cc:?
  @ 0x7f37df11d500 
testing::internal::HandleExceptionsInMethodIfSupported<>() at ??:0
  @ 0x7f37df0f2452 testing::Test::Run() at ??:0
  @ 0x7f37df0f39ea testing::TestInfo::Run() at ??:0
  @ 0x7f37df0f4757 testing::TestSuite::Run() at ??:0
  @ 0x7f37df108527 testing::internal::UnitTestImpl::RunAllTests() 
at ??:0
  @ 0x7f37df11e6e0 
testing::internal::HandleExceptionsInMethodIfSupported<>() at ??:0
  @ 0x7f37df107acd testing::UnitTest::Run() at ??:0
  @ 0x7f37e11c2e1c RUN_ALL_TESTS() at ??:0
  @ 0x7f37e11c1c6a main at ??:0
  @ 0x7f37ddbc3bf7 __libc_start_main at ??:0
  @   0x4306ad (unknown) at ??:?

Change-Id: I5cf8f7b9acfac7b1935e6e6d9b3ac5016ee753d9
Reviewed-on: http://gerrit.cloudera.org:8080/18425
    Reviewed-by: Andrew Wong 
Tested-by: Alexey Serbin 
---
 src/kudu/util/net/socket-test.cc | 11 +++
 1 file changed, 7 insertions(+), 4 deletions(-)

diff --git a/src/kudu/util/net/socket-test.cc b/src/kudu/util/net/socket-test.cc
index 882a695d4..dacae8512 100644
--- a/src/kudu/util/net/socket-test.cc
+++ b/src/kudu/util/net/socket-test.cc
@@ -133,17 +133,17 @@ class SocketTest : public KuduTest {
 CHECK_OK(listener_.Close());
   }
 });
+SCOPED_CLEANUP({
+  t.join();
+});
 
 Socket client = ConnectToListeningServer();
 int n;
 std::unique_ptr buf(new uint8_t[kEchoChunkSize]);
-Status s = client.Recv(buf.get(), kEchoChunkSize, );
+const auto s = client.Recv(buf.get(), kEchoChunkSize, );
 
-ASSERT_TRUE(!s.ok());
 ASSERT_TRUE(s.IsNetworkError()) << s.ToString();
 ASSERT_STR_MATCHES(s.message().ToString(), message);
-
-t.join();
   }
 
   void DoUnixSocketTest(const string& path) {
@@ -166,6 +166,7 @@ class SocketTest : public KuduTest {
   MonoTime::Now() + MonoDelta::FromSeconds(10)));
   CHECK_OK(sock.Close());
 });
+auto cleanup = MakeScopedCleanup([&] { t.join(); });
 
 Socket client = ConnectToListeningServer();
 
@@ -178,7 +179,9 @@ class SocketTest : public KuduTest {
 char buf[kData.size()];
 ASSERT_OK(client.BlockingRecv(reinterpret_cast(buf), 
kData.size(), ,
   MonoTime::Now() + 
MonoDelta::FromSeconds(5)));
+cleanup.cancel();
 t.join();
+
 ASSERT_OK(client.Close());
 
 ASSERT_EQ(n, kData.size());



[kudu] 01/02: [cmake_modules] add '-o /dev/null' to the linker command

2022-04-08 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit fe2ba596dda3872789de8d42ccc2b82b6db42d47
Author: Alexey Serbin 
AuthorDate: Thu Apr 7 15:06:51 2022 -0700

[cmake_modules] add '-o /dev/null' to the linker command

I noticed that every time I run cmake in the build directory
on macOS, a.out file appears.  It turned out it's generated
by the GET_LINKER_VERSION procedure from KuduLinker.cmake.

This patch add '-o /dev/null' to the command that determines
the version of the linker to avoid generating a.out file
in the current working directory.

Change-Id: Id095902679a2f0e2648c45bcd4024fc734b9a1e7
Reviewed-on: http://gerrit.cloudera.org:8080/18394
Tested-by: Alexey Serbin 
Reviewed-by: Andrew Wong 
---
 cmake_modules/KuduLinker.cmake | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/cmake_modules/KuduLinker.cmake b/cmake_modules/KuduLinker.cmake
index d8dacfc21..0725bb03d 100644
--- a/cmake_modules/KuduLinker.cmake
+++ b/cmake_modules/KuduLinker.cmake
@@ -113,7 +113,7 @@ function(GET_LINKER_VERSION)
 set(ld_version_flag "--version")
   endif()
   execute_process(
-COMMAND ${CMAKE_CXX_COMPILER} "-Wl,${ld_version_flag}" ${ARGN}
+COMMAND ${CMAKE_CXX_COMPILER} "-Wl,${ld_version_flag}" -o /dev/null ${ARGN}
 ERROR_VARIABLE LINKER_STDERR
 OUTPUT_VARIABLE LINKER_STDOUT
 RESULT_VARIABLE LINKER_EXITCODE)



[kudu] branch master updated (ed04c8b6f -> 49e0d6d42)

2022-04-08 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


from ed04c8b6f [UT] Fix env-test resource limit overflow
 new fe2ba596d [cmake_modules] add '-o /dev/null' to the linker command
 new 49e0d6d42 [tests] make one scenario of client-test more stable

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 cmake_modules/KuduLinker.cmake |  2 +-
 src/kudu/client/client-test.cc | 16 ++--
 2 files changed, 11 insertions(+), 7 deletions(-)



[kudu] 02/02: [tests] make one scenario of client-test more stable

2022-04-08 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 49e0d6d428eed80dd8d96ab191c2d5062a9ed5dd
Author: Alexey Serbin 
AuthorDate: Thu Apr 7 17:37:22 2022 -0700

[tests] make one scenario of client-test more stable

This patch makes ClientTest.TestScannerKeepAlive more stable.

The motivation for this patch was seeing the following test
failure in a TSAN build [1]:

  src/kudu/client/client-test.cc:2763
  Failed
  Bad status: Not found: Scanner 0d9072070e4443ba8c49b089806415b4 not found 
(it may have expired)

Before: 2 out of 32 failed
  http://dist-test.cloudera.org/job?job_id=aserbin.1649379879.23423

After:  0 out of 32 failed
  http://dist-test.cloudera.org/job?job_id=aserbin.1649381490.37476

[1] http://jenkins.kudu.apache.org/job/kudu-gerrit/25299/BUILD_TYPE=TSAN

Change-Id: Ia7ebddc5499d59e37282e4f1e6513bc5f83cdb8a
Reviewed-on: http://gerrit.cloudera.org:8080/18396
Reviewed-by: Yingchun Lai 
Tested-by: Alexey Serbin 
---
 src/kudu/client/client-test.cc | 16 ++--
 1 file changed, 10 insertions(+), 6 deletions(-)

diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index 75b2bce1d..c8597bd29 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -2722,9 +2722,11 @@ int64_t SumResults(const KuduScanBatch& batch) {
 } // anonymous namespace
 
 TEST_F(ClientTest, TestScannerKeepAlive) {
+  SKIP_IF_SLOW_NOT_ALLOWED();
+
   NO_FATALS(InsertTestRows(client_table_.get(), 1000));
-  // Set the scanner ttl really low
-  FLAGS_scanner_ttl_ms = 100; // 100 milliseconds
+  // Set the scanner TTL low.
+  FLAGS_scanner_ttl_ms = 500;
   // Start a scan but don't get the whole data back
   KuduScanner scanner(client_table_.get());
   // This will make sure we have to do multiple NextBatch calls to the second 
tablet.
@@ -2756,9 +2758,11 @@ TEST_F(ClientTest, TestScannerKeepAlive) {
   sum += SumResults(batch);
   ASSERT_TRUE(scanner.HasMoreRows());
 
-  // Now loop while keeping the scanner alive. Each time we loop we sleep 1/2 
a scanner
-  // ttl interval (the garbage collector is running each 50 msecs too.).
-  for (int i = 0; i < 5; i++) {
+  // Now loop while keeping the scanner alive. Each loop we sleep about 1/10
+  // of the scanner's TTL interval to avoid flakiness due to scheduling
+  // anomalies. The garbage collector runs each 50 msec as well in this test
+  // scenario (controlled by FLAGS_scanner_gc_check_interval_us).
+  for (int i = 0; i < 15; i++) {
 SleepFor(MonoDelta::FromMilliseconds(50));
 ASSERT_OK(scanner.KeepAlive());
   }
@@ -2772,7 +2776,7 @@ TEST_F(ClientTest, TestScannerKeepAlive) {
   }
 
   ASSERT_TRUE(scanner.HasMoreRows());
-  for (int i = 0; i < 5; i++) {
+  for (int i = 0; i < 15; i++) {
 SleepFor(MonoDelta::FromMilliseconds(50));
 ASSERT_OK(scanner.KeepAlive());
   }



[kudu] branch master updated: [UT] Fix env-test resource limit overflow

2022-04-06 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new ed04c8b6f [UT] Fix env-test resource limit overflow
ed04c8b6f is described below

commit ed04c8b6f70c6dd5b63cc517e33fb8e52c1c3ce9
Author: Yingchun Lai 
AuthorDate: Wed Apr 6 23:59:57 2022 +0800

[UT] Fix env-test resource limit overflow

Fix the test failure as following:
[ RUN  ] ResourceLimitTypes/ResourceLimitTypeTest.TestIncreaseLimit/1
I0406 23:56:52.783593 1668634 env_posix.cc:2174] Raising this process' 
running threads per effective uid limit from 4096 to 18446744073709551615
.../kudu/src/kudu/util/env-test.cc:690: Failure
Expected: (limit_after) >= (limit_before), actual: -1 vs 4096
I0406 23:56:52.784036 1668634 test_util.cc:164] 
---
I0406 23:56:52.784056 1668634 test_util.cc:165] Had fatal failures, leaving 
test files at 
.../env-test.ResourceLimitTypes_ResourceLimitTypeTest.TestIncreaseLimit_1.1649260612778581-1668634-0
[  FAILED  ] ResourceLimitTypes/ResourceLimitTypeTest.TestIncreaseLimit/1, 
where GetParam() = running threads per effective uid (2 ms)

Change-Id: Ic919722ca0f7eab00baf8e7514b1c1bd1e1b86e9
Reviewed-on: http://gerrit.cloudera.org:8080/18388
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 src/kudu/util/env-test.cc | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/src/kudu/util/env-test.cc b/src/kudu/util/env-test.cc
index f3e23439e..cec87dc9f 100644
--- a/src/kudu/util/env-test.cc
+++ b/src/kudu/util/env-test.cc
@@ -684,14 +684,14 @@ INSTANTIATE_TEST_SUITE_P(ResourceLimitTypes,
 TEST_P(ResourceLimitTypeTest, TestIncreaseLimit) {
   // Increase the resource limit. It should either increase or remain the same.
   Env::ResourceLimitType t = GetParam();
-  int64_t limit_before = env_->GetResourceLimit(t);
+  uint64_t limit_before = env_->GetResourceLimit(t);
   env_->IncreaseResourceLimit(t);
-  int64_t limit_after = env_->GetResourceLimit(t);
+  uint64_t limit_after = env_->GetResourceLimit(t);
   ASSERT_GE(limit_after, limit_before);
 
   // Try again. It should definitely be the same now.
   env_->IncreaseResourceLimit(t);
-  int64_t limit_after_again = env_->GetResourceLimit(t);
+  uint64_t limit_after_again = env_->GetResourceLimit(t);
   ASSERT_EQ(limit_after, limit_after_again);
 }
 



[kudu] branch master updated: [tools] range rebalancing for 'kudu cluster rebalance'

2022-04-04 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 01ca8c3f2 [tools] range rebalancing for 'kudu cluster rebalance'
01ca8c3f2 is described below

commit 01ca8c3f2bb38bc8b738d86d3d88aa8d1ffaad87
Author: Alexey Serbin 
AuthorDate: Wed Mar 9 14:57:45 2022 -0800

[tools] range rebalancing for 'kudu cluster rebalance'

This patch adds range rebalancing functionality into the
'kudu cluster rebalance' CLI tool.  The implementation is rather an
MVP: the range rebalancing can now be performed only for a single
table per run.  As far as I can see, there is a room for improvement
since it's possible to perform range-aware replica movements even
during standard whole cluster rebalancing.

Below is two snapshots of distribution of the range-specific tablet
replicas in a cluster.  Those are produced by running the tool with
extra --report_only --output_replica_distribution_details flags
before and after range rebalancing for a single table:

  kudu cluster rebalance \
 --enable_range_rebalancing \
 --tables=default.loadgen_auto_6800f4ec4e164b2b8e42db7b5044df09 \
 127.0.0.1:8765

before:


Table: abb2bbf8b4ff4bc0989bc82c78d4ae2b

Number of tablet replicas at servers for each range
 Max Skew | Total Count |   Range Start Key
--+-+--
 8| 8   |
 8| 8   | ff81fff4
 8| 8   | ff83ffe8
 8| 8   | ff85ffdc

Range start key: ''
   UUID   | Server address | Replica Count
--++---
 15a8d0fef42c4da2bd5d9e1c5a2de301 | 127.0.0.1:9870 | 8
 3243029e0db04680a2653c6acc048813 | 127.0.0.1:9876 | 0
 324ef10666b14ab9bb61e775fa351ad6 | 127.0.0.1:9872 | 0
 a5c6f822f5cc4645bbb4a14874e311d4 | 127.0.0.1:9874 | 0

Range start key: 'ff81fff4'
   UUID   | Server address | Replica Count
--++---
 15a8d0fef42c4da2bd5d9e1c5a2de301 | 127.0.0.1:9870 | 0
 3243029e0db04680a2653c6acc048813 | 127.0.0.1:9876 | 8
 324ef10666b14ab9bb61e775fa351ad6 | 127.0.0.1:9872 | 0
 a5c6f822f5cc4645bbb4a14874e311d4 | 127.0.0.1:9874 | 0

Range start key: 'ff83ffe8'
   UUID   | Server address | Replica Count
--++---
 15a8d0fef42c4da2bd5d9e1c5a2de301 | 127.0.0.1:9870 | 0
 3243029e0db04680a2653c6acc048813 | 127.0.0.1:9876 | 0
 324ef10666b14ab9bb61e775fa351ad6 | 127.0.0.1:9872 | 8
 a5c6f822f5cc4645bbb4a14874e311d4 | 127.0.0.1:9874 | 0

Range start key: 'ff85ffdc'
   UUID   | Server address | Replica Count
--++---
 15a8d0fef42c4da2bd5d9e1c5a2de301 | 127.0.0.1:9870 | 0
 3243029e0db04680a2653c6acc048813 | 127.0.0.1:9876 | 0
 324ef10666b14ab9bb61e775fa351ad6 | 127.0.0.1:9872 | 0
 a5c6f822f5cc4645bbb4a14874e311d4 | 127.0.0.1:9874 | 8

after:


Table: abb2bbf8b4ff4bc0989bc82c78d4ae2b

Number of tablet replicas at servers for each range
 Max Skew | Total Count |   Range Start Key
--+-+--
 0| 8   |
 0| 8   | ff81fff4
 0| 8   | ff83ffe8
 0| 8   | ff85ffdc

Range start key: ''
   UUID   | Server address | Replica Count
--++---
 15a8d0fef42c4da2bd5d9e1c5a2de301 | 127.0.0.1:9870 | 2
 3243029e0db04680a2653c6acc048813 | 127.0.0.1:9876 | 2
 324ef10666b14ab9bb61e775fa351ad6 | 127.0.0.1:9872 | 2
 a5c6f822f5cc4645bbb4a14874e311d4 | 127.0.0.1:9874 | 2

Range start key: 'ff81fff4'
   UUID   | Server address | Replica Count
--++---
 15a8d0fef42c4da2bd5d9e1c5a2de301 | 127.0.0.1:9870 | 2
 3243029e0db04680a2653c6acc048813 | 127.0.0.1:9876 | 2
 324ef10666b14ab9bb61e775fa351ad6 | 127.0.0.1:9872 | 2
 a5c6f822f5cc4645bbb4a14874e311d4 | 127.0.0.1:9874 | 2

Range start key: 'ff83ffe8'
   UUID   | Server address | Replica Count

[kudu] branch master updated: [build] thirdparty issues on Ubuntu 21.10

2022-04-04 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new fb6dc6062 [build] thirdparty issues on Ubuntu 21.10
fb6dc6062 is described below

commit fb6dc60621aa49f214b06d70527860c6a26c0e15
Author: Zoltan Chovan 
AuthorDate: Fri Apr 1 17:11:21 2022 +0200

[build] thirdparty issues on Ubuntu 21.10

Three issues popped up during thirdparty build on Ubuntu 21.10:
* The linux kernel has removed the interface to cyclades, which led to llvm 
build failure:
  
https://reviews.llvm.org/rG68d5235cb58f988c71b403334cd9482d663841ab#change-OqOKlUOZCPKc
* Missing include which led to llvm build failure:
  https://reviews.llvm.org/rGb498303066a63a203d24f739b2d2e0e56dca70d1
* A cast mismatch that led to breakpad build failure

Change-Id: I2d019ea6b2c5319252a78855bfe84e0f17acb535
Reviewed-on: http://gerrit.cloudera.org:8080/18381
Tested-by: Kudu Jenkins
Reviewed-by: Attila Bukor 
Reviewed-by: Alexey Serbin 
---
 thirdparty/download-thirdparty.sh  |  11 +-
 thirdparty/patches/breakpad-SIGSTKSZ-error.patch   |  11 ++
 thirdparty/patches/llvm-fix-missing-include.patch  |  21 
 ...vm-remove-cyclades-inclusion-in-sanitizer.patch | 117 +
 4 files changed, 156 insertions(+), 4 deletions(-)

diff --git a/thirdparty/download-thirdparty.sh 
b/thirdparty/download-thirdparty.sh
index b35aa01f0..e7bf40071 100755
--- a/thirdparty/download-thirdparty.sh
+++ b/thirdparty/download-thirdparty.sh
@@ -329,7 +329,7 @@ fetch_and_patch \
  $PYTHON_SOURCE \
  $PYTHON_PATCHLEVEL
 
-LLVM_PATCHLEVEL=5
+LLVM_PATCHLEVEL=6
 fetch_and_patch \
  llvm-${LLVM_VERSION}-iwyu-${IWYU_VERSION}.src.tar.gz \
  $LLVM_SOURCE \
@@ -338,7 +338,9 @@ fetch_and_patch \
   "patch -p1 < $TP_DIR/patches/llvm-iwyu-include-picker.patch" \
   "patch -p1 < 
$TP_DIR/patches/llvm-MicrosoftDemangleNodes-e0402b5c9813a2458b8dd3f640883110db280395.patch"
 \
   "patch -p0 < $TP_DIR/patches/llvm-iwyu-sized-deallocation.patch" \
-  "patch -d projects -p1 < 
$TP_DIR/patches/llvm-947f9692440836dcb8d88b74b69dd379d85974ce.patch"
+  "patch -d projects -p1 < 
$TP_DIR/patches/llvm-947f9692440836dcb8d88b74b69dd379d85974ce.patch" \
+  "patch -d projects -p1 < 
$TP_DIR/patches/llvm-remove-cyclades-inclusion-in-sanitizer.patch" \
+  "patch -p2 < $TP_DIR/patches/llvm-fix-missing-include.patch"
 
 LZ4_PATCHLEVEL=0
 fetch_and_patch \
@@ -364,13 +366,14 @@ fetch_and_patch \
  $BOOST_SOURCE \
  $BOOST_PATCHLEVEL
 
-BREAKPAD_PATCHLEVEL=2
+BREAKPAD_PATCHLEVEL=3
 fetch_and_patch \
  breakpad-${BREAKPAD_VERSION}.tar.gz \
  $BREAKPAD_SOURCE \
  $BREAKPAD_PATCHLEVEL \
  "patch -p1 < 
$TP_DIR/patches/breakpad-add-basic-support-for-dwz-dwarf-extension.patch" \
- "patch -p1 < $TP_DIR/patches/breakpad-syscall-rsp-clobber-fix.patch"
+ "patch -p1 < $TP_DIR/patches/breakpad-syscall-rsp-clobber-fix.patch" \
+ "patch -p0 < $TP_DIR/patches/breakpad-SIGSTKSZ-error.patch"
 
 SPARSEHASH_PATCHLEVEL=3
 fetch_and_patch \
diff --git a/thirdparty/patches/breakpad-SIGSTKSZ-error.patch 
b/thirdparty/patches/breakpad-SIGSTKSZ-error.patch
new file mode 100644
index 0..0045a4dc6
--- /dev/null
+++ b/thirdparty/patches/breakpad-SIGSTKSZ-error.patch
@@ -0,0 +1,11 @@
+--- src/client/linux/handler/exception_handler.cc  2022-04-01 
18:31:33.463554421 +0200
 src/client/linux/handler/exception_handler.cc  2022-04-01 
18:32:20.503739341 +0200
+@@ -138,7 +138,7 @@
+   // SIGSTKSZ may be too small to prevent the signal handlers from overrunning
+   // the alternative stack. Ensure that the size of the alternative stack is
+   // large enough.
+-  static const unsigned kSigStackSize = std::max(16384, SIGSTKSZ);
++  static const unsigned kSigStackSize = std::max(16384u, (unsigned)SIGSTKSZ);
+ 
+   // Only set an alternative stack if there isn't already one, or if the 
current
+   // one is too small.
diff --git a/thirdparty/patches/llvm-fix-missing-include.patch 
b/thirdparty/patches/llvm-fix-missing-include.patch
new file mode 100644
index 0..373af741b
--- /dev/null
+++ b/thirdparty/patches/llvm-fix-missing-include.patch
@@ -0,0 +1,21 @@
+From b498303066a63a203d24f739b2d2e0e56dca70d1 Mon Sep 17 00:00:00 2001
+From: serge-sans-paille 
+Date: Tue, 10 Nov 2020 14:55:25 +0100
+Subject: [PATCH] [nfc] Fix missing include
+
+---
+ llvm/utils/benchmark/src/benchmark_register.h | 1 +
+ 1 file changed, 1 insertion(+)
+
+diff --git a/llvm/utils/benchmark/src/benchmark_register.h 
b/llvm/utils/benchmark/src/benchmark_register.h
+index 0705e219f2fa2..4caa5ad4da079 100644
+--- a/llvm/utils/benchmark/src/benchmark_register.h
 b/llvm/utils/benchmark/src/benchmark_register.h
+@@ -1,6 +1,7 @@
+ #ifndef BENC

[kudu] branch master updated: [tablet-copy] Simple refactor on tablet_copy_service{.h,.cc,-test.cc}

2022-04-04 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 2bf2b2d5d [tablet-copy] Simple refactor on 
tablet_copy_service{.h,.cc,-test.cc}
2bf2b2d5d is described below

commit 2bf2b2d5dcd61a80f928e9f4a20fe70c42f00592
Author: Yingchun Lai 
AuthorDate: Sun Apr 3 23:17:54 2022 +0800

[tablet-copy] Simple refactor on tablet_copy_service{.h,.cc,-test.cc}

There is no functional changes in this patch, but only code style
related changes.

Change-Id: If1f4fd31cf3b217e604b565165a04efa017d19d2
Reviewed-on: http://gerrit.cloudera.org:8080/18379
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin 
---
 src/kudu/tserver/tablet_copy_service-test.cc | 12 +--
 src/kudu/tserver/tablet_copy_service.cc  |  6 +++---
 src/kudu/tserver/tablet_copy_service.h   | 32 +---
 3 files changed, 24 insertions(+), 26 deletions(-)

diff --git a/src/kudu/tserver/tablet_copy_service-test.cc 
b/src/kudu/tserver/tablet_copy_service-test.cc
index 97c6578ac..d7a0714d1 100644
--- a/src/kudu/tserver/tablet_copy_service-test.cc
+++ b/src/kudu/tserver/tablet_copy_service-test.cc
@@ -106,9 +106,9 @@ class TabletCopyServiceTest : public TabletCopyTest {
   }
 
   Status DoBeginValidTabletCopySession(string* session_id,
-tablet::TabletSuperBlockPB* 
superblock = nullptr,
-uint64_t* idle_timeout_millis = 
nullptr,
-vector* sequence_numbers 
= nullptr) {
+   tablet::TabletSuperBlockPB* superblock 
= nullptr,
+   uint64_t* idle_timeout_millis = nullptr,
+   vector* sequence_numbers = 
nullptr) {
 BeginTabletCopySessionResponsePB resp;
 RpcController controller;
 RETURN_NOT_OK(DoBeginTabletCopySession(GetTabletId(), GetLocalUUID(), 
, ));
@@ -218,9 +218,9 @@ TEST_F(TabletCopyServiceTest, TestSimpleBeginEndSession) {
   uint64_t idle_timeout_millis;
   vector segment_seqnos;
   ASSERT_OK(DoBeginValidTabletCopySession(_id,
-   ,
-   _timeout_millis,
-   _seqnos));
+  ,
+  _timeout_millis,
+  _seqnos));
   // Basic validation of returned params.
   ASSERT_FALSE(session_id.empty());
   ASSERT_EQ(FLAGS_tablet_copy_idle_timeout_sec * 1000, idle_timeout_millis);
diff --git a/src/kudu/tserver/tablet_copy_service.cc 
b/src/kudu/tserver/tablet_copy_service.cc
index 55ef98a3a..68fc2b249 100644
--- a/src/kudu/tserver/tablet_copy_service.cc
+++ b/src/kudu/tserver/tablet_copy_service.cc
@@ -34,6 +34,7 @@
 #include "kudu/fs/fs_manager.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/map-util.h"
+#include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/rpc/rpc_context.h"
 #include "kudu/server/server_base.h"
@@ -285,7 +286,7 @@ void TabletCopyServiceImpl::FetchData(const 
FetchDataRequestPB* req,
 
   const DataIdPB& data_id = req->data_id();
   TabletCopyErrorPB::Code error_code = TabletCopyErrorPB::UNKNOWN_ERROR;
-  RPC_RETURN_NOT_OK(ValidateFetchRequestDataId(data_id, _code, session),
+  RPC_RETURN_NOT_OK(ValidateFetchRequestDataId(data_id, _code),
 error_code, "Invalid DataId", context);
 
   DataChunkPB* data_chunk = resp->mutable_chunk();
@@ -368,8 +369,7 @@ Status TabletCopyServiceImpl::FindSessionUnlocked(
 
 Status TabletCopyServiceImpl::ValidateFetchRequestDataId(
 const DataIdPB& data_id,
-TabletCopyErrorPB::Code* app_error,
-const scoped_refptr& session) const {
+TabletCopyErrorPB::Code* app_error) {
   if (PREDICT_FALSE(data_id.has_block_id() && 
data_id.has_wal_segment_seqno())) {
 *app_error = TabletCopyErrorPB::INVALID_TABLET_COPY_REQUEST;
 return Status::InvalidArgument(
diff --git a/src/kudu/tserver/tablet_copy_service.h 
b/src/kudu/tserver/tablet_copy_service.h
index 19ac07a58..4882f7926 100644
--- a/src/kudu/tserver/tablet_copy_service.h
+++ b/src/kudu/tserver/tablet_copy_service.h
@@ -20,7 +20,6 @@
 #include 
 #include 
 
-#include "kudu/gutil/port.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/tserver/tablet_copy.pb.h"
 #include "kudu/tserver/tablet_copy.service.h"
@@ -63,23 +62,23 @@ class TabletCopyServiceImpl : public TabletCopyServiceIf {
 google::protobuf::Message* resp,

[kudu] branch master updated: [tools] add beginning of the key range into TabletSummary

2022-03-31 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 66fe336  [tools] add beginning of the key range into TabletSummary
66fe336 is described below

commit 66fe33642ac9662b8d2a3e52a90168b2184e4d88
Author: Alexey Serbin 
AuthorDate: Thu Mar 10 16:57:23 2022 -0800

[tools] add beginning of the key range into TabletSummary

This patch adds the beginning of the key range into the TabletSummary
class used in ksck reporting.  Corresponding TabletSummaryPB has been
updated as well.

This patch also contains a test to verify the presence of the newly
added field in TabletSummary entries produced by ksck.

The newly added field will be used in a follow-up changelist for
range-aware rebalancing.

Change-Id: Icf083e1ae33919200f6e0b0f0a667ee6eea1bab3
Reviewed-on: http://gerrit.cloudera.org:8080/18356
Tested-by: Kudu Jenkins
Reviewed-by: Andrew Wong 
---
 src/kudu/client/CMakeLists.txt   |  1 +
 src/kudu/client/client.h |  2 +
 src/kudu/client/tablet_info_provider-internal.cc | 50 ++
 src/kudu/client/tablet_info_provider-internal.h  | 49 +
 src/kudu/rebalance/cluster_status.h  |  1 +
 src/kudu/tools/ksck-test.cc  | 16 +++--
 src/kudu/tools/ksck.cc   | 13 
 src/kudu/tools/ksck.h| 15 ++--
 src/kudu/tools/ksck_remote-test.cc   | 87 +++-
 src/kudu/tools/ksck_remote.cc| 13 +++-
 src/kudu/tools/ksck_results.cc   |  1 +
 src/kudu/tools/tool.proto|  1 +
 12 files changed, 236 insertions(+), 13 deletions(-)

diff --git a/src/kudu/client/CMakeLists.txt b/src/kudu/client/CMakeLists.txt
index fcf5c2c..65821b3 100644
--- a/src/kudu/client/CMakeLists.txt
+++ b/src/kudu/client/CMakeLists.txt
@@ -50,6 +50,7 @@ set(CLIENT_SRCS
   scanner-internal.cc
   replica-internal.cc
   replica_controller-internal.cc
+  tablet_info_provider-internal.cc
   resource_metrics.cc
   schema.cc
   session-internal.cc
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 3905ec8..35dab86 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -122,6 +122,7 @@ class RemoteTabletServer;
 class ReplicaController;
 class RetrieveAuthzTokenRpc;
 class ScanBatchDataInterface;
+class TabletInfoProvider;
 class WriteRpc;
 template 
 class AsyncLeaderMasterRpc; // IWYU pragma: keep
@@ -986,6 +987,7 @@ class KUDU_EXPORT KuduClient : public 
sp::enable_shared_from_this {
   friend class internal::RemoteTablet;
   friend class internal::RemoteTabletServer;
   friend class internal::RetrieveAuthzTokenRpc;
+  friend class internal::TabletInfoProvider;
   friend class internal::WriteRpc;
   friend class kudu::AuthzTokenTest;
   friend class kudu::DisableWriteWhenExceedingQuotaTest;
diff --git a/src/kudu/client/tablet_info_provider-internal.cc 
b/src/kudu/client/tablet_info_provider-internal.cc
new file mode 100644
index 000..ef6383e
--- /dev/null
+++ b/src/kudu/client/tablet_info_provider-internal.cc
@@ -0,0 +1,50 @@
+// 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/client/tablet_info_provider-internal.h"
+
+#include 
+
+#include "kudu/client/client-internal.h"
+#include "kudu/client/client.h"
+#include "kudu/client/meta_cache.h"
+#include "kudu/common/partition.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/async_util.h"
+#include "kudu/util/monotime.h"
+
+namespace kudu {
+namespace client {
+namespace internal {
+
+Status TabletInfoProvider::GetPartitionInfo(KuduClient* client,
+const std::string& tablet_id,
+Partition* partition) {
+  DCHECK(partition);
+  scoped_refptr rt;
+  Synchronizer sync;
+  client->data_->meta_cache_->LookupTabletById(
+  cl

[kudu] branch master updated: [security] set minimum TLS protocol version to TSLv1.2

2022-03-23 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new a8fb42d  [security] set minimum TLS protocol version to TSLv1.2
a8fb42d is described below

commit a8fb42dc34e8f1f876db5b26fc3f5eb3196ce854
Author: Alexey Serbin 
AuthorDate: Thu Apr 1 18:04:29 2021 -0700

[security] set minimum TLS protocol version to TSLv1.2

Since support for CentOS 6/RHEL 6, Ubuntu 14, Ubuntu 16, and Debian 8
is dropped since Kudu 1.14 [1], we can bump the minimum required TLS
protocol version up to TLSv1.2 for securing Kudu RPC.  That's because
  * Supported server-side OSes have OpenSSL of at least version 1.0.1
in their stock distribution, so Kudu servers running on supported
OSes automatically support TLSv1.2
  * Kudu Java client requires Java 8+ in runtime since Kudu 1.10 [2],
so Kudu Java clients automatically support TLSv1.2 since then
  * As of now, with contemporary Linux OS and updated Java 8 [4], Kudu
servers and clients should be able to use even TLSv1.3.

In addition, this patch updates the list of the preferred TLSv1.2
ciphers, bringing it up-to-date with the "intermediate compatibility"
cipher list of the Mozilla Security Server Side TLS recommendations [3]
(without the DH AES ciphers).

At some point I thought to set TLSv1.2 as the minimum protocol version
only for the server side, keeping TLSv1 as the minimum version for the
client side to allow newer Kudu clients work with Kudu servers running
at no longer supported, beyond-EOL OSes.  However, I realized that the
set of TLS ciphers supported by Kudu servers in 1.3 -- 1.14 release
range contained only a couple of non-TLSv1.2 ciphers suites:
  * AES128-SHA (TLS_RSA_WITH_AES_128_CBC_SHA)
  * AES256-SHA (TLS_RSA_WITH_AES_256_CBC_SHA)
These can be used starting SSLv3, so these are the only ones that Kudu
clients and servers could use if either side didn't support TLSv1.2.
Both use non-GCM ciphers for data encryption, and data transfer rate
with those ciphers is an order of magnitute slower than with TLSv1.2
suites which use GCM ciphers for data encryption.  Given all those
facts, I realized it's not worth it keeping such a compatibility around.
The rationale is: if people are still stuck with RHEL/CentOS 6 or other
EOL OS release, they most likely do not care about security and will
be able to use one of the workarounds listed below.

So, with this patch:

  * TLSv1.2 is the minimum TLS protocol version that newer Kudu clients
are able to use for secure Kudu RPC.  The newer clients are not able
to communicate with servers built and run with OpenSSL of versions
prior to 1.0.1.  If such a Kudu cluster is running on a deprecated
OS (e.g., RHEL/CentOS 6.4), the following options are available to
work around the incompatibility:
  ** use Kudu clients of 1.14 or earlier versions to communicate
 with such cluster
  ** disable RPC encryption and authentication for Kudu RPC, setting
 --rpc_authentication=disabled and --rpc_encryption=disabled
 for all masters and tablet servers in the cluster to allow the
 new client to work with the old cluster

  * TLSv1.2 is the minimum TLS protocol version that newer Kudu servers
are able to use for Kudu RPC.  The newer servers are not able to
communicate using secure Kudu RPC with Kudu C++ client applications
linked with libkudu_client library built against OpenSSL of versions
prior to 1.0.1 or with Java client applications run with outdated
Java runtime that doesn't support TLSv1.2.  The following options
are available to work around this incompatibility:
  ** customize settings for the --rpc_tls_min_protocol and
 --rpc_tls_ciphers flags on all masters and tablet servers
 in the cluster, setting --rpc_tls_min_protocol=TLSv1 and
 adding TLSv1-capable cipher suites (e.g. AES128-SHA and
 AES256-SHA) into the list
  ** disable RPC encryption and authentication for Kudu RPC, setting
 --rpc_authentication=disabled and --rpc_encryption=disabled
 for all masters and tablet servers in the cluster to allow such
 Kudu clients to work with newer clusters

[1] https://kudu.apache.org/releases/1.14.0/docs/release_notes.html
[2] https://issues.apache.org/jira/browse/KUDU-2099
[3] https://wiki.mozilla.org/Security/Server_Side_TLS
[4] https://www.oracle.com/java/technologies/javase/8all-relnotes.html

Change-Id: I07633a04d3828100f148e5de3905094198d13396
Reviewed-on: http://gerrit.cloudera.org:

[kudu] 02/02: [tools] run intra-location rebalancing in parallel

2022-03-17 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 91d3101008961338078d0b80e5f2e3d1dc552133
Author: Alexey Serbin 
AuthorDate: Wed Mar 9 14:57:45 2022 -0800

[tools] run intra-location rebalancing in parallel

This patch addresses TODO in RebalancerTool::Run() to run intra-location
rebalancing in parallel for location-aware Kudu cluster.  Since the
location assignment provides an invariant for a tablet server to be only
in one location, a locatation-aware Kudu cluster automatically consists
of non-intersecting set of tablet servers.  With that, it's possible to
independently move replicas within different locations while satisfying
various run-time properties, e.g. set by --max_moves_per_server and
other flags.

The new binary was tested on a large production cluster where location
awareness was enabled and it did significantly speed up the rebalancing
process.  Running the binary in my test cluster showed significant
reduction of run times as well.  Theoretically, running intra-location
rebalancing in parallel might shorten the runtime by N times compared
with running sequentially, where N is the number of locations
in a Kudu cluster.

Change-Id: Ie4fe3ef3ec2fcac57114c97d5b6cd81d5d9953c4
Reviewed-on: http://gerrit.cloudera.org:8080/18308
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 src/kudu/rebalance/rebalancer.cc   |   7 +-
 src/kudu/rebalance/rebalancer.h|   7 +-
 src/kudu/tools/rebalancer_tool-test.cc |  26 ++
 src/kudu/tools/rebalancer_tool.cc  | 151 -
 src/kudu/tools/rebalancer_tool.h   |  15 +++-
 src/kudu/tools/tool_action_cluster.cc  |  13 ++-
 6 files changed, 191 insertions(+), 28 deletions(-)

diff --git a/src/kudu/rebalance/rebalancer.cc b/src/kudu/rebalance/rebalancer.cc
index 2d5fdc2..f93dc99 100644
--- a/src/kudu/rebalance/rebalancer.cc
+++ b/src/kudu/rebalance/rebalancer.cc
@@ -73,7 +73,8 @@ Rebalancer::Config::Config(vector 
ignored_tservers_param,
bool run_cross_location_rebalancing,
bool run_intra_location_rebalancing,
double load_imbalance_threshold,
-   bool 
force_rebalance_replicas_on_maintenance_tservers)
+   bool 
force_rebalance_replicas_on_maintenance_tservers,
+   size_t intra_location_rebalancing_concurrency)
 : ignored_tservers(ignored_tservers_param.begin(), 
ignored_tservers_param.end()),
   master_addresses(std::move(master_addresses)),
   table_filters(std::move(table_filters)),
@@ -88,7 +89,9 @@ Rebalancer::Config::Config(vector 
ignored_tservers_param,
   run_intra_location_rebalancing(run_intra_location_rebalancing),
   load_imbalance_threshold(load_imbalance_threshold),
   force_rebalance_replicas_on_maintenance_tservers(
-  force_rebalance_replicas_on_maintenance_tservers) {
+  force_rebalance_replicas_on_maintenance_tservers),
+  intra_location_rebalancing_concurrency(
+  intra_location_rebalancing_concurrency) {
   DCHECK_GE(max_moves_per_server, 0);
 }
 
diff --git a/src/kudu/rebalance/rebalancer.h b/src/kudu/rebalance/rebalancer.h
index 15cf5bc..079fbcc 100644
--- a/src/kudu/rebalance/rebalancer.h
+++ b/src/kudu/rebalance/rebalancer.h
@@ -68,7 +68,8 @@ class Rebalancer {
bool run_cross_location_rebalancing = true,
bool run_intra_location_rebalancing = true,
double load_imbalance_threshold = kLoadImbalanceThreshold,
-   bool force_rebalance_replicas_on_maintenance_tservers = false);
+   bool force_rebalance_replicas_on_maintenance_tservers = false,
+   size_t intra_location_rebalancing_concurrency = 0);
 
 // UUIDs of ignored servers. If empty, run the rebalancing on
 // all tablet servers in the cluster only when all tablet servers
@@ -136,6 +137,10 @@ class Rebalancer {
 double load_imbalance_threshold;
 
 bool force_rebalance_replicas_on_maintenance_tservers;
+
+// The maximum number of intra-location rebalancing sessions that can be 
run
+// in parallel. Value of 0 means 'the number of CPU cores at the node'.
+size_t intra_location_rebalancing_concurrency;
   };
 
   // Represents a concrete move of a replica from one tablet server to another.
diff --git a/src/kudu/tools/rebalancer_tool-test.cc 
b/src/kudu/tools/rebalancer_tool-test.cc
index 37437c7..e9af759 100644
--- a/src/kudu/tools/rebalancer_tool-test.cc
+++ b/src/kudu/tools/rebalancer_tool-test.cc
@@ -1759,6 +1759,32 @@ TEST_F(LocationAwareRebalancingBasicTest, Basic) {
   }
 }
 
+// Test that rebalancing tasks are scheduled for each of the existing locations
+// even if running the intra-location rebalancing in a non-concurrent

[kudu] branch master updated (c45eba4 -> 91d3101)

2022-03-17 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git.


from c45eba4  [tool] Fix rebuild master tool's help infomation
 new df65b3a  [write_throttling-itest] relax threshold to avoid flakiness
 new 91d3101  [tools] run intra-location rebalancing in parallel

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../integration-tests/write_throttling-itest.cc|   2 +-
 src/kudu/rebalance/rebalancer.cc   |   7 +-
 src/kudu/rebalance/rebalancer.h|   7 +-
 src/kudu/tools/rebalancer_tool-test.cc |  26 
 src/kudu/tools/rebalancer_tool.cc  | 151 ++---
 src/kudu/tools/rebalancer_tool.h   |  15 +-
 src/kudu/tools/tool_action_cluster.cc  |  13 +-
 7 files changed, 192 insertions(+), 29 deletions(-)


[kudu] 01/02: [write_throttling-itest] relax threshold to avoid flakiness

2022-03-17 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit df65b3a3d04115fc38bdb739207123816999557d
Author: Alexey Serbin 
AuthorDate: Mon Mar 14 17:13:16 2022 -0700

[write_throttling-itest] relax threshold to avoid flakiness

I saw a failure [1] of the WriteThrottlingTest.ThrottleWriteRpcPerSec
test scenario during gerrit pre-commit tests for [2]:

  src/kudu/integration-tests/write_throttling-itest.cc:107
  Expected: (qps) <= (TARGET_QPS * 1.2f), actual: 121.467 vs 120

This patch relaxes the threshold for the target QPS: 1.2 --> 1.25.
FWIW, the token-based accounting could be affected by scheduler
anomalies and bursts in request rates, so there isn't anything
unexpected here.

[1] http://jenkins.kudu.apache.org/job/kudu-gerrit/25223/
[2] http://gerrit.cloudera.org:8080/18318

Change-Id: Iad3335595b02e66cdc588755b8f53c77442d5736
Reviewed-on: http://gerrit.cloudera.org:8080/18320
Tested-by: Kudu Jenkins
Reviewed-by: Andrew Wong 
---
 src/kudu/integration-tests/write_throttling-itest.cc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/kudu/integration-tests/write_throttling-itest.cc 
b/src/kudu/integration-tests/write_throttling-itest.cc
index 8db7a2a..839db17 100644
--- a/src/kudu/integration-tests/write_throttling-itest.cc
+++ b/src/kudu/integration-tests/write_throttling-itest.cc
@@ -104,7 +104,7 @@ TEST_F(WriteThrottlingTest, ThrottleWriteRpcPerSec) {
 MonoDelta delta = end - begin;
 double qps = TARGET_QPS / delta.ToSeconds();
 LOG(INFO) << "Iteration " << t << " qps: " << qps;
-ASSERT_LE(qps, TARGET_QPS * 1.2f);
+ASSERT_LE(qps, TARGET_QPS * 1.25f);
   }
 }
 


[kudu] branch master updated: [tool] Fix rebuild master tool's help infomation

2022-03-16 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new c45eba4  [tool] Fix rebuild master tool's help infomation
c45eba4 is described below

commit c45eba48ed37c04700b25ffd8664ce7c9c27135b
Author: shenxingwuying 
AuthorDate: Mon Mar 14 23:39:29 2022 +0800

[tool] Fix rebuild master tool's help infomation

Change-Id: I301df64f0c1a42a00f19bced18f13aa5057b068c
Reviewed-on: http://gerrit.cloudera.org:8080/18319
Tested-by: Kudu Jenkins
Reviewed-by: Yingchun Lai 
Reviewed-by: Andrew Wong 
Reviewed-by: Alexey Serbin 
---
 src/kudu/tools/tool_action_master.cc | 8 
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/src/kudu/tools/tool_action_master.cc 
b/src/kudu/tools/tool_action_master.cc
index 48cb94a..12a672b 100644
--- a/src/kudu/tools/tool_action_master.cc
+++ b/src/kudu/tools/tool_action_master.cc
@@ -112,10 +112,10 @@ namespace kudu {
 namespace tools {
 namespace {
 
-const char* const kTabletServerAddressArg = "tserver_address";
-const char* const kTabletServerAddressDesc = "Address of a Kudu tablet server "
-"of form 'hostname:port'. Port may be omitted if the tablet server is "
-"bound to the default port.";
+const char* const kTabletServerAddressArg = "tserver_addresses";
+const char* const kTabletServerAddressDesc = "Address list of Kudu tablet 
servers"
+" of form 'hostname-0:port-0 hostname-1:port-1 ... hostname-n:port-n'. 
Port may"
+" be omitted if the tablet server is bound to the default port.";
 const char* const kFlagArg = "flag";
 const char* const kValueArg = "value";
 


[kudu] branch master updated: [tools] Support copying unpartitioned table

2022-03-14 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 7189b51  [tools] Support copying unpartitioned table
7189b51 is described below

commit 7189b519625d8f6ef7f70bc259733a672a28267f
Author: zhangyifan27 
AuthorDate: Mon Mar 14 23:08:00 2022 +0800

[tools] Support copying unpartitioned table

The "kudu table copy" tool generated an error while copying
an unpartitioned table: Invalid argument: Table partitioning must be
specified using add_hash_partitions or set_range_partition_columns
This patch fixed it.

Change-Id: I4fcdcf93aadfaa2df59e59afa7bb3524ede2ac60
Reviewed-on: http://gerrit.cloudera.org:8080/18318
Reviewed-by: Mahesh Reddy 
Reviewed-by: Andrew Wong 
Reviewed-by: Alexey Serbin 
Tested-by: Alexey Serbin 
---
 src/kudu/tools/kudu-tool-test.cc | 22 ++
 src/kudu/tools/table_scanner.cc  |  6 ++
 2 files changed, 28 insertions(+)

diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 1b73a3b..793e738 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -698,6 +698,7 @@ enum RunCopyTableCheckArgsType {
   kTestCopyTableUpsert,
   kTestCopyTableSchemaOnly,
   kTestCopyTableComplexSchema,
+  kTestCopyUnpartitionedTable,
   kTestCopyTablePredicates
 };
 // Subclass of ToolTest that allows running individual test cases with 
different parameters to run
@@ -721,6 +722,10 @@ class ToolTestCopyTableParameterized :
   KuduSchema schema;
   ASSERT_OK(CreateComplexSchema());
   ww.set_schema(schema);
+} else if (test_case_ == kTestCopyUnpartitionedTable) {
+  KuduSchema schema;
+  ASSERT_OK(CreateUnpartitionedTable());
+  ww.set_schema(schema);
 }
 ww.Setup();
 ww.Start();
@@ -759,6 +764,9 @@ class ToolTestCopyTableParameterized :
 args.columns = kComplexSchemaColumns;
 args.mode = TableCopyMode::INSERT_TO_NOT_EXIST_TABLE;
 return { args };
+  case kTestCopyUnpartitionedTable:
+args.mode = TableCopyMode::INSERT_TO_NOT_EXIST_TABLE;
+return {args};
   case kTestCopyTablePredicates: {
 auto mid = total_rows_ / 2;
 vector multi_args;
@@ -898,6 +906,19 @@ class ToolTestCopyTableParameterized :
 return Status::OK();
   }
 
+  Status CreateUnpartitionedTable(KuduSchema* schema) {
+shared_ptr client;
+RETURN_NOT_OK(cluster_->CreateClient(nullptr, ));
+unique_ptr table_creator(client->NewTableCreator());
+*schema = KuduSchema::FromSchema(GetSimpleTestSchema());
+RETURN_NOT_OK(table_creator->table_name(kTableName)
+  .schema(schema)
+  .set_range_partition_columns({})
+  .num_replicas(1)
+  .Create());
+return Status::OK();
+  }
+
   void InsertOneRowWithNullCell() {
 shared_ptr client;
 ASSERT_OK(cluster_->CreateClient(nullptr, ));
@@ -932,6 +953,7 @@ INSTANTIATE_TEST_SUITE_P(CopyTableParameterized,
kTestCopyTableUpsert,
kTestCopyTableSchemaOnly,
kTestCopyTableComplexSchema,
+   kTestCopyUnpartitionedTable,
kTestCopyTablePredicates));
 
 void ToolTest::StartExternalMiniCluster(ExternalMiniClusterOptions opts) {
diff --git a/src/kudu/tools/table_scanner.cc b/src/kudu/tools/table_scanner.cc
index 3c7923f..8be3b4c 100644
--- a/src/kudu/tools/table_scanner.cc
+++ b/src/kudu/tools/table_scanner.cc
@@ -461,6 +461,12 @@ Status CreateDstTableIfNeeded(const 
client::sp::shared_ptr& src_table
 table_creator->add_range_partition(lower.release(), upper.release());
   }
 
+  if (partition_schema.hash_schema().empty() &&
+  partition_schema.range_schema().column_ids.empty()) {
+// This src table is unpartitioned, just create a table range partitioned 
on no columns.
+table_creator->set_range_partition_columns({});
+  }
+
   // Create table.
   RETURN_NOT_OK(table_creator->Create());
   LOG(INFO) << "Table " << dst_table_name << " created successfully";


[kudu] branch master updated: [tablet] add tablet_id into trace of a slow write op

2022-03-14 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 2a5e565  [tablet] add tablet_id into trace of a slow write op
2a5e565 is described below

commit 2a5e56594a9dac28df9b02365d340bbd6a8e480f
Author: Alexey Serbin 
AuthorDate: Fri Mar 11 14:16:27 2022 -0800

[tablet] add tablet_id into trace of a slow write op

While troubleshooting one performance issue, I found that the traces
on long running write operations didn't contain information on their
tablet identifier.

This patch address that, so now it's easier to tell what's the tablet
where write operations run longer.  I also took the liberty to update
the code around, making the style of the trace messages more uniform.

Change-Id: I06b05e8be1f5376b60244228809be9d260ed8398
Reviewed-on: http://gerrit.cloudera.org:8080/18315
Tested-by: Kudu Jenkins
Reviewed-by: Andrew Wong 
---
 src/kudu/tablet/ops/write_op.cc | 37 +++--
 src/kudu/tablet/ops/write_op.h  |  2 +-
 2 files changed, 20 insertions(+), 19 deletions(-)

diff --git a/src/kudu/tablet/ops/write_op.cc b/src/kudu/tablet/ops/write_op.cc
index 7f1defb..b5ae129 100644
--- a/src/kudu/tablet/ops/write_op.cc
+++ b/src/kudu/tablet/ops/write_op.cc
@@ -159,7 +159,9 @@ void WriteOp::NewReplicateMsg(unique_ptr* 
replicate_msg) {
 
 Status WriteOp::Prepare() {
   TRACE_EVENT0("op", "WriteOp::Prepare");
-  TRACE("PREPARE: Starting.");
+
+  Tablet* tablet = state()->tablet_replica()->tablet();
+  TRACE(Substitute("PREPARE: Starting on tablet $0", tablet->tablet_id()));
   // Decode everything first so that we give up if something major is wrong.
   Schema client_schema;
   RETURN_NOT_OK_PREPEND(SchemaFromPB(state_->request()->schema(), 
_schema),
@@ -172,8 +174,6 @@ Status WriteOp::Prepare() {
 return s;
   }
 
-  Tablet* tablet = state()->tablet_replica()->tablet();
-
   // Before taking any other locks, acquire the transaction state lock and
   // ensure it is open.
   Status s;
@@ -224,7 +224,7 @@ Status WriteOp::Prepare() {
   }
   RETURN_NOT_OK(tablet->AcquireRowLocks(state()));
 
-  TRACE("PREPARE: Finished.");
+  TRACE("PREPARE: Finished");
   return Status::OK();
 }
 
@@ -264,7 +264,7 @@ void WriteOp::UpdatePerRowMetricsAndErrors() {
 // it seems pointless to return a Status!
 Status WriteOp::Apply(CommitMsg** commit_msg) {
   TRACE_EVENT0("op", "WriteOp::Apply");
-  TRACE("APPLY: Starting.");
+  TRACE("APPLY: Starting");
 
   if (PREDICT_FALSE(ANNOTATE_UNPROTECTED_READ(
   FLAGS_tablet_inject_latency_on_apply_write_op_ms) > 0)) {
@@ -275,7 +275,7 @@ Status WriteOp::Apply(CommitMsg** commit_msg) {
 
   Tablet* tablet = state()->tablet_replica()->tablet();
   RETURN_NOT_OK(tablet->ApplyRowOperations(state()));
-  TRACE("APPLY: Finished.");
+  TRACE("APPLY: Finished");
 
   UpdatePerRowMetricsAndErrors();
 
@@ -293,29 +293,30 @@ void WriteOp::Finish(OpResult result) {
   state()->FinishApplyingOrAbort(result);
 
   if (PREDICT_FALSE(result == Op::ABORTED)) {
-TRACE("FINISH: Op aborted.");
+TRACE("FINISH: Op aborted");
 return;
   }
 
   DCHECK_EQ(result, Op::APPLIED);
 
-  TRACE("FINISH: Updating metrics.");
+  TRACE("FINISH: Updating metrics");
 
-  TabletMetrics* metrics = state_->tablet_replica()->tablet()->metrics();
-  if (metrics) {
+  if (auto* metrics = state_->tablet_replica()->tablet()->metrics();
+  PREDICT_TRUE(metrics != nullptr)) {
 // TODO(unknown): should we change this so it's actually incremented by the
 // Tablet code itself instead of this wrapper code?
-metrics->rows_inserted->IncrementBy(state_->metrics().successful_inserts);
-
metrics->insert_ignore_errors->IncrementBy(state_->metrics().insert_ignore_errors);
-metrics->rows_upserted->IncrementBy(state_->metrics().successful_upserts);
-metrics->rows_updated->IncrementBy(state_->metrics().successful_updates);
-
metrics->update_ignore_errors->IncrementBy(state_->metrics().update_ignore_errors);
-metrics->rows_deleted->IncrementBy(state_->metrics().successful_deletes);
-
metrics->delete_ignore_errors->IncrementBy(state_->metrics().delete_ignore_errors);
+const auto& op_m = state_->metrics();
+metrics->rows_inserted->IncrementBy(op_m.successful_inserts);
+metrics->insert_ignore_errors->IncrementBy(op_m.insert_ignore_errors);
+metrics->rows_upserted->IncrementBy(op_m.successful_upserts);
+metrics->rows_updated->IncrementBy(op_m.successful_updates);
+metrics->update_ignore_e

[kudu] branch master updated: [tablet] output more info for long row lock waits

2022-03-09 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 8e704f2  [tablet] output more info for long row lock waits
8e704f2 is described below

commit 8e704f28b172e211477bd06409a429b20a1e55a1
Author: Alexey Serbin 
AuthorDate: Mon Mar 7 19:47:36 2022 -0800

[tablet] output more info for long row lock waits

While troubleshooting one performance issue, I found that the messages
on long-wait row locks didn't provide a lot of useful information
especially when log redaction was on (it's on by default).

This patch address that, so now it's easier to tell:
  * what's the tablet where the congestion happens
  * whether recurrent messages are about the same primary key
as for prior messages

Before:
  Waited 2 seconds to obtain row lock on key  cur holder: 
0x2f2e85180

After:
  Waited 2 seconds to obtain row lock on key '' (key hash 
8560710080558783806) tablet 18b47e40a6864ff39d1d82accf7202ec cur holder 
0x2f2e85180

Change-Id: I521976ff6986676ad9a33a810c56eaa8e7deeab8
Reviewed-on: http://gerrit.cloudera.org:8080/18300
Tested-by: Kudu Jenkins
Reviewed-by: Andrew Wong 
---
 src/kudu/tablet/lock_manager.cc| 18 ---
 src/kudu/tablet/tablet_replica-test.cc | 58 ++
 2 files changed, 71 insertions(+), 5 deletions(-)

diff --git a/src/kudu/tablet/lock_manager.cc b/src/kudu/tablet/lock_manager.cc
index bf9d57f..2981b5d 100644
--- a/src/kudu/tablet/lock_manager.cc
+++ b/src/kudu/tablet/lock_manager.cc
@@ -34,6 +34,8 @@
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/walltime.h"
+#include "kudu/tablet/ops/op.h"
+#include "kudu/tablet/tablet_replica.h"
 #include "kudu/tserver/tserver.pb.h"
 #include "kudu/util/array_view.h"
 #include "kudu/util/faststring.h"
@@ -53,8 +55,6 @@ using strings::Substitute;
 namespace kudu {
 namespace tablet {
 
-class OpState;
-
 // 
 //  LockTable
 // 
@@ -79,6 +79,10 @@ class LockEntry {
 return KUDU_REDACT(key_.ToDebugString());
   }
 
+  uint64_t key_hash() const {
+return key_hash_;
+  }
+
   void Unlock();
 
   // Mutex used by the LockManager
@@ -579,8 +583,11 @@ void LockManager::AcquireLockOnEntry(LockEntry* entry, 
const OpState* op) {
 int waited_seconds = 0;
 while (!entry->sem.TimedAcquire(MonoDelta::FromSeconds(1))) {
   const OpState* cur_holder = ANNOTATE_UNPROTECTED_READ(entry->holder_);
-  LOG(WARNING) << "Waited " << (++waited_seconds) << " seconds to obtain 
row lock on key "
-   << entry->ToString() << " cur holder: " << cur_holder;
+  LOG(WARNING) << Substitute(
+  "Waited $0 seconds to obtain row lock on key '$1' (key hash $2) "
+  "tablet $3 cur holder $4",
+  ++waited_seconds, entry->ToString(), entry->key_hash(),
+  op->tablet_replica()->tablet_id(), cur_holder);
   // TODO(unknown): would be nice to also include some info about the 
blocking op,
   // but it's a bit tricky to do in a non-racy fashion (the other op may
   // complete at any point)
@@ -588,7 +595,8 @@ void LockManager::AcquireLockOnEntry(LockEntry* entry, 
const OpState* op) {
 MicrosecondsInt64 wait_us = GetMonoTimeMicros() - start_wait_us;
 TRACE_COUNTER_INCREMENT("row_lock_wait_us", wait_us);
 if (wait_us > 100 * 1000) {
-  TRACE("Waited $0us for lock on $1", wait_us, 
KUDU_REDACT(entry->ToString()));
+  TRACE("Waited $0us for lock on $1 (key hash $2)",
+wait_us, entry->ToString(), entry->key_hash());
 }
   }
 
diff --git a/src/kudu/tablet/tablet_replica-test.cc 
b/src/kudu/tablet/tablet_replica-test.cc
index e6980a9..0e7734c 100644
--- a/src/kudu/tablet/tablet_replica-test.cc
+++ b/src/kudu/tablet/tablet_replica-test.cc
@@ -22,6 +22,7 @@
 #include 
 #include 
 #include 
+#include 
 #include 
 
 #include 
@@ -46,6 +47,7 @@
 #include "kudu/fs/fs_manager.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/ref_counted.h"
+#include "kudu/tablet/lock_manager.h"
 #include "kudu/tablet/ops/alter_schema_op.h"
 #include "kudu/tablet/ops/op.h"
 #include "kudu/tablet/ops/op_driver.h"
@@ -57,12 +59,14 @@
 #include "kudu/tablet/tablet_replica_mm_ops.h"
 #include "kudu/tserver/tserver.pb.h"
 #include "kudu/tserver/tserver_admin.pb.h

[kudu] branch master updated: [tools] correct rebalancer's initial status message

2022-03-09 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 9e952ad  [tools] correct rebalancer's initial status message
9e952ad is described below

commit 9e952ad9d87eefc9ef840cd3edbdd62dd9f0243e
Author: Alexey Serbin 
AuthorDate: Mon Mar 7 12:27:35 2022 -0800

[tools] correct rebalancer's initial status message

I noticed that the 'kudu cluster rebalance' tool was reporting that
it had been '...running whole-cluster rebalancing...' even when running
with a particular set of tables to rebalance, i.e. not a whole-cluster
rebalancing by any means if there is more than one table in a cluster.

This patch fixes the issue.

Change-Id: Ic224a07f9ce9ed3ca9a447022f183b799a1bebee
Reviewed-on: http://gerrit.cloudera.org:8080/18297
Reviewed-by: Abhishek Chennaka 
Tested-by: Alexey Serbin 
Reviewed-by: Andrew Wong 
---
 src/kudu/tools/rebalancer_tool.cc | 8 +++-
 1 file changed, 7 insertions(+), 1 deletion(-)

diff --git a/src/kudu/tools/rebalancer_tool.cc 
b/src/kudu/tools/rebalancer_tool.cc
index b69ca05..8c53b3e 100644
--- a/src/kudu/tools/rebalancer_tool.cc
+++ b/src/kudu/tools/rebalancer_tool.cc
@@ -39,6 +39,7 @@
 #include "kudu/gutil/basictypes.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/rebalance/cluster_status.h"
@@ -190,7 +191,12 @@ Status RebalancerTool::Run(RunStatus* result_status, 
size_t* moves_count) {
   }
   if (ts_id_by_location.size() == 1) {
 const auto& location = ts_id_by_location.cbegin()->first;
-LOG(INFO) << "running whole-cluster rebalancing";
+const auto& table_filters = config_.table_filters;
+const auto& msg = table_filters.empty()
+? "running whole-cluster rebalancing"
+: Substitute("running rebalancing for tables: $0",
+ JoinStrings(table_filters, ","));
+LOG(INFO) << msg;
 IntraLocationRunner runner(
 this, config_.ignored_tservers, config_.max_moves_per_server, 
deadline, location);
 RETURN_NOT_OK(runner.Init(config_.master_addresses));


[kudu] branch branch-1.16.x updated: [thirdparty] build curl without brotli

2022-03-04 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch branch-1.16.x
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/branch-1.16.x by this push:
 new fd18103  [thirdparty] build curl without brotli
fd18103 is described below

commit fd1810320be74fd5ac50f607fc2714c8f6d0b8df
Author: Alexey Serbin 
AuthorDate: Thu Mar 3 14:39:34 2022 -0800

[thirdparty] build curl without brotli

The default build configuration for curl has the brotli compression
opportunistically enabled.  If brotli libraries and headers are found
at the build machine, the configure script picks them up and then the
generated curl library depends on corresponding brotli libraries.  That
on itself requires extra brotli libraries to be linked in when building
statically linked kudu binaries (that's so for RELEASE builds).

This patch disables broli for the libcurl library build as a part of
Kudu's thirdparty.

Change-Id: I14ef696302cc1653f3efd6dba5714ce3927c1d20
Reviewed-on: http://gerrit.cloudera.org:8080/18288
Tested-by: Alexey Serbin 
Reviewed-by: Andrew Wong 
(cherry picked from commit 1750aedbbe20134b7dec52cd52ac6c2e330d77e0)
Reviewed-on: http://gerrit.cloudera.org:8080/18292
Reviewed-by: Alexey Serbin 
---
 thirdparty/build-definitions.sh | 1 +
 1 file changed, 1 insertion(+)

diff --git a/thirdparty/build-definitions.sh b/thirdparty/build-definitions.sh
index 7a1ce24..d851e32 100644
--- a/thirdparty/build-definitions.sh
+++ b/thirdparty/build-definitions.sh
@@ -742,6 +742,7 @@ build_curl() {
 --disable-smtp \
 --disable-telnet \
 --disable-tftp \
+--without-brotli \
 --without-libidn2 \
 --without-libpsl \
 --without-librtmp \


[kudu] 01/02: KUDU-3197 [tserver] optimal Schema's memory used, using std::shared_ptr

2022-03-03 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit f4b6d8917b79b9de53957174ade1a7ffc76e0090
Author: shenxingwuying 
AuthorDate: Tue Jan 4 20:07:29 2022 +0800

KUDU-3197 [tserver] optimal Schema's memory used, using std::shared_ptr

Change TabletMeta's variable Schema* to std::shared_ptr
to reduce memory used when alter schema.

Because TabletMeta save old_schemas to reserve the elder schemas
when alter schema, maybe they have been used by scanners or
compaction jobs. As jira KUDU-3197 said, frequently alter schema will
lead to tserver's memory becomes very large, just like memory leak,
especially column's number is very large.

The jira issued by wangningito, and I continue his work, and
now use std::shared_ptr instead of scoped_refptr, because
scoped_refptr causes too many changes, just as:
https://gerrit.cloudera.org/c/18098/

Change-Id: Ic284dde108c49130419d876c6698b40c195e9b35
Reviewed-on: http://gerrit.cloudera.org:8080/18255
Tested-by: Kudu Jenkins
Reviewed-by: Andrew Wong 
---
 src/kudu/client/client-test.cc | 10 ++---
 src/kudu/common/schema.h   |  1 +
 src/kudu/integration-tests/linked_list-test-util.h |  2 +-
 src/kudu/master/sys_catalog.cc |  5 ++-
 src/kudu/tablet/all_types-scan-correctness-test.cc |  2 +-
 src/kudu/tablet/cfile_set.cc   |  4 +-
 src/kudu/tablet/cfile_set.h|  2 +-
 src/kudu/tablet/diff_scan-test.cc  |  6 +--
 src/kudu/tablet/diskrowset.cc  |  8 ++--
 src/kudu/tablet/mt-tablet-test.cc  |  2 +-
 src/kudu/tablet/ops/alter_schema_op.cc | 10 ++---
 src/kudu/tablet/ops/alter_schema_op.h  |  8 ++--
 src/kudu/tablet/ops/op.h   |  4 +-
 src/kudu/tablet/ops/write_op.h |  7 ++-
 src/kudu/tablet/rowset_metadata.h  |  2 +-
 src/kudu/tablet/tablet-schema-test.cc  | 10 ++---
 src/kudu/tablet/tablet-test-util.h |  3 +-
 src/kudu/tablet/tablet.cc  | 47 +++-
 src/kudu/tablet/tablet.h   |  6 +--
 src/kudu/tablet/tablet_bootstrap.cc|  8 ++--
 src/kudu/tablet/tablet_metadata.cc | 51 +++---
 src/kudu/tablet/tablet_metadata.h  | 29 +---
 src/kudu/tablet/tablet_replica-test.cc | 10 ++---
 src/kudu/tools/kudu-tool-test.cc   |  8 ++--
 src/kudu/tools/tool_action_fs.cc   |  8 ++--
 src/kudu/tools/tool_action_local_replica.cc| 15 ---
 src/kudu/tserver/scanners.cc   |  5 ++-
 src/kudu/tserver/tablet_server-test.cc |  2 +-
 src/kudu/tserver/tablet_service.cc | 23 ++
 src/kudu/tserver/tserver_path_handlers.cc  | 12 ++---
 30 files changed, 161 insertions(+), 149 deletions(-)

diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index 94ec37d..75b2bce 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -4709,7 +4709,7 @@ TEST_F(ClientTest, TestBasicAlterOperations) {
 ->Default(KuduValue::CopyString("hello!"));
 ASSERT_OK(table_alterer->Alter());
 ASSERT_EQ(4, tablet_replica->tablet()->metadata()->schema_version());
-Schema schema = tablet_replica->tablet()->metadata()->schema();
+Schema schema = *tablet_replica->tablet()->metadata()->schema();
 ColumnSchema col_schema = schema.column(schema.find_column("string_val"));
 ASSERT_FALSE(col_schema.has_read_default());
 ASSERT_TRUE(col_schema.has_write_default());
@@ -4723,7 +4723,7 @@ TEST_F(ClientTest, TestBasicAlterOperations) {
 ->Default(KuduValue::FromInt(54321));
 ASSERT_OK(table_alterer->Alter());
 ASSERT_EQ(5, tablet_replica->tablet()->metadata()->schema_version());
-Schema schema = tablet_replica->tablet()->metadata()->schema();
+Schema schema = *tablet_replica->tablet()->metadata()->schema();
 ColumnSchema col_schema = 
schema.column(schema.find_column("non_null_with_default"));
 ASSERT_TRUE(col_schema.has_read_default()); // Started with a default
 ASSERT_TRUE(col_schema.has_write_default());
@@ -4737,7 +4737,7 @@ TEST_F(ClientTest, TestBasicAlterOperations) {
 ->RemoveDefault();
 ASSERT_OK(table_alterer->Alter());
 ASSERT_EQ(6, tablet_replica->tablet()->metadata()->schema_version());
-Schema schema = tablet_replica->tablet()->metadata()->schema();
+Schema schema = *tablet_replica->tablet()->metadata()->schema();
 ColumnSchema 

[kudu] 02/02: [thirdparty] build curl without brotli

2022-03-03 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 1750aedbbe20134b7dec52cd52ac6c2e330d77e0
Author: Alexey Serbin 
AuthorDate: Thu Mar 3 14:39:34 2022 -0800

[thirdparty] build curl without brotli

The default build configuration for curl has the brotli compression
opportunistically enabled.  If brotli libraries and headers are found
at the build machine, the configure script picks them up and then the
generated curl library depends on corresponding brotli libraries.  That
on itself requires extra brotli libraries to be linked in when building
statically linked kudu binaries (that's so for RELEASE builds).

This patch disables broli for the libcurl library build as a part of
Kudu's thirdparty.

Change-Id: I14ef696302cc1653f3efd6dba5714ce3927c1d20
Reviewed-on: http://gerrit.cloudera.org:8080/18288
Tested-by: Alexey Serbin 
Reviewed-by: Andrew Wong 
---
 thirdparty/build-definitions.sh | 1 +
 1 file changed, 1 insertion(+)

diff --git a/thirdparty/build-definitions.sh b/thirdparty/build-definitions.sh
index 7a1ce24..d851e32 100644
--- a/thirdparty/build-definitions.sh
+++ b/thirdparty/build-definitions.sh
@@ -742,6 +742,7 @@ build_curl() {
 --disable-smtp \
 --disable-telnet \
 --disable-tftp \
+--without-brotli \
 --without-libidn2 \
 --without-libpsl \
 --without-librtmp \


[kudu] branch master updated (43905d8 -> 1750aed)

2022-03-03 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git.


from 43905d8  [security] avoid allocating string in SimpleAcl::UserAllowed()
 new f4b6d89  KUDU-3197 [tserver] optimal Schema's memory used, using 
std::shared_ptr
 new 1750aed  [thirdparty] build curl without brotli

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/kudu/client/client-test.cc | 10 ++---
 src/kudu/common/schema.h   |  1 +
 src/kudu/integration-tests/linked_list-test-util.h |  2 +-
 src/kudu/master/sys_catalog.cc |  5 ++-
 src/kudu/tablet/all_types-scan-correctness-test.cc |  2 +-
 src/kudu/tablet/cfile_set.cc   |  4 +-
 src/kudu/tablet/cfile_set.h|  2 +-
 src/kudu/tablet/diff_scan-test.cc  |  6 +--
 src/kudu/tablet/diskrowset.cc  |  8 ++--
 src/kudu/tablet/mt-tablet-test.cc  |  2 +-
 src/kudu/tablet/ops/alter_schema_op.cc | 10 ++---
 src/kudu/tablet/ops/alter_schema_op.h  |  8 ++--
 src/kudu/tablet/ops/op.h   |  4 +-
 src/kudu/tablet/ops/write_op.h |  7 ++-
 src/kudu/tablet/rowset_metadata.h  |  2 +-
 src/kudu/tablet/tablet-schema-test.cc  | 10 ++---
 src/kudu/tablet/tablet-test-util.h |  3 +-
 src/kudu/tablet/tablet.cc  | 47 +++-
 src/kudu/tablet/tablet.h   |  6 +--
 src/kudu/tablet/tablet_bootstrap.cc|  8 ++--
 src/kudu/tablet/tablet_metadata.cc | 51 +++---
 src/kudu/tablet/tablet_metadata.h  | 29 +---
 src/kudu/tablet/tablet_replica-test.cc | 10 ++---
 src/kudu/tools/kudu-tool-test.cc   |  8 ++--
 src/kudu/tools/tool_action_fs.cc   |  8 ++--
 src/kudu/tools/tool_action_local_replica.cc| 15 ---
 src/kudu/tserver/scanners.cc   |  5 ++-
 src/kudu/tserver/tablet_server-test.cc |  2 +-
 src/kudu/tserver/tablet_service.cc | 23 ++
 src/kudu/tserver/tserver_path_handlers.cc  | 12 ++---
 thirdparty/build-definitions.sh|  1 +
 31 files changed, 162 insertions(+), 149 deletions(-)


[kudu] branch master updated: [tablet] don't call UpdateMetricsForOp() for rows with errors

2022-02-28 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new a2ba855  [tablet] don't call UpdateMetricsForOp() for rows with errors
a2ba855 is described below

commit a2ba855deaba72846362c95d3614bdb3dd282a87
Author: Alexey Serbin 
AuthorDate: Mon Feb 28 10:16:28 2022 -0800

[tablet] don't call UpdateMetricsForOp() for rows with errors

This patch updates the control path in WriteOp::UpdatePerRowErrors()
to avoid calling UpdateMetricsForOp() for a row that contains an error
since that's effectively a no-op.

Also, renamed UpdatePerRowErrors() into UpdatePerRowMetricsAndErrors().

Change-Id: Ic1f57ee7d1b0064569a34ba93d35979426f76812
Reviewed-on: http://gerrit.cloudera.org:8080/18281
Tested-by: Alexey Serbin 
Reviewed-by: Andrew Wong 
---
 src/kudu/tablet/ops/write_op.cc | 21 ++---
 src/kudu/tablet/ops/write_op.h  |  7 +--
 2 files changed, 15 insertions(+), 13 deletions(-)

diff --git a/src/kudu/tablet/ops/write_op.cc b/src/kudu/tablet/ops/write_op.cc
index 83097f9..7f1defb 100644
--- a/src/kudu/tablet/ops/write_op.cc
+++ b/src/kudu/tablet/ops/write_op.cc
@@ -243,19 +243,20 @@ Status WriteOp::Start() {
   return Status::OK();
 }
 
-void WriteOp::UpdatePerRowErrors() {
-  // Add per-row errors to the result, update metrics.
-  for (int i = 0; i < state()->row_ops().size(); ++i) {
-const RowOp* op = state()->row_ops()[i];
+void WriteOp::UpdatePerRowMetricsAndErrors() {
+  // Update metrics or add per-row errors to the result.
+  size_t idx = 0;
+  for (const auto* op : state()->row_ops()) {
 if (op->result->has_failed_status()) {
   // Replicas disregard the per row errors, for now
   // TODO(unknown): check the per-row errors against the leader's, at 
least in debug mode
   WriteResponsePB::PerRowErrorPB* error = 
state()->response()->add_per_row_errors();
-  error->set_row_index(i);
+  error->set_row_index(idx);
   error->mutable_error()->CopyFrom(op->result->failed_status());
+} else {
+  state()->UpdateMetricsForOp(*op);
 }
-
-state()->UpdateMetricsForOp(*op);
+++idx;
   }
 }
 
@@ -276,7 +277,7 @@ Status WriteOp::Apply(CommitMsg** commit_msg) {
   RETURN_NOT_OK(tablet->ApplyRowOperations(state()));
   TRACE("APPLY: Finished.");
 
-  UpdatePerRowErrors();
+  UpdatePerRowMetricsAndErrors();
 
   // Create the Commit message
   *commit_msg = 
google::protobuf::Arena::CreateMessage(state_->pb_arena());
@@ -477,9 +478,7 @@ void WriteOpState::ReleaseTxResultPB(TxResultPB* result) 
const {
 }
 
 void WriteOpState::UpdateMetricsForOp(const RowOp& op) {
-  if (op.result->has_failed_status()) {
-return;
-  }
+  DCHECK(!op.result->has_failed_status());
   switch (op.decoded_op.type) {
 case RowOperationsPB::INSERT:
   DCHECK(!op.error_ignored);
diff --git a/src/kudu/tablet/ops/write_op.h b/src/kudu/tablet/ops/write_op.h
index c8621fa..8c14176 100644
--- a/src/kudu/tablet/ops/write_op.h
+++ b/src/kudu/tablet/ops/write_op.h
@@ -29,7 +29,7 @@
 #include 
 
 #include "kudu/common/row_operations.h"
-#include "kudu/common/wire_protocol.pb.h"
+#include "kudu/common/row_operations.pb.h"
 #include "kudu/consensus/consensus.pb.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/ref_counted.h"
@@ -385,7 +385,10 @@ class WriteOp : public Op {
   std::string ToString() const override;
 
  private:
-  void UpdatePerRowErrors();
+  // For each row of this write operation, update corresponding metrics or set
+  // corresponding error information in the response. The former is for
+  // successfully written rows, the latter is for failed ones.
+  void UpdatePerRowMetricsAndErrors();
 
   // this op's start time
   MonoTime start_time_;


[kudu] branch master updated: [security] handle a few unexpected authn token conditions

2022-02-25 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 24f85ce  [security] handle a few unexpected authn token conditions
24f85ce is described below

commit 24f85cedcd47e585ea9c308c317da963c5ab8fa9
Author: Alexey Serbin 
AuthorDate: Wed Feb 23 21:22:23 2022 -0800

[security] handle a few unexpected authn token conditions

This patch addresses a few unexpected error conditions in the
server-side negotiation code when authenticating a client with
its authn token.

I didn't add any tests since there is no mock for the corresponding
client side and implementing it from scratch just to verify these
two simple cases looked like an overkill.

Change-Id: Ic05ff6a9b289877d8440b94f00b2375da938c901
Reviewed-on: http://gerrit.cloudera.org:8080/18271
Tested-by: Alexey Serbin 
Reviewed-by: Attila Bukor 
Reviewed-by: Andrew Wong 
---
 src/kudu/rpc/server_negotiation.cc | 22 ++
 1 file changed, 14 insertions(+), 8 deletions(-)

diff --git a/src/kudu/rpc/server_negotiation.cc 
b/src/kudu/rpc/server_negotiation.cc
index bcbebf6..025c36a 100644
--- a/src/kudu/rpc/server_negotiation.cc
+++ b/src/kudu/rpc/server_negotiation.cc
@@ -688,12 +688,17 @@ Status ServerNegotiation::AuthenticateByToken(faststring* 
recv_buf) {
   NegotiatePB pb;
   RETURN_NOT_OK(RecvNegotiatePB(, recv_buf));
 
-  if (pb.step() != NegotiatePB::TOKEN_EXCHANGE) {
-Status s =  Status::NotAuthorized("expected TOKEN_EXCHANGE step",
-  
NegotiatePB::NegotiateStep_Name(pb.step()));
+  if (PREDICT_FALSE(pb.step() != NegotiatePB::TOKEN_EXCHANGE)) {
+const auto s = Status::NotAuthorized("expected TOKEN_EXCHANGE step",
+ 
NegotiatePB::NegotiateStep_Name(pb.step()));
+RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+return s;
   }
-  if (!pb.has_authn_token()) {
-Status s = Status::NotAuthorized("TOKEN_EXCHANGE message must include an 
authentication token");
+  if (PREDICT_FALSE(!pb.has_authn_token())) {
+const auto s = Status::NotAuthorized(
+"TOKEN_EXCHANGE message must include an authentication token");
+RETURN_NOT_OK(SendError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+return s;
   }
 
   // TODO(KUDU-1924): propagate the specific token verification failure back 
to the client,
@@ -701,9 +706,10 @@ Status ServerNegotiation::AuthenticateByToken(faststring* 
recv_buf) {
   security::TokenPB token;
   auto verification_result = 
token_verifier_->VerifyTokenSignature(pb.authn_token(), );
   ErrorStatusPB::RpcErrorCodePB error;
-  Status s = ParseTokenVerificationResult(verification_result,
-  ErrorStatusPB::FATAL_INVALID_AUTHENTICATION_TOKEN, );
-  if (!s.ok()) {
+  if (const auto s = ParseTokenVerificationResult(
+verification_result,
+ErrorStatusPB::FATAL_INVALID_AUTHENTICATION_TOKEN,
+); !s.ok()) {
 RETURN_NOT_OK(SendError(error, s));
 return s;
   }


[kudu] branch master updated: [tests] harmonize assertions on the expected status

2022-02-23 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new cd48ce7  [tests] harmonize assertions on the expected status
cd48ce7 is described below

commit cd48ce74025dd2d8287eb66b9d615a492b2267bb
Author: Alexey Serbin 
AuthorDate: Tue Nov 23 16:37:46 2021 -0800

[tests] harmonize assertions on the expected status

I was trying to understand why a test scenario from security-itest
failed in one of gerrit pre-commit runs and found that it would be
great to know the actual vs the expected type of the status when an
assertion of type ASSERT_{FALSE,TRUE}(s.IsXxx()) triggers.

This patch updates the corresponding call sites of ASSERT_{FALSE,TRUE}
in the code of that and other tests to provide such information for
easier troubleshooting and debugging in future.

Change-Id: I09cba638a5328fc57cc1ca70196bab42f09569b4
Reviewed-on: http://gerrit.cloudera.org:8080/18269
Reviewed-by: Andrew Wong 
Tested-by: Kudu Jenkins
---
 src/kudu/cfile/cfile-test.cc   |  2 +-
 src/kudu/client/client-test.cc | 52 +++---
 src/kudu/client/client-unittest.cc |  2 +-
 src/kudu/client/predicate-test.cc  |  2 +-
 src/kudu/common/schema-test.cc |  6 +--
 src/kudu/consensus/consensus_meta-test.cc  |  2 +-
 src/kudu/consensus/consensus_queue-test.cc |  2 +-
 src/kudu/consensus/leader_election-test.cc |  6 +--
 src/kudu/consensus/log-test.cc |  6 +--
 src/kudu/fs/data_dirs-test.cc  | 10 ++---
 src/kudu/fs/fs_manager-test.cc | 14 +++---
 src/kudu/fs/log_block_manager-test.cc  |  6 +--
 src/kudu/integration-tests/alter_table-test.cc | 18 
 src/kudu/integration-tests/delete_table-itest.cc   |  3 +-
 src/kudu/integration-tests/delete_tablet-itest.cc  |  2 +-
 src/kudu/integration-tests/master_authz-itest.cc   | 10 ++---
 .../integration-tests/master_replication-itest.cc  |  2 +-
 src/kudu/integration-tests/raft_consensus-itest.cc |  4 +-
 src/kudu/integration-tests/security-itest.cc   | 12 ++---
 .../integration-tests/tablet_replacement-itest.cc  |  2 +-
 .../timestamp_advancement-itest.cc |  2 +-
 .../tombstoned_voting-imc-itest.cc |  6 +--
 src/kudu/integration-tests/txn_write_ops-itest.cc  |  2 +-
 src/kudu/master/dynamic_multi_master-test.cc   |  2 +-
 src/kudu/master/master-test.cc |  2 +-
 src/kudu/rpc/exactly_once_rpc-test.cc  |  2 +-
 src/kudu/rpc/rpc-test.cc   | 10 ++---
 src/kudu/rpc/rpc_stub-test.cc  |  2 +-
 src/kudu/security/ca/cert_management-test.cc   |  2 +-
 src/kudu/server/webserver-test.cc  |  2 +-
 src/kudu/tablet/deltafile-test.cc  |  4 +-
 src/kudu/tablet/diskrowset-test.cc |  4 +-
 src/kudu/tablet/mvcc-test.cc   |  4 +-
 src/kudu/tablet/ops/op_tracker-test.cc |  2 +-
 src/kudu/tablet/tablet-schema-test.cc  |  2 +-
 src/kudu/tablet/tablet_bootstrap-test.cc   |  4 +-
 src/kudu/tablet/txn_participant-test.cc|  8 ++--
 src/kudu/tserver/tablet_copy_client-test.cc| 10 ++---
 src/kudu/util/net/socket-test.cc   |  2 +-
 src/kudu/util/oid_generator-test.cc|  2 +-
 src/kudu/util/threadpool-test.cc   |  2 +-
 src/kudu/util/yamlreader-test.cc   |  2 +-
 42 files changed, 121 insertions(+), 120 deletions(-)

diff --git a/src/kudu/cfile/cfile-test.cc b/src/kudu/cfile/cfile-test.cc
index 4294eee..bd79ba8 100644
--- a/src/kudu/cfile/cfile-test.cc
+++ b/src/kudu/cfile/cfile-test.cc
@@ -917,7 +917,7 @@ TEST_P(TestCFileBothCacheMemoryTypes, TestDataCorruption) {
   for (size_t i = 0; i < file_size; i++) {
 for (uint8_t flip = 0; flip < 8; flip++) {
   Status s = CorruptAndReadBlock(id, i, flip);
-  ASSERT_TRUE(s.IsCorruption());
+  ASSERT_TRUE(s.IsCorruption()) << s.ToString();
   ASSERT_STR_MATCHES(s.ToString(), "block [0-9]+");
 }
   }
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index fc9c768..94ec37d 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -937,7 +937,7 @@ TEST_F(ClientTest, TestGetTableStatistics) {
 TEST_F(ClientTest, TestBadTable) {
   shared_ptr t;
   Status s = client_->OpenTable("xxx-does-not-exist", );
-  ASSERT_TRUE(s.IsNotFound());
+  ASSERT_TRUE(s.IsNotFound()) << s.ToString();
   ASSERT_STR_CONTAINS(s.ToString(), "Not found: the table does not exist");
 }
 
@@ -1008,7 +1008,7 @@ TEST_F(ClientTest, TestConfiguringScanne

[kudu] branch branch-1.16.x updated: [tool] fix a command bug, cmd: kudu wal dump ...

2022-02-21 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch branch-1.16.x
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/branch-1.16.x by this push:
 new f5e06a5  [tool] fix a command bug, cmd: kudu wal dump ...
f5e06a5 is described below

commit f5e06a556905574ee56a948f969cf235b894b04c
Author: shenxingwuying 
AuthorDate: Thu Jan 20 23:04:19 2022 +0800

[tool] fix a command bug, cmd: kudu wal dump ...

kudu wal dump command will interrupt after reading alter schema entry.

Change-Id: I27acc71597d038cafbbe687117bddb1ce16576c0
Reviewed-on: http://gerrit.cloudera.org:8080/18169
Tested-by: Kudu Jenkins
Reviewed-by: Abhishek Chennaka 
Reviewed-by: Andrew Wong 
(cherry picked from commit 9f164b3c542a2e44c963c43488b4fc5b0d0f0f65)
Reviewed-on: http://gerrit.cloudera.org:8080/18260
Tested-by: Alexey Serbin 
---
 src/kudu/common/wire_protocol-test-util.h |  41 +-
 src/kudu/tools/kudu-tool-test.cc  | 218 ++
 src/kudu/tools/tool_action_common.cc  |  14 +-
 3 files changed, 269 insertions(+), 4 deletions(-)

diff --git a/src/kudu/common/wire_protocol-test-util.h 
b/src/kudu/common/wire_protocol-test-util.h
index a5a0412..2fcf03b 100644
--- a/src/kudu/common/wire_protocol-test-util.h
+++ b/src/kudu/common/wire_protocol-test-util.h
@@ -20,6 +20,7 @@
 
 #include "kudu/common/wire_protocol.h"
 
+#include 
 #include 
 
 #include "kudu/common/partial_row.h"
@@ -35,6 +36,13 @@ inline Schema GetSimpleTestSchema() {
 1);
 }
 
+inline void RowAppendColumn(KuduPartialRow* row,
+const std::map& columns) 
{
+  for (const auto& column : columns) {
+CHECK_OK(row->SetStringCopy(column.first.c_str(), column.second.c_str()));
+  }
+}
+
 inline void AddTestRowWithNullableStringToPB(RowOperationsPB::Type op_type,
  const Schema& schema,
  int32_t key,
@@ -46,12 +54,32 @@ inline void 
AddTestRowWithNullableStringToPB(RowOperationsPB::Type op_type,
   CHECK_OK(row.SetInt32("key", key));
   CHECK_OK(row.SetInt32("int_val", int_val));
   if (string_val) {
-CHECK_OK(row.SetStringCopy("string_val", string_val));
+RowAppendColumn(, {{"string_val", std::string(string_val)}});
   }
   RowOperationsPBEncoder enc(ops);
   enc.Add(op_type, row);
 }
 
+inline void AddTestRowWithNullableColumnsStringToPB(
+RowOperationsPB::Type op_type, const Schema& schema,
+int32_t key, int32_t int_val, const char* string_val,
+const std::map& columns,
+RowOperationsPB* ops) {
+  DCHECK(schema.initialized());
+  KuduPartialRow row();
+  CHECK_OK(row.SetInt32("key", key));
+  CHECK_OK(row.SetInt32("int_val", int_val));
+  if (string_val) {
+RowAppendColumn(, {{"string_val", std::string(string_val)}});
+  }
+  if (!columns.empty()) {
+RowAppendColumn(, columns);
+  }
+  RowOperationsPBEncoder enc(ops);
+  enc.Add(op_type, row);
+}
+
+
 inline void AddTestRowToPB(RowOperationsPB::Type op_type,
const Schema& schema,
int32_t key,
@@ -61,6 +89,17 @@ inline void AddTestRowToPB(RowOperationsPB::Type op_type,
   AddTestRowWithNullableStringToPB(op_type, schema, key, int_val, 
string_val.c_str(), ops);
 }
 
+inline void AddTestRowToPBAppendColumns(RowOperationsPB::Type op_type,
+const Schema& schema, int32_t key,
+int32_t int_val,
+const std::string& string_val,
+const std::map& columns,
+RowOperationsPB* ops) {
+  AddTestRowWithNullableColumnsStringToPB(op_type, schema,
+  key, int_val, string_val.c_str(),
+  columns, ops);
+}
+
 inline void AddTestKeyToPB(RowOperationsPB::Type op_type,
 const Schema& schema,
 int32_t key,
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 72898c2..2383b2f 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -2128,6 +2128,224 @@ TEST_F(ToolTest, TestWalDump) {
   }
 }
 
+TEST_F(ToolTest, TestWalDumpWithAlterSchema) {
+  const string kTestDir = GetTestPath("test");
+  const string kTestTablet = "";
+  Schema schema(GetSimpleTestSchema());
+  Schema schema_with_ids(SchemaBuilder(schema).Build());
+
+  FsManager fs(env_, FsManagerOpts(kTestDir));
+  ASSERT_OK(fs.CreateInitialFileSystemLayout());
+  ASSERT_OK(fs.Open());
+
+  const std::string kFirstMes

[kudu] branch master updated: KUDU-3344: catalog manager clean up metadata for deleted tables/tablets

2022-02-21 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 2de290d  KUDU-3344: catalog manager clean up metadata for deleted 
tables/tablets
2de290d is described below

commit 2de290d1b342696cd22c0a800aee5b1bf684a0fa
Author: Zhang Yifan 
AuthorDate: Fri Jan 7 21:02:47 2022 +0800

KUDU-3344: catalog manager clean up metadata for deleted tables/tablets

Kudu masters now retain metadata for deleted tables and tablets forever, and
the leader master loads all of them into memory when starts. If we have a 
lot
of tables and tablets in a cluster, memory usage of the leader master will 
be
large and it will take a long time to start the leader master. Consider that
in many cases users drop tables and partitions, useless metadata should be
cleaned up in backgroud tasks.

But it's hard to decide when we should clean them up, because the deletion 
of
tablet replicas is asynchronous. If metadata is deleted before the tablet 
data
is deleted, the unknown tablet reported by a tablet server will not be 
processed
by catalog manager and we must delete it manually. So this patch add a new 
flag
'deleted_table_and_tablet_reserved_secs', its default value is the same as
'unresponsive_ts_rpc_timeout_ms', we can roughly assume that after this 
amount
of time the tablet data will be actually deleted and it's safe to delete its
metadata entry from sys.catalog table and in-memory map.

Change-Id: Idefa2ee2f5108ba913fe0057a4061c3c28351547
Reviewed-on: http://gerrit.cloudera.org:8080/18152
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 src/kudu/client/client.cc  |   1 +
 src/kudu/integration-tests/master-stress-test.cc   |  26 +++-
 .../integration-tests/ts_tablet_manager-itest.cc   |   2 +-
 src/kudu/master/auto_rebalancer.cc |   2 +-
 src/kudu/master/catalog_manager.cc | 163 +++--
 src/kudu/master/catalog_manager.h  |  23 ++-
 src/kudu/master/master-test.cc |  68 -
 src/kudu/master/master.proto   |   6 +
 8 files changed, 267 insertions(+), 24 deletions(-)

diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 4b22424..2e37713 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -562,6 +562,7 @@ Status 
KuduClient::ListTabletServers(vector* tablet_servers)
 }
 
 Status KuduClient::ListTables(vector* tables, const string& filter) {
+  tables->clear();
   vector tables_info;
   RETURN_NOT_OK(data_->ListTablesWithInfo(this, _info, filter));
   for (auto& info : tables_info) {
diff --git a/src/kudu/integration-tests/master-stress-test.cc 
b/src/kudu/integration-tests/master-stress-test.cc
index 58bf519..1889ff4 100644
--- a/src/kudu/integration-tests/master-stress-test.cc
+++ b/src/kudu/integration-tests/master-stress-test.cc
@@ -22,6 +22,7 @@
 #include 
 #include 
 #include 
+#include 
 #include 
 #include 
 #include 
@@ -116,7 +117,7 @@ static const MonoDelta kTransientStateBackoff = 
MonoDelta::FromMilliseconds(50);
 
 // Parameterized based on HmsMode.
 class MasterStressTest : public ExternalMiniClusterITestBase,
- public ::testing::WithParamInterface {
+ public 
::testing::WithParamInterface> {
  public:
   MasterStressTest()
 : done_(1),
@@ -146,10 +147,19 @@ class MasterStressTest : public 
ExternalMiniClusterITestBase,
 opts.start_process_timeout = MonoDelta::FromSeconds(60);
 opts.rpc_negotiation_timeout = MonoDelta::FromSeconds(30);
 
-opts.hms_mode = GetParam();
+opts.hms_mode = std::get<0>(GetParam());
 // Tune down the notification log poll period in order to speed up catalog 
convergence.
 
opts.extra_master_flags.emplace_back("--hive_metastore_notification_log_poll_period_seconds=1");
 
+if (std::get<1>(GetParam())) {
+  // Set shorter intervals to trigger frequent cleanup tasks.
+  opts.extra_master_flags.emplace_back(
+  "--enable_metadata_cleanup_for_deleted_tables_and_tablets=true");
+  
opts.extra_master_flags.emplace_back("--catalog_manager_bg_task_wait_ms=10");
+  opts.extra_master_flags.emplace_back(
+  "--metadata_for_deleted_table_and_tablet_reserved_secs=0");
+}
+
 // Set max missed heartbeats periods to 1.0 (down from 3.0).
 
opts.extra_master_flags.emplace_back("--leader_failure_max_missed_heartbeat_periods=1.0");
 
@@ -205,7 +215,7 @@ class MasterStressTest : public 
ExternalMiniClusterITestBase,
 new MasterServiceProxy(cluster_->messenger(), addr, addr.host()));
 ASSERT_OK(CreateTabletServerMap(m_proxy, cluster_->messenger(), _m

[kudu] branch master updated: [webserver-test] unify generation of the server's URL

2022-02-18 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new 7db93eb  [webserver-test] unify generation of the server's URL
7db93eb is described below

commit 7db93eba9ad84ddc276256a4d72faa941c182344
Author: Alexey Serbin 
AuthorDate: Thu Feb 17 20:59:36 2022 -0800

[webserver-test] unify generation of the server's URL

Change-Id: Ic1e2536b2f548df9f28131368a5a8d1f46cd3053
Reviewed-on: http://gerrit.cloudera.org:8080/18254
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin 
---
 src/kudu/server/webserver-test.cc | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/kudu/server/webserver-test.cc 
b/src/kudu/server/webserver-test.cc
index 58cf31d..d52bd12 100644
--- a/src/kudu/server/webserver-test.cc
+++ b/src/kudu/server/webserver-test.cc
@@ -111,7 +111,7 @@ class WebserverTest : public KuduTest {
   ASSERT_EQ(addrs.size(), 1);
   ASSERT_TRUE(addrs[0].IsWildcard());
   ASSERT_OK(addr_.ParseString("127.0.0.1", addrs[0].port()));
-  url_ = Substitute("http://$0;, addr_.ToString());
+  url_ = Substitute(use_ssl() ? "https://$0/; : "http://$0;, 
addr_.ToString());
   // For testing purposes, we assume the server has been initialized. 
Typically this
   // is set to true after the rpc server is started in the server startup 
process.
   server_->SetStartupComplete(true);
@@ -414,7 +414,7 @@ TEST_F(SslWebserverTest, TestSSL) {
   // We use a self-signed cert, so we need to disable cert verification in 
curl.
   curl_.set_verify_peer(false);
 
-  ASSERT_OK(curl_.FetchURL(Substitute("https://$0/;, addr_.ToString()), 
_));
+  ASSERT_OK(curl_.FetchURL(url_, _));
   // Should have expected title.
   ASSERT_STR_CONTAINS(buf_.ToString(), "Kudu");
 }


[kudu] branch master updated: [util] harmonize random_util

2022-02-17 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
 new d1cfc20  [util] harmonize random_util
d1cfc20 is described below

commit d1cfc20dc4523784f11f9d60ee2ca9d2546c13db
Author: Andrew Wong 
AuthorDate: Wed Feb 16 20:30:49 2022 -0800

[util] harmonize random_util

Functions defined in int128_util.h caused ambiguous overload errors
for:
  std::ostream& operator<<(std::ostream& os, const unsigned __int128& val),

This patch includes "kudu/util/int128.h" instead of
"kudu/util/int128_util.h".

Change-Id: I247d68bedbd021aa59b8f057e0eed3166c08ac33
Reviewed-on: http://gerrit.cloudera.org:8080/18248
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 src/kudu/util/random_util-test.cc | 1 +
 src/kudu/util/random_util.h   | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)

diff --git a/src/kudu/util/random_util-test.cc 
b/src/kudu/util/random_util-test.cc
index e86b232..e01fc1a 100644
--- a/src/kudu/util/random_util-test.cc
+++ b/src/kudu/util/random_util-test.cc
@@ -29,6 +29,7 @@
 
 #include "kudu/gutil/map-util.h"
 #include "kudu/util/int128.h"
+#include "kudu/util/int128_util.h"
 #include "kudu/util/random.h"
 #include "kudu/util/test_util.h"
 
diff --git a/src/kudu/util/random_util.h b/src/kudu/util/random_util.h
index 4448023..abeef3e 100644
--- a/src/kudu/util/random_util.h
+++ b/src/kudu/util/random_util.h
@@ -27,7 +27,7 @@
 #include 
 
 #include "kudu/gutil/map-util.h"
-#include "kudu/util/int128_util.h"
+#include "kudu/util/int128.h"
 #include "kudu/util/random.h"
 
 namespace kudu {


[kudu] 02/03: [util] harmonize logging.h

2022-02-17 Thread alexey
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit b18e07a18fec1ecc868e18acbf06a19f4d8a410f
Author: Andrew Wong 
AuthorDate: Wed Feb 16 18:40:45 2022 -0800

[util] harmonize logging.h

This adds the ::kudu namespace to logging::LogThrottler, to match
changes required for Impala to use the library.

Change-Id: Ie3c6a4597cda3077df3374e907eb8672d279d9e9
Reviewed-on: http://gerrit.cloudera.org:8080/18247
Tested-by: Alexey Serbin 
Reviewed-by: Alexey Serbin 
---
 src/kudu/util/logging.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/kudu/util/logging.h b/src/kudu/util/logging.h
index f8b03b5..8e2241f 100644
--- a/src/kudu/util/logging.h
+++ b/src/kudu/util/logging.h
@@ -162,7 +162,7 @@ class ScopedDisableRedaction {
   ::LogMessage::SendToLog).stream()
 
 #define KLOG_EVERY_N_SECS(severity, n_secs) \
-  static logging::LogThrottler LOG_THROTTLER;  \
+  static ::kudu::logging::LogThrottler LOG_THROTTLER;  \
   KLOG_EVERY_N_SECS_THROTTLER(severity, n_secs, LOG_THROTTLER, "no-tag")
 
 #define WARN_NOT_OK_EVERY_N_SECS(to_call, warning_prefix, n_secs) do { 
\


<    1   2   3   4   5   6   7   8   9   10   >