You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2016/09/26 15:59:35 UTC

[1/6] kudu git commit: [client-test] one more test for AUTO_FLUSH_BACKGROUND

Repository: kudu
Updated Branches:
  refs/heads/master c5b07fa8f -> e14bb60cc


[client-test] one more test for AUTO_FLUSH_BACKGROUND

An additional test for the AUTO_FLUSH_BACKGROUND flush mode:
verify that it's safe to perform synchronous and/or asynchronous
flush while having the auto-flusher thread running in the background.

Change-Id: I3bd5d248d4d44393689c8da81ed669395c393257
Reviewed-on: http://gerrit.cloudera.org:8080/4492
Tested-by: Kudu Jenkins
Reviewed-by: David Ribeiro Alves <dr...@apache.org>


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

Branch: refs/heads/master
Commit: 799e067b4ad9446d22a4c00796ccda909a69a91b
Parents: c5b07fa
Author: Alexey Serbin <as...@cloudera.com>
Authored: Tue Sep 20 22:03:25 2016 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Fri Sep 23 19:11:18 2016 +0000

----------------------------------------------------------------------
 src/kudu/client/client-test.cc | 22 ++++++++++++++++++++++
 1 file changed, 22 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/799e067b/src/kudu/client/client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index 8da6d88..25b141a 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -2429,6 +2429,28 @@ TEST_F(ClientTest, TestFlushModesCompareOpRatesRandomSize) {
   EXPECT_GT(t_afs.wall, t_afb.wall);
 }
 
+// A test to verify that it's safe to perform synchronous and/or asynchronous
+// flush while having the auto-flusher thread running in the background.
+TEST_F(ClientTest, TestAutoFlushBackgroundAndExplicitFlush) {
+  const size_t kIterNum = AllowSlowTests() ? 8192 : 1024;
+  shared_ptr<KuduSession> session(client_->NewSession());
+  // The background flush interval is short to have more contention.
+  ASSERT_OK(session->SetMutationBufferFlushInterval(3));
+  ASSERT_OK(session->SetFlushMode(KuduSession::AUTO_FLUSH_BACKGROUND));
+  for (size_t i = 0; i < kIterNum; i += 2) {
+    ASSERT_OK(ApplyInsertToSession(session.get(), client_table_, i, i, "x"));
+    SleepFor(MonoDelta::FromMilliseconds(1));
+    session->FlushAsync(nullptr);
+    ASSERT_OK(ApplyInsertToSession(session.get(), client_table_, i + 1, i + 1, "y"));
+    SleepFor(MonoDelta::FromMilliseconds(1));
+    ASSERT_OK(session->Flush());
+  }
+  EXPECT_EQ(0, session->CountPendingErrors());
+  EXPECT_FALSE(session->HasPendingOperations());
+  // Check that all rows have reached the table.
+  EXPECT_EQ(kIterNum, CountRowsFromClient(client_table_.get()));
+}
+
 // A test which verifies that a session in AUTO_FLUSH_BACKGROUND mode can
 // be safely abandoned: its pending data should not be flushed.
 // This test also checks that the reference to a session stored by the


[6/6] kudu git commit: [java client] Reinstate KUDU-1364's behavior, fix NPE

Posted by al...@apache.org.
[java client] Reinstate KUDU-1364's behavior, fix NPE

When d5082d8 tried to fix the client2tablets leak, it also undid the work
from KUDU-1364, while also adding new problems.

This patch brings back the caching of replica locations even when getting
TS disconnections by not purging the RemoteTablet caches on disconnection.
Instead, it is now done by the retried RPCs themselves after TabletClient
detects an uncaughtException, similarly to how it was calling
demoteAsLeaderForAllTablets before.

The NPE is fixed with a null check, it's an unfortunate race. I spent some
time trying to come up with a simple test but failed. ITClient has found
the issue before so we know we have _some_ coverage.

Change-Id: I8e0ed23fbf4c655037b77173a187c3fa11de4f63
Reviewed-on: http://gerrit.cloudera.org:8080/4501
Tested-by: Kudu Jenkins
Reviewed-by: David Ribeiro Alves <dr...@apache.org>


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

Branch: refs/heads/master
Commit: e14bb60ccd8290b0162bc701837f4d137625d912
Parents: 92f7c19
Author: Jean-Daniel Cryans <jd...@apache.org>
Authored: Wed Sep 21 17:20:37 2016 -0700
Committer: Jean-Daniel Cryans <jd...@apache.org>
Committed: Mon Sep 26 00:05:10 2016 +0000

----------------------------------------------------------------------
 .../org/apache/kudu/client/AsyncKuduClient.java | 45 ++++----------------
 .../org/apache/kudu/client/TabletClient.java    | 18 ++++++--
 2 files changed, 23 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/e14bb60c/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
----------------------------------------------------------------------
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 c478714..7bbfdb9 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
@@ -1775,12 +1775,11 @@ public class AsyncKuduClient implements AutoCloseable {
     }
 
     TabletClient old;
-    ArrayList<RemoteTablet> tablets = null;
     synchronized (ip2client) {
-      if ((old = ip2client.remove(hostport)) != null) {
-        tablets = client2tablets.remove(client);
-      }
+      old = ip2client.remove(hostport);
+      client2tablets.remove(client);
     }
+
     LOG.debug("Removed from IP cache: {" + hostport + "} -> {" + client + "}");
     if (old == null) {
       // Currently we're seeing this message when masters are disconnected and the hostport we got
@@ -1790,37 +1789,6 @@ public class AsyncKuduClient implements AutoCloseable {
       LOG.trace("When expiring " + client + " from the client cache (host:port="
           + hostport + "), it was found that there was no entry"
           + " corresponding to " + remote + ".  This shouldn't happen.");
-    } else {
-      if (tablets != null) {
-        // Make a copy so we don't need to synchronize on it while iterating.
-        RemoteTablet[] tablets_copy;
-        synchronized (tablets) {
-          tablets_copy = tablets.toArray(new RemoteTablet[tablets.size()]);
-        }
-        for (final RemoteTablet remoteTablet : tablets_copy) {
-          remoteTablet.removeTabletClient(client);
-        }
-      }
-    }
-  }
-
-  /**
-   * Call this method after encountering an error connecting to a tablet server so that we stop
-   * considering it a leader for the tablets it serves.
-   * @param client tablet server to use for demotion
-   */
-  void demoteAsLeaderForAllTablets(final TabletClient client) {
-    ArrayList<RemoteTablet> tablets = client2tablets.get(client);
-    if (tablets != null) {
-      // Make a copy so we don't need to synchronize on it while iterating.
-      RemoteTablet[] tablets_copy;
-      synchronized (tablets) {
-        tablets_copy = tablets.toArray(new RemoteTablet[tablets.size()]);
-      }
-      for (final RemoteTablet remoteTablet : tablets_copy) {
-        // It will be a no-op if it's not already a leader.
-        remoteTablet.demoteLeader(client);
-      }
     }
   }
 
@@ -2076,7 +2044,12 @@ public class AsyncKuduClient implements AutoCloseable {
       }
       TabletClient client = newClient(uuid, ip, port);
 
-      final ArrayList<RemoteTablet> tablets = client2tablets.get(client);
+      ArrayList<RemoteTablet> tablets = client2tablets.get(client);
+
+      if (tablets == null) {
+        // We lost a race, someone removed the client we received.
+        return;
+      }
 
       synchronized (tablets) {
         tabletServers.add(client);

http://git-wip-us.apache.org/repos/asf/kudu/blob/e14bb60c/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java b/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java
index 59ecf3b..d4d5d13 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java
@@ -145,6 +145,10 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
 
   private final RequestTracker requestTracker;
 
+  // If an uncaught exception forced us to shutdown this TabletClient, we'll handle the retry
+  // differently by also clearing the caches.
+  private volatile boolean gotUncaughtException = false;
+
   public TabletClient(AsyncKuduClient client, String uuid, String host, int port) {
     this.kuduClient = client;
     this.uuid = uuid;
@@ -752,7 +756,13 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
     if (tablet == null) {  // Can't retry, dunno where this RPC should go.
       rpc.errback(exception);
     } else {
-      kuduClient.handleRetryableError(rpc, exception);
+      if (gotUncaughtException) {
+        // This will remove this TabletClient from this RPC's cache since there's something wrong
+        // about it.
+        kuduClient.handleTabletNotFound(rpc, exception, this);
+      } else {
+        kuduClient.handleRetryableError(rpc, exception);
+      }
     }
   }
 
@@ -770,9 +780,9 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
       LOG.debug(getPeerUuidLoggingString() + "Encountered a read timeout, will close the channel");
     } else {
       LOG.error(getPeerUuidLoggingString() + "Unexpected exception from downstream on " + c, e);
-      // For any other exception, likely a connection error, we clear the leader state
-      // for those tablets that this TS is the cached leader of.
-      kuduClient.demoteAsLeaderForAllTablets(this);
+      // For any other exception, likely a connection error, we'll clear the tablet caches for the
+      // RPCs we're going to retry.
+      gotUncaughtException = true;
     }
     if (c.isOpen()) {
       Channels.close(c);  // Will trigger channelClosed(), which will cleanup()


[5/6] kudu git commit: Document Impala and Spark integration known issues & limitations

Posted by al...@apache.org.
Document Impala and Spark integration known issues & limitations

Change-Id: I993a09a00f5ab0049fec95e967abc1740b44dc8d
Reviewed-on: http://gerrit.cloudera.org:8080/4443
Tested-by: Dan Burkert <da...@cloudera.com>
Reviewed-by: Jean-Daniel Cryans <jd...@apache.org>


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

Branch: refs/heads/master
Commit: 92f7c1914ab29061d324a9a38aa5bb05ca598d47
Parents: 4824f64
Author: Dan Burkert <da...@cloudera.com>
Authored: Fri Sep 16 14:16:36 2016 -0700
Committer: Will Berkeley <wd...@gmail.com>
Committed: Sat Sep 24 16:14:40 2016 +0000

----------------------------------------------------------------------
 docs/developing.adoc              | 14 ++++++++++++++
 docs/kudu_impala_integration.adoc | 22 ++++++++++++++++++++++
 2 files changed, 36 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/92f7c191/docs/developing.adoc
----------------------------------------------------------------------
diff --git a/docs/developing.adoc b/docs/developing.adoc
index b4d8604..8833369 100644
--- a/docs/developing.adoc
+++ b/docs/developing.adoc
@@ -151,6 +151,20 @@ kuduContext.tableExists("another_table")
 kuduContext.deleteTable("unwanted_table")
 ----
 
+=== Spark Integration Known Issues and Limitations
+
+- The Kudu Spark integration is tested and developed against Spark 1.6 and Scala
+  2.10.
+- Kudu tables with a name containing upper case or non-ascii characters must be
+  assigned an alternate name when registered as a temporary table.
+- Kudu tables with a column name containing upper case or non-ascii characters
+  may not be used with SparkSQL. Non-primary key columns may be renamed in Kudu
+  to work around this issue.
+- `NULL`, `NOT NULL`, `<>`, `OR`, `LIKE`, and `IN` predicates are not pushed to
+  Kudu, and instead will be evaluated by the Spark task.
+- Kudu does not support all types supported by Spark SQL, such as `Date`,
+  `Decimal` and complex types.
+
 == Integration with MapReduce, YARN, and Other Frameworks
 
 Kudu was designed to integrate with MapReduce, YARN, Spark, and other frameworks in

http://git-wip-us.apache.org/repos/asf/kudu/blob/92f7c191/docs/kudu_impala_integration.adoc
----------------------------------------------------------------------
diff --git a/docs/kudu_impala_integration.adoc b/docs/kudu_impala_integration.adoc
index e2fe89c..ec86c18 100755
--- a/docs/kudu_impala_integration.adoc
+++ b/docs/kudu_impala_integration.adoc
@@ -1083,3 +1083,25 @@ The examples above have only explored a fraction of what you can do with Impala
 - View the link:http://www.cloudera.com/content/www/en-us/documentation/enterprise/latest/topics/impala_langref.html[Impala SQL reference].
 - Read about Impala internals or learn how to contribute to Impala on the link:https://github.com/cloudera/Impala/wiki[Impala Wiki].
 - Read about the native link:installation.html#view_api[Kudu APIs].
+
+=== Known Issues and Limitations
+
+- Kudu tables with a name containing upper case or non-ascii characters must be
+  assigned an alternate name when used as an external table in Impala.
+- Kudu tables with a column name containing upper case or non-ascii characters
+  may not be used as an external table in Impala. Non-primary key columns may be
+  renamed in Kudu to work around this issue.
+- When creating a Kudu table, the `CREATE TABLE` statement must include the
+  primary key columns before other columns, in primary key order.
+- Kudu tables containing `UNIXTIME_MICROS`-typed columns may not be used as an
+  external table in Impala.
+- Impala can not create Kudu tables with `TIMESTAMP` or nested-typed columns.
+- Impala can not update values in primary key columns.
+- `NULL`, `NOT NULL`, `!=`, and `IN` predicates are not pushed to Kudu, and
+  instead will be evaluated by the Impala scan node.
+- Impala can not specify column encoding or compression during Kudu table
+  creation, or alter a columns encoding or compression.
+- Impala can not create Kudu tables with bounded range partitions, and can not
+  alter a table to add or remove range partitions.
+- When bulk writing to a Kudu table, performance may be improved by setting the
+  `batch_size` option (see <<kudu_impala_insert_bulk>>).


[3/6] kudu git commit: metrics: move SCOPED_LATENCY_METRIC to metrics.h

Posted by al...@apache.org.
metrics: move SCOPED_LATENCY_METRIC to metrics.h

Addresses a simple TODO.

Change-Id: I1f1287ade44c60236d2859b5e528ce7e365645ef
Reviewed-on: http://gerrit.cloudera.org:8080/4469
Tested-by: Kudu Jenkins
Reviewed-by: David Ribeiro Alves <dr...@apache.org>


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

Branch: refs/heads/master
Commit: 8da741f215919dd15be1c63546bee9b3e5afe776
Parents: 4e3d255
Author: Todd Lipcon <to...@apache.org>
Authored: Mon Sep 19 18:23:16 2016 -0700
Committer: David Ribeiro Alves <dr...@apache.org>
Committed: Fri Sep 23 22:30:27 2016 +0000

----------------------------------------------------------------------
 src/kudu/consensus/log_metrics.h | 4 ----
 src/kudu/util/metrics.h          | 4 ++++
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/8da741f2/src/kudu/consensus/log_metrics.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log_metrics.h b/src/kudu/consensus/log_metrics.h
index f29e58c..30dfcde 100644
--- a/src/kudu/consensus/log_metrics.h
+++ b/src/kudu/consensus/log_metrics.h
@@ -43,10 +43,6 @@ struct LogMetrics {
   scoped_refptr<Histogram> entry_batches_per_group;
 };
 
-// TODO extract and generalize this for all histogram metrics
-#define SCOPED_LATENCY_METRIC(_mtx, _h) \
-  ScopedLatencyMetric _h##_metric((_mtx) ? (_mtx)->_h.get() : NULL)
-
 } // namespace log
 } // namespace kudu
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/8da741f2/src/kudu/util/metrics.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/metrics.h b/src/kudu/util/metrics.h
index 4952410..b9e739b 100644
--- a/src/kudu/util/metrics.h
+++ b/src/kudu/util/metrics.h
@@ -1013,6 +1013,10 @@ class ScopedLatencyMetric {
   MonoTime time_started_;
 };
 
+#define SCOPED_LATENCY_METRIC(_mtx, _h) \
+  ScopedLatencyMetric _h##_metric((_mtx) ? (_mtx)->_h.get() : NULL)
+
+
 ////////////////////////////////////////////////////////////
 // Inline implementations of template methods
 ////////////////////////////////////////////////////////////


[4/6] kudu git commit: [c++client] use ATTRIBUTE_DEPRECATED attribute

Posted by al...@apache.org.
[c++client] use ATTRIBUTE_DEPRECATED attribute

Added the deprecated attribute to the methods/functions marked as such
in the C++ client API.

Also, added -Wdocumentation-deprecated-sync flag to spot cases when
a method marked as deprecated by in-line doc lacks corresponding
deprecation attribute (if compiling with clang).

Change-Id: I1a48587d1132ada7da63953d7da6cae0f84a0baf
Reviewed-on: http://gerrit.cloudera.org:8080/4529
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 4824f6458ccb680209c537f5967a8a42d2a6dd76
Parents: 8da741f
Author: Alexey Serbin <as...@cloudera.com>
Authored: Fri Sep 23 14:21:19 2016 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Sat Sep 24 05:35:24 2016 +0000

----------------------------------------------------------------------
 CMakeLists.txt           |  5 +++++
 src/kudu/client/client.h | 20 +++++++++++++-------
 src/kudu/client/schema.h | 18 +++++++++++-------
 3 files changed, 29 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/4824f645/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 9a652c0..9b6c40a 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -221,6 +221,11 @@ if ("${COMPILER_FAMILY}" STREQUAL "clang")
   # Clang generates ambiguous member template warnings when calling the ev++ api.
   set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-ambiguous-member-template")
 
+  # Emit a warning when the method/function marked as deprecated
+  # in its in-line documentation but lacks the deprecated attribute
+  # ATTRIBUTE_DEPRECATED in its signature.
+  set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wdocumentation-deprecated-sync")
+
   # Only hardcode -fcolor-diagnostics if stderr is opened on a terminal. Otherwise
   # the color codes show up as noisy artifacts.
   #

http://git-wip-us.apache.org/repos/asf/kudu/blob/4824f645/src/kudu/client/client.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 5a3f7b7..b3cf5ba 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -675,7 +675,8 @@ class KUDU_EXPORT KuduTableCreator {
   /// @param [in] split_rows
   ///   The row to use for partitioning.
   /// @return Reference to the modified table creator.
-  KuduTableCreator& split_rows(const std::vector<const KuduPartialRow*>& split_rows);
+  KuduTableCreator& split_rows(const std::vector<const KuduPartialRow*>& split_rows)
+      ATTRIBUTE_DEPRECATED("use add_range_partition_split() instead");
 
   /// Set the table replication factor.
   ///
@@ -1573,7 +1574,8 @@ class KUDU_EXPORT KuduScanner {
   ///   Column names to use for the projection.
   /// @return Operation result status.
   Status SetProjectedColumns(const std::vector<std::string>& col_names)
-    WARN_UNUSED_RESULT;
+      WARN_UNUSED_RESULT
+      ATTRIBUTE_DEPRECATED("use SetProjectedColumnNames() instead");
 
   /// Add a predicate for the scan.
   ///
@@ -1602,7 +1604,8 @@ class KUDU_EXPORT KuduScanner {
   /// @param [in] key
   ///   The primary key to use as an opaque slice of data.
   /// @return Operation result status.
-  Status AddLowerBoundRaw(const Slice& key);
+  Status AddLowerBoundRaw(const Slice& key)
+      ATTRIBUTE_DEPRECATED("use AddLowerBound() instead");
 
   /// Add an upper bound (exclusive) primary key for the scan.
   ///
@@ -1621,7 +1624,8 @@ class KUDU_EXPORT KuduScanner {
   /// @param [in] key
   ///   The encoded primary key is an opaque slice of data.
   /// @return Operation result status.
-  Status AddExclusiveUpperBoundRaw(const Slice& key);
+  Status AddExclusiveUpperBoundRaw(const Slice& key)
+      ATTRIBUTE_DEPRECATED("use AddExclusiveUpperBound() instead");
 
   /// Add a lower bound (inclusive) partition key for the scan.
   ///
@@ -1702,7 +1706,8 @@ class KUDU_EXPORT KuduScanner {
   /// @param [out] rows
   ///   Placeholder for the result.
   /// @return Operation result status.
-  Status NextBatch(std::vector<KuduRowResult>* rows);
+  Status NextBatch(std::vector<KuduRowResult>* rows)
+      ATTRIBUTE_DEPRECATED("use NextBatch(KuduScanBatch*) instead");
 
   /// Fetch the next batch of results for this scanner.
   ///
@@ -1755,9 +1760,10 @@ class KUDU_EXPORT KuduScanner {
   /// @deprecated Use SetFaultTolerant() instead.
   ///
   /// @param [in] order_mode
-  ///   Result record orderind mode to set.
+  ///   Result record ordering mode to set.
   /// @return Operation result status.
-  Status SetOrderMode(OrderMode order_mode) WARN_UNUSED_RESULT;
+  Status SetOrderMode(OrderMode order_mode) WARN_UNUSED_RESULT
+      ATTRIBUTE_DEPRECATED("use SetFaultTolerant() instead");
 
   /// Make scans resumable at another tablet server if current server fails.
   ///

http://git-wip-us.apache.org/repos/asf/kudu/blob/4824f645/src/kudu/client/schema.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/schema.h b/src/kudu/client/schema.h
index a09a88d..a7bb667 100644
--- a/src/kudu/client/schema.h
+++ b/src/kudu/client/schema.h
@@ -85,12 +85,14 @@ class KUDU_EXPORT KuduColumnStorageAttributes {
   ///   Compression type for the column storage.
   /// @param [in] block_size
   ///   Block size (in bytes, uncompressed data) for the column storage.
-  KuduColumnStorageAttributes(EncodingType encoding = AUTO_ENCODING,
-                              CompressionType compression = DEFAULT_COMPRESSION,
-                              int32_t block_size = 0)
+  explicit KuduColumnStorageAttributes(
+      EncodingType encoding = AUTO_ENCODING,
+      CompressionType compression = DEFAULT_COMPRESSION,
+      int32_t block_size = 0)
+      ATTRIBUTE_DEPRECATED("this constructor will be private in a future release")
       : encoding_(encoding),
-      compression_(compression),
-      block_size_(block_size) {
+        compression_(compression),
+        block_size_(block_size) {
   }
 
   /// @return Encoding type for the column storage.
@@ -155,7 +157,8 @@ class KUDU_EXPORT KuduColumnSchema {
                    DataType type,
                    bool is_nullable = false,
                    const void* default_value = NULL,
-                   KuduColumnStorageAttributes attributes = KuduColumnStorageAttributes());
+                   KuduColumnStorageAttributes attributes = KuduColumnStorageAttributes())
+      ATTRIBUTE_DEPRECATED("use KuduSchemaBuilder instead");
 
   /// Construct KuduColumnSchema object as a copy of another object.
   ///
@@ -439,7 +442,8 @@ class KUDU_EXPORT KuduSchema {
   ///   Number of key columns in the schema.
   /// @return Operation result status.
   Status Reset(const std::vector<KuduColumnSchema>& columns, int key_columns)
-    WARN_UNUSED_RESULT;
+      ATTRIBUTE_DEPRECATED("this method will be removed in a future release")
+      WARN_UNUSED_RESULT;
 
   /// Check whether the schema is identical to the other one.
   ///


[2/6] kudu git commit: [misc] : Remove few more warnings

Posted by al...@apache.org.
[misc] : Remove few more warnings

Observed few warnings with fresh builds:
[365/1113] Building CXX object src/kudu/common/CMakeFiles/kudu_common_exported.dir/row_operations.cc.o
../../src/kudu/common/row_operations.cc: In member function \u2018std::string kudu::DecodedRowOperation::ToString(const kudu::Schema&) const\u2019:
../../src/kudu/common/row_operations.cc:58:1: warning: control reaches end of non-void function [-Wreturn-type]
 }
 ^
[403/1113] Building CXX object src/kudu/common/CMakeFiles/kudu_common.dir/row_operations.cc.o
../../src/kudu/common/row_operations.cc: In member function \u2018std::string kudu::DecodedRowOperation::ToString(const kudu::Schema&) const\u2019:
../../src/kudu/common/row_operations.cc:58:1: warning: control reaches end of non-void function [-Wreturn-type]
 }
 ^
[516/1113] Building CXX object src/kudu/benchmarks/CMakeFiles/rle.dir/rle.cc.o
../../src/kudu/benchmarks/rle.cc: In function \u2018void kudu::BooleanRLE()\u2019:
../../src/kudu/benchmarks/rle.cc:93:10: warning: variable \u2018run_length\u2019 set but not used [-Wunused-but-set-variable]
   size_t run_length;
          ^
[575/1113] Building CXX object src/kudu/cfile/CMakeFiles/cfile.dir/compression_codec.cc.o
In file included from ../../src/kudu/cfile/compression_codec.h:24:0,
                 from ../../src/kudu/cfile/compression_codec.cc:26:
../../src/kudu/gutil/macros.h:102:0: warning: "DISALLOW_COPY_AND_ASSIGN" redefined
 #define DISALLOW_COPY_AND_ASSIGN(TypeName) \
 ^
In file included from ../../thirdparty/installed-deps/include/snappy.h:45:0,
                 from ../../src/kudu/cfile/compression_codec.cc:20:
../../thirdparty/installed-deps/include/snappy-stubs-public.h:79:0: note: this is the location of the previous definition
 #define DISALLOW_COPY_AND_ASSIGN(TypeName) \
 ^

Also added a trivial comment update.

Change-Id: I89b96d52dfed6b38f17cf8cdebeed840fb32f98d
Reviewed-on: http://gerrit.cloudera.org:8080/4518
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: 4e3d25502cff273c67b4a408cb2d4716a50c6560
Parents: 799e067
Author: Dinesh Bhat <di...@cloudera.com>
Authored: Thu Sep 22 14:14:46 2016 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Fri Sep 23 20:14:12 2016 +0000

----------------------------------------------------------------------
 src/kudu/benchmarks/rle.cc        | 12 +++---------
 src/kudu/codegen/row_projector.cc |  2 +-
 src/kudu/common/row_operations.cc |  1 +
 src/kudu/gutil/macros.h           |  2 ++
 4 files changed, 7 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/4e3d2550/src/kudu/benchmarks/rle.cc
----------------------------------------------------------------------
diff --git a/src/kudu/benchmarks/rle.cc b/src/kudu/benchmarks/rle.cc
index cb2c6f2..93c514b 100644
--- a/src/kudu/benchmarks/rle.cc
+++ b/src/kudu/benchmarks/rle.cc
@@ -24,6 +24,7 @@
 #include <glog/logging.h>
 #include <gflags/gflags.h>
 
+#include "kudu/gutil/basictypes.h"
 #include "kudu/gutil/mathlimits.h"
 #include "kudu/util/bit-stream-utils.h"
 #include "kudu/util/logging.h"
@@ -90,15 +91,8 @@ void BooleanRLE() {
 
   RleDecoder<bool> decoder(buffer.data(), encoder.len(), 1);
   bool val = false;
-  size_t run_length;
-  for (int i = 0; i < num_iters; i++) {
-    run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
-    run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
-    run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
-    run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
-    run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
-    run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
-    run_length = decoder.GetNextRun(&val, MathLimits<size_t>::kMax);
+  for (int i = 0; i < num_iters * 7; i++) {
+    ignore_result(decoder.GetNextRun(&val, MathLimits<size_t>::kMax));
   }
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/4e3d2550/src/kudu/codegen/row_projector.cc
----------------------------------------------------------------------
diff --git a/src/kudu/codegen/row_projector.cc b/src/kudu/codegen/row_projector.cc
index c6076cf..fe34be6 100644
--- a/src/kudu/codegen/row_projector.cc
+++ b/src/kudu/codegen/row_projector.cc
@@ -405,7 +405,7 @@ Status RowProjector::Init() {
   //
   // This equivalence relation can be decomposed as:
   //
-  //   ProjectionsCompatible((base1, proj1), (base2, proj2)) :=
+  //   compat_check((base1, proj1), (base2, proj2)) :=
   //     WELLFORMED(base1, proj1) &&
   //     WELLFORMED(base2, proj2) &&
   //     PROJEQUALS(base1, base2) &&

http://git-wip-us.apache.org/repos/asf/kudu/blob/4e3d2550/src/kudu/common/row_operations.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/row_operations.cc b/src/kudu/common/row_operations.cc
index 17effcd..4c8adc6 100644
--- a/src/kudu/common/row_operations.cc
+++ b/src/kudu/common/row_operations.cc
@@ -55,6 +55,7 @@ string DecodedRowOperation::ToString(const Schema& schema) const {
     case RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND:
       return Substitute("INCLUSIVE_RANGE_UPPER_BOUND $0", split_row->ToString());
   }
+  return "UNKNOWN";
 }
 
 RowOperationsPBEncoder::RowOperationsPBEncoder(RowOperationsPB* pb)

http://git-wip-us.apache.org/repos/asf/kudu/blob/4e3d2550/src/kudu/gutil/macros.h
----------------------------------------------------------------------
diff --git a/src/kudu/gutil/macros.h b/src/kudu/gutil/macros.h
index a57a375..7ebb4b4 100644
--- a/src/kudu/gutil/macros.h
+++ b/src/kudu/gutil/macros.h
@@ -98,6 +98,7 @@ struct CompileAssert {
 // http://gcc.gnu.org/PR51213 in gcc-4.7 / Crosstool v16.
 // TODO(user): Remove "&& !defined(__clang_)" when =delete is
 // gcc-4.7 before =delete is allowed, go back to the C++98 definition.
+#ifndef DISALLOW_COPY_AND_ASSIGN
 #if LANG_CXX11 && !defined(__clang__)
 #define DISALLOW_COPY_AND_ASSIGN(TypeName) \
   TypeName(const TypeName&) = delete;      \
@@ -107,6 +108,7 @@ struct CompileAssert {
   TypeName(const TypeName&);               \
   void operator=(const TypeName&)
 #endif
+#endif
 
 // An older, politically incorrect name for the above.
 // Prefer DISALLOW_COPY_AND_ASSIGN for new code.