You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by jd...@apache.org on 2016/02/04 01:03:20 UTC

[1/2] incubator-kudu git commit: docs: update build-from-source instructions for blessed build layout

Repository: incubator-kudu
Updated Branches:
  refs/heads/branch-0.7.0 cf89d3286 -> 6e6ffa750


docs: update build-from-source instructions for blessed build layout

I'm not convinced that this makes sense; in my opinion, installation.adoc
isn't intended for Kudu developers, but for people running on platforms for
which we lack prebuilt binaries. JD pointed out that the Java client build
muddies this somewhat, as it's reasonable to expect these people to want to
build the Java client, and by default maven will run unit tests, which will
fail unless they're using the blessed build layout (or pass -DbinDir).

Anyway, the blessed build layout doesn't actually hurt casual
build-from-source people; it's just more complexity.

I also removed the various mentions of openssl-devel as with the dlopen()
change to squeasel it's no longer necessary for building.

Change-Id: Ic482c084397d13a0fd9e0b3e710449bc5cd866c4
Reviewed-on: http://gerrit.cloudera.org:8080/1997
Tested-by: Kudu Jenkins
Reviewed-by: Jean-Daniel Cryans
(cherry picked from commit cf1acc6bee8352898a5a164790a26dd2dcc25071)
Reviewed-on: http://gerrit.cloudera.org:8080/2017
Tested-by: Adar Dembo <ad...@cloudera.com>


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

Branch: refs/heads/branch-0.7.0
Commit: d24deb678cb8e675ffc2673d94ef8c979c118e7a
Parents: cf89d32
Author: Adar Dembo <ad...@cloudera.com>
Authored: Tue Feb 2 14:22:04 2016 -0800
Committer: Jean-Daniel Cryans <jd...@gerrit.cloudera.org>
Committed: Wed Feb 3 23:56:45 2016 +0000

----------------------------------------------------------------------
 docs/installation.adoc | 83 ++++++++++++++++++++-------------------------
 1 file changed, 37 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/d24deb67/docs/installation.adoc
----------------------------------------------------------------------
diff --git a/docs/installation.adoc b/docs/installation.adoc
index b36abff..a695bcd 100644
--- a/docs/installation.adoc
+++ b/docs/installation.adoc
@@ -212,7 +212,7 @@ on a version older than 7.0, the Red Hat Developer Toolset must be installed
 +
 ----
 $ sudo yum install gcc gcc-c++ autoconf automake libtool \
-  boost-static boost-devel openssl-devel cyrus-sasl-devel \
+  boost-static boost-devel cyrus-sasl-devel \
   cyrus-sasl-plain patch pkgconfig make rsync vim-common gdb
 ----
 
@@ -254,12 +254,12 @@ except for the `kudu` directory itself.
 +
 [source,bash]
 ----
-mkdir -p build
-cd build
-../build-support/enable_devtoolset.sh \
-  ../thirdparty/installed/bin/cmake \
+mkdir -p build/release
+cd build/release
+../../build-support/enable_devtoolset.sh \
+  ../../thirdparty/installed/bin/cmake \
   -DCMAKE_BUILD_TYPE=release \
-  ..
+  ../..
 make -j4
 ----
 
@@ -290,7 +290,7 @@ automated deployment scenario. It skips the steps marked *Optional* above.
 #!/bin/bash
 
 sudo yum -y install gcc gcc-c++ autoconf automake libtool \
-  boost-static boost-devel openssl-devel cyrus-sasl-devel \
+  boost-static boost-devel cyrus-sasl-devel \
   cyrus-sasl-plain patch pkgconfig make rsync vim-common gdb
 DTLS_RPM=rhscl-devtoolset-3-epel-6-x86_64.noarch.rpm
 DTLS_RPM_URL=https://www.softwarecollections.org/en/scls/rhscl/devtoolset-3/epel-6-x86_64/download/${DTLS_RPM}
@@ -299,12 +299,12 @@ sudo yum install -y scl-utils ${DTLS_RPM}
 sudo yum install -y devtoolset-3-toolchain
 cd kudu
 build-support/enable_devtoolset.sh thirdparty/build-if-necessary.sh
-mkdir -p build
-cd build
-../build-support/enable_devtoolset.sh \
-  ../thirdparty/installed/bin/cmake \
+mkdir -p build/release
+cd build/release
+../../build-support/enable_devtoolset.sh \
+  ../../thirdparty/installed/bin/cmake \
   -DCMAKE_BUILD_TYPE=release \
-  ..
+  ../..
 make -j4
 ----
 ====
@@ -316,7 +316,7 @@ make -j4
 +
 ----
 $ sudo apt-get install git autoconf automake libboost-thread-dev \
-  libboost-system-dev curl gcc g++ libssl-dev libsasl2-dev libsasl2-modules \
+  libboost-system-dev curl gcc g++ libsasl2-dev libsasl2-modules \
   libtool ntp patch pkg-config make rsync unzip vim-common gdb python
 ----
 
@@ -347,9 +347,9 @@ except for the `kudu` directory itself.
 +
 [source,bash]
 ----
-mkdir -p build
-cd build
-../thirdparty/installed/bin/cmake -DCMAKE_BUILD_TYPE=release ..
+mkdir -p build/release
+cd build/release
+../../thirdparty/installed/bin/cmake -DCMAKE_BUILD_TYPE=release ../..
 make -j4
 ----
 
@@ -380,16 +380,16 @@ the steps marked *Optional* above.
 #!/bin/bash
 
 sudo apt-get -y install git autoconf automake libboost-thread-dev \
-  libboost-system-dev curl gcc g++ libssl-dev libsasl2-dev libsasl2-modules \
+  libboost-system-dev curl gcc g++ libsasl2-dev libsasl2-modules \
   libtool ntp patch pkg-config make rsync unzip vim-common gdb python
 git clone https://github.com/cloudera/kudu
 cd kudu
 thirdparty/build-if-necessary.sh
-mkdir -p build
-cd build
-../thirdparty/installed/bin/cmake \
+mkdir -p build/release
+cd build/release
+../../thirdparty/installed/bin/cmake \
   -DCMAKE_BUILD_TYPE=release \
-  ..
+  ../..
 make -j4
 ----
 ====
@@ -406,7 +406,7 @@ built alongside Kudu.
 +
 ----
 $ sudo zypper install autoconf automake curl cyrus-sasl-devel gcc gcc-c++ \
-  gdb git libtool make ntp openssl-devel patch pkg-config python rsync unzip vim
+  gdb git libtool make ntp patch pkg-config python rsync unzip vim
 ----
 
 . Install Boost.
@@ -442,12 +442,12 @@ except for the `kudu` directory itself.
 +
 [source,bash]
 ----
-mkdir -p build
-cd build
+mkdir -p build/release
+cd build/release
 BOOST_ROOT=../../boost_1_59_0 \
-  ../thirdparty/installed/bin/cmake \
+  ../../thirdparty/installed/bin/cmake \
   -DCMAKE_BUILD_TYPE=release \
-  ..
+  ../..
 make -j4
 ----
 
@@ -471,7 +471,7 @@ the steps marked *Optional* above.
 #!/bin/bash
 
 sudo zypper install autoconf automake curl cyrus-sasl-devel gcc gcc-c++ \
-  gdb git libtool make ntp openssl-devel patch pkg-config python rsync unzip vim
+  gdb git libtool make ntp patch pkg-config python rsync unzip vim
 wget https://downloads.sourceforge.net/project/boost/boost/1.59.0/boost_1_59_0.tar.gz
 tar xzf boost_1_59_0.tar.gz
 pushd boost_1_59_0
@@ -481,12 +481,12 @@ popd
 git clone https://github.com/cloudera/kudu
 cd kudu
 thirdparty/build-if-necessary.sh
-mkdir -p build
-cd build
+mkdir -p build/release
+cd build/release
 BOOST_ROOT=../../boost_1_59_0 \
-  ../thirdparty/installed/bin/cmake \
+  ../../thirdparty/installed/bin/cmake \
   -DCMAKE_BUILD_TYPE=release \
-  ..
+  ../..
 make -j4
 ----
 ====
@@ -512,13 +512,6 @@ for more information.
 $ brew install autoconf automake cmake libtool pkg-config boost pstree
 ----
 
-. OS X 10.11 El Capitan only: install OpenSSL.
-+
-----
-$ brew install openssl
-$ brew link -f openssl
-----
-
 . Clone the Git repository and change to the new `kudu` directory.
 +
 [source,bash]
@@ -539,9 +532,9 @@ anywhere in your filesystem except for the `kudu` directory itself.
 +
 [source,bash]
 ----
-mkdir -p build
-cd build
-../thirdparty/installed/bin/cmake ..
+mkdir -p build/release
+cd build/release
+../../thirdparty/installed/bin/cmake -DCMAKE_BUILD_TYPE=release ../..
 make -j4
 ----
 
@@ -555,14 +548,12 @@ are installed.
 #!/bin/bash
 
 brew install autoconf automake cmake libtool pkg-config boost pstree
-brew install openssl
-brew link -f openssl
 git clone https://github.com/cloudera/kudu
 cd kudu
 thirdparty/build-if-necessary.sh
-mkdir -p build
-cd build
-../thirdparty/installed/bin/cmake ..
+mkdir -p build/release
+cd build/release
+../../thirdparty/installed/bin/cmake -DCMAKE_BUILD_TYPE=release ../..
 make -j4
 ----
 ====


[2/2] incubator-kudu git commit: KUDU-842. Implement Operation.toString() in Java client

Posted by jd...@apache.org.
KUDU-842. Implement Operation.toString() in Java client

This patch adds a new method to stringify row keys and uses it in
the new Operation.toString().

This patch also cleans up our use of booleans in row keys, which
aren't supported.

Change-Id: Ia258bc9bd4140ae0085bebe51935b465dac43db9
Reviewed-on: http://gerrit.cloudera.org:8080/1945
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@cloudera.com>
Reviewed-by: Todd Lipcon <to...@apache.org>
(cherry picked from commit f00d20c76973975cfbf7c681bc9caf849c4f22e1)
Reviewed-on: http://gerrit.cloudera.org:8080/2022
Tested-by: Jean-Daniel Cryans


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

Branch: refs/heads/branch-0.7.0
Commit: 6e6ffa750267a26264d66aa828d6d6a4b646a50d
Parents: d24deb6
Author: Jean-Daniel Cryans <jd...@cloudera.com>
Authored: Thu Jan 28 09:25:27 2016 -0800
Committer: Jean-Daniel Cryans <jd...@gerrit.cloudera.org>
Committed: Thu Feb 4 00:02:37 2016 +0000

----------------------------------------------------------------------
 .../main/java/org/kududb/client/KeyEncoder.java |  4 --
 .../main/java/org/kududb/client/Operation.java  |  9 ++-
 .../main/java/org/kududb/client/PartialRow.java | 63 ++++++++++++++++++++
 .../java/org/kududb/client/TestKeyEncoding.java | 23 -------
 .../java/org/kududb/client/TestOperation.java   | 42 +++++++++++++
 5 files changed, 113 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/6e6ffa75/java/kudu-client/src/main/java/org/kududb/client/KeyEncoder.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/KeyEncoder.java b/java/kudu-client/src/main/java/org/kududb/client/KeyEncoder.java
index 9b28696..2fbde58 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/KeyEncoder.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/KeyEncoder.java
@@ -160,10 +160,6 @@ class KeyEncoder {
    */
   private void addComponent(byte[] value, int offset, int len, Type type) {
     switch (type) {
-      case BOOL:
-        assert len == 1;
-        buf.write(value[0]);
-        break;
       case INT8:
       case INT16:
       case INT32:

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/6e6ffa75/java/kudu-client/src/main/java/org/kududb/client/Operation.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/Operation.java b/java/kudu-client/src/main/java/org/kududb/client/Operation.java
index de75a10..61b7889 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/Operation.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/Operation.java
@@ -30,7 +30,6 @@ import org.kududb.tserver.Tserver;
 import org.kududb.util.Pair;
 import org.jboss.netty.buffer.ChannelBuffer;
 
-import java.io.ByteArrayOutputStream;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.util.ArrayList;
@@ -151,6 +150,14 @@ public abstract class Operation extends KuduRpc<OperationResponse> implements Ku
     return this.row;
   }
 
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder(super.toString());
+    sb.append(" row_key=");
+    sb.append(row.stringifyRowKey());
+    return sb.toString();
+  }
+
   /**
    * Helper method that puts a list of Operations together into a WriteRequestPB.
    * @param operations The list of ops to put together in a WriteRequestPB

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/6e6ffa75/java/kudu-client/src/main/java/org/kududb/client/PartialRow.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/kududb/client/PartialRow.java b/java/kudu-client/src/main/java/org/kududb/client/PartialRow.java
index f28a068..b5f3069 100644
--- a/java/kudu-client/src/main/java/org/kududb/client/PartialRow.java
+++ b/java/kudu-client/src/main/java/org/kududb/client/PartialRow.java
@@ -21,6 +21,7 @@ import java.util.Arrays;
 import java.util.BitSet;
 import java.util.List;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import org.kududb.ColumnSchema;
 import org.kududb.Schema;
@@ -513,6 +514,68 @@ public class PartialRow {
   }
 
   /**
+   * Transforms the row key into a string representation where each column is in the format:
+   * "type col_name=value".
+   * @return a string representation of the operation's row key
+   */
+  public String stringifyRowKey() {
+    int numRowKeys = schema.getPrimaryKeyColumnCount();
+    StringBuilder sb = new StringBuilder();
+    sb.append("(");
+    for (int i = 0; i < numRowKeys; i++) {
+      if (i > 0) {
+        sb.append(", ");
+      }
+
+      ColumnSchema col = schema.getColumnByIndex(i);
+      assert !col.isNullable();
+      Preconditions.checkState(columnsBitSet.get(i),
+          "Full row key not specified, missing at least col: " + col.getName());
+      Type type = col.getType();
+      sb.append(type.getName());
+      sb.append(" ");
+      sb.append(col.getName());
+      sb.append("=");
+
+      if (type == Type.STRING || type == Type.BINARY) {
+        ByteBuffer value = getVarLengthData().get(i).duplicate();
+        value.reset(); // Make sure we start at the beginning.
+        byte[] data = new byte[value.limit()];
+        value.get(data);
+        if (type == Type.STRING) {
+          sb.append(Bytes.getString(data));
+        } else {
+          sb.append(Bytes.pretty(data));
+        }
+      } else {
+        switch (type) {
+          case INT8:
+            sb.append(Bytes.getByte(rowAlloc, schema.getColumnOffset(i)));
+            break;
+          case INT16:
+            sb.append(Bytes.getShort(rowAlloc, schema.getColumnOffset(i)));
+            break;
+          case INT32:
+            sb.append(Bytes.getInt(rowAlloc, schema.getColumnOffset(i)));
+            break;
+          case INT64:
+            sb.append(Bytes.getLong(rowAlloc, schema.getColumnOffset(i)));
+            break;
+          case TIMESTAMP:
+            sb.append(Bytes.getLong(rowAlloc, schema.getColumnOffset(i)));
+            break;
+          default:
+            throw new IllegalArgumentException(String.format(
+                "The column type %s is not a valid key component type", type));
+        }
+      }
+    }
+    sb.append(")");
+
+    return sb.toString();
+  }
+
+  /**
    * Get the schema used for this row.
    * @return a schema that came from KuduTable
    */

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/6e6ffa75/java/kudu-client/src/test/java/org/kududb/client/TestKeyEncoding.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/kududb/client/TestKeyEncoding.java b/java/kudu-client/src/test/java/org/kududb/client/TestKeyEncoding.java
index c94852a..e446445 100644
--- a/java/kudu-client/src/test/java/org/kududb/client/TestKeyEncoding.java
+++ b/java/kudu-client/src/test/java/org/kududb/client/TestKeyEncoding.java
@@ -78,22 +78,6 @@ public class TestKeyEncoding {
   }
 
   @Test
-  public void testBoolPrimaryKeys() {
-    Schema schema = buildSchema(new ColumnSchema.ColumnSchemaBuilder("key", Type.BOOL).key(true));
-
-    KuduTable table = new KuduTable(null, "one", "one", schema, defaultPartitionSchema(schema));
-    Insert oneKeyInsert = new Insert(table);
-    PartialRow row = oneKeyInsert.getRow();
-    row.addBoolean("key", true);
-    assertBytesEquals(row.encodePrimaryKey(), "\1");
-
-    oneKeyInsert = new Insert(table);
-    row = oneKeyInsert.getRow();
-    row.addBoolean("key", false);
-    assertBytesEquals(row.encodePrimaryKey(), "\0");
-  }
-
-  @Test
   public void testPrimaryKeys() {
     Schema schemaOneString =
         buildSchema(new ColumnSchema.ColumnSchemaBuilder("key", Type.STRING).key(true));
@@ -160,7 +144,6 @@ public class TestKeyEncoding {
   @Test
   public void testPrimaryKeyEncoding() {
     Schema schema = buildSchema(
-        new ColumnSchemaBuilder("bool", Type.BOOL).key(true),
         new ColumnSchemaBuilder("int8", Type.INT8).key(true),
         new ColumnSchemaBuilder("int16", Type.INT16).key(true),
         new ColumnSchemaBuilder("int32", Type.INT32).key(true),
@@ -169,7 +152,6 @@ public class TestKeyEncoding {
         new ColumnSchemaBuilder("binary", Type.BINARY).key(true));
 
     PartialRow rowA = schema.newPartialRow();
-    rowA.addBoolean("bool", false);
     rowA.addByte("int8", Byte.MIN_VALUE);
     rowA.addShort("int16", Short.MIN_VALUE);
     rowA.addInt("int32", Integer.MIN_VALUE);
@@ -179,7 +161,6 @@ public class TestKeyEncoding {
 
     assertBytesEquals(rowA.encodePrimaryKey(),
                       "\0"
-                    + "\0"
                     + "\0\0"
                     + "\0\0\0\0"
                     + "\0\0\0\0\0\0\0\0"
@@ -187,7 +168,6 @@ public class TestKeyEncoding {
                     + "");
 
     PartialRow rowB = schema.newPartialRow();
-    rowB.addBoolean("bool", true);
     rowB.addByte("int8", Byte.MAX_VALUE);
     rowB.addShort("int16", Short.MAX_VALUE);
     rowB.addInt("int32", Integer.MAX_VALUE);
@@ -197,7 +177,6 @@ public class TestKeyEncoding {
 
     assertBytesEquals(rowB.encodePrimaryKey(),
                       new byte[] {
-                          1,
                           -1,
                           -1, -1,
                           -1, -1, -1, -1,
@@ -207,7 +186,6 @@ public class TestKeyEncoding {
                       });
 
     PartialRow rowC = schema.newPartialRow();
-    rowC.addBoolean("bool", false);
     rowC.addByte("int8", (byte) 1);
     rowC.addShort("int16", (short) 2);
     rowC.addInt("int32", 3);
@@ -217,7 +195,6 @@ public class TestKeyEncoding {
 
     assertBytesEquals(rowC.encodePrimaryKey(),
                       new byte[] {
-                          0,
                           (byte) 0x81,
                           (byte) 0x80, 2,
                           (byte) 0x80, 0, 0, 3,

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/6e6ffa75/java/kudu-client/src/test/java/org/kududb/client/TestOperation.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/kududb/client/TestOperation.java b/java/kudu-client/src/test/java/org/kududb/client/TestOperation.java
index dd39f8b..b84dc36 100644
--- a/java/kudu-client/src/test/java/org/kududb/client/TestOperation.java
+++ b/java/kudu-client/src/test/java/org/kududb/client/TestOperation.java
@@ -17,6 +17,7 @@
 package org.kududb.client;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
 
@@ -118,4 +119,45 @@ public class TestOperation {
     }
   }
 
+  private Schema createAllTypesKeySchema() {
+    ArrayList<ColumnSchema> columns = new ArrayList<ColumnSchema>(7);
+    columns.add(new ColumnSchema.ColumnSchemaBuilder("c0", Type.INT8).key(true).build());
+    columns.add(new ColumnSchema.ColumnSchemaBuilder("c1", Type.INT16).key(true).build());
+    columns.add(new ColumnSchema.ColumnSchemaBuilder("c2", Type.INT32).key(true).build());
+    columns.add(new ColumnSchema.ColumnSchemaBuilder("c3", Type.INT64).key(true).build());
+    columns.add(new ColumnSchema.ColumnSchemaBuilder("c4", Type.TIMESTAMP).key(true).build());
+    columns.add(new ColumnSchema.ColumnSchemaBuilder("c5", Type.STRING).key(true).build());
+    columns.add(new ColumnSchema.ColumnSchemaBuilder("c6", Type.BINARY).key(true).build());
+    return new Schema(columns);
+  }
+
+  @Test
+  public void testRowKeyStringify() {
+    KuduTable table = Mockito.mock(KuduTable.class);
+    Mockito.doReturn(createAllTypesKeySchema()).when(table).getSchema();
+    Insert insert = new Insert(table);
+    PartialRow row = insert.getRow();
+    row.addByte("c0", (byte) 1);
+    row.addShort("c1", (short) 2);
+    row.addInt("c2", 3);
+    row.addLong("c3", 4);
+    row.addLong("c4", 5);
+    row.addString("c5", "c5_val");
+    row.addBinary("c6", Bytes.fromString("c6_val"));
+
+    assertEquals("(int8 c0=1, int16 c1=2, int32 c2=3, int64 c3=4, timestamp c4=5, string" +
+            " c5=c5_val, binary c6=\"c6_val\")",
+        insert.getRow().stringifyRowKey());
+
+    // Test an incomplete row key.
+    insert = new Insert(table);
+    row = insert.getRow();
+    row.addByte("c0", (byte) 1);
+    try {
+      row.stringifyRowKey();
+      fail("Should not be able to stringifyRowKey when not all keys are specified");
+    } catch (IllegalStateException ise) {
+      // Expected.
+    }
+  }
 }