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

[1/5] kudu git commit: Change to UNIXTIME_MICROS in RegexpKuduOperationsProducer

Repository: kudu
Updated Branches:
  refs/heads/master 708f38b2b -> c1d16f221


Change to UNIXTIME_MICROS in RegexpKuduOperationsProducer

Original patch was verified before the name change but
committed after, so needs a quick update.

Change-Id: Ide0ebf0d5457a6fa5a31e7fdc8c4fe7b3569dce3
Reviewed-on: http://gerrit.cloudera.org:8080/4383
Reviewed-by: David Ribeiro Alves <dr...@apache.org>
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/17f49b0f
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/17f49b0f
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/17f49b0f

Branch: refs/heads/master
Commit: 17f49b0fabda090801bfcc51978ae8076356ff32
Parents: 708f38b
Author: Will Berkeley <wd...@gmail.com>
Authored: Mon Sep 12 13:35:37 2016 -0400
Committer: David Ribeiro Alves <dr...@apache.org>
Committed: Mon Sep 12 17:57:37 2016 +0000

----------------------------------------------------------------------
 .../org/apache/kudu/flume/sink/RegexpKuduOperationsProducer.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/17f49b0f/java/kudu-flume-sink/src/main/java/org/apache/kudu/flume/sink/RegexpKuduOperationsProducer.java
----------------------------------------------------------------------
diff --git a/java/kudu-flume-sink/src/main/java/org/apache/kudu/flume/sink/RegexpKuduOperationsProducer.java b/java/kudu-flume-sink/src/main/java/org/apache/kudu/flume/sink/RegexpKuduOperationsProducer.java
index 8d5e22e..c4de305 100644
--- a/java/kudu-flume-sink/src/main/java/org/apache/kudu/flume/sink/RegexpKuduOperationsProducer.java
+++ b/java/kudu-flume-sink/src/main/java/org/apache/kudu/flume/sink/RegexpKuduOperationsProducer.java
@@ -263,7 +263,7 @@ public class RegexpKuduOperationsProducer implements KuduOperationsProducer {
       case DOUBLE:
         row.addDouble(colName, Double.parseDouble(rawVal));
         break;
-      case TIMESTAMP:
+      case UNIXTIME_MICROS:
         row.addLong(colName, Long.parseLong(rawVal));
         break;
       default:


[4/5] kudu git commit: flex_partitioning-itest: shard into separate cases

Posted by to...@apache.org.
flex_partitioning-itest: shard into separate cases

Rather than doing a loop in the main test case, shard the test into one
test case per setup.

This slightly increases the runtime of a non-parallel invocation, since
we now build and tear down a cluster for each configuration. However,
this permits parallelizing its run on dist-test. Since this is one of
the longer-running tests, it's worth it.

Change-Id: I6d352be66573fb95aac1e3ee8bc74569d4e8a0b3
Reviewed-on: http://gerrit.cloudera.org:8080/4376
Reviewed-by: Dan Burkert <da...@cloudera.com>
Tested-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 43bc07bbd7e288236f09effb9301e378532cdf43
Parents: 6ab1b56
Author: Todd Lipcon <to...@apache.org>
Authored: Sun Sep 11 17:14:18 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Mon Sep 12 18:12:22 2016 +0000

----------------------------------------------------------------------
 .../flex_partitioning-itest.cc                  | 103 +++++++++++--------
 1 file changed, 60 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/43bc07bb/src/kudu/integration-tests/flex_partitioning-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/flex_partitioning-itest.cc b/src/kudu/integration-tests/flex_partitioning-itest.cc
index 43393f7..9387505 100644
--- a/src/kudu/integration-tests/flex_partitioning-itest.cc
+++ b/src/kudu/integration-tests/flex_partitioning-itest.cc
@@ -142,7 +142,10 @@ string PartitionOptionsToString(const vector<HashPartitionOptions>& hash_partiti
   return s;
 }
 
-class FlexPartitioningITest : public KuduTest {
+typedef std::tuple<vector<HashPartitionOptions>, RangePartitionOptions> TestParamType;
+
+class FlexPartitioningITest : public KuduTest,
+                              public testing::WithParamInterface<TestParamType> {
  public:
   FlexPartitioningITest()
     : random_(GetRandomSeed32()) {
@@ -543,48 +546,62 @@ void FlexPartitioningITest::InsertAndVerifyScans(const RangePartitionOptions& ra
   }
 }
 
-TEST_F(FlexPartitioningITest, TestFlexPartitioning) {
-  vector<vector<HashPartitionOptions>> hash_options {
-    // No hash partitioning
-    {},
-    // HASH (c1) INTO 4 BUCKETS
-    { HashPartitionOptions { { "c1" }, 4 } },
-    // HASH (c0, c1) INTO 3 BUCKETS
-    { HashPartitionOptions { { "c0", "c1" }, 3 } },
-    // HASH (c1, c0) INTO 3 BUCKETS, HASH (c2) INTO 3 BUCKETS
-    { HashPartitionOptions { { "c1", "c0" }, 3 },
-      HashPartitionOptions { { "c2" }, 3 } },
-    // HASH (c2) INTO 2 BUCKETS, HASH (c1) INTO 2 BUCKETS, HASH (c0) INTO 2 BUCKETS
-    { HashPartitionOptions { { "c2" }, 2 },
-      HashPartitionOptions { { "c1" }, 2 },
-      HashPartitionOptions { { "c0" }, 2 } },
-  };
-
-  vector<RangePartitionOptions> range_options {
-    // No range partitioning
-    RangePartitionOptions { {}, {}, {} },
-    // RANGE (c0)
-    RangePartitionOptions { { "c0" }, { }, { } },
-    // RANGE (c0) SPLIT ROWS (500)
-    RangePartitionOptions { { "c0" }, { { 500 } }, { } },
-    // RANGE (c2, c1) SPLIT ROWS (500, 0), (500, 500), (1000, 0)
-    RangePartitionOptions { { "c2", "c1" }, { { 500, 0 }, { 500, 500 }, { 1000, 0 } }, { } },
-    // RANGE (c0) BOUNDS ((0), (500)), ((500), (1000))
-    RangePartitionOptions { { "c0" }, { }, { { { 0 }, { 500 } }, { { 500 }, { 1000 } } } },
-    // RANGE (c0) SPLIT ROWS (500) BOUNDS ((0), (1000))
-    RangePartitionOptions { { "c0" }, { }, { { { 0 }, { 500 } }, { { 500 }, { 1000 } } } },
-    // RANGE (c0, c1) SPLIT ROWS (500), (2001), (2500), (2999)
-    //                BOUNDS ((0), (1000)), ((2000), (3000))
-     RangePartitionOptions{ { "c0", "c1" }, { { 500 }, { 2001 }, { 2500 }, { 2999 } },
-                            { { { 0 }, { 1000 } }, { { 2000 }, { 3000 } } } },
-  };
-
-  for (const auto& hash_option : hash_options) {
-    for (const auto& range_option: range_options) {
-      SCOPED_TRACE(PartitionOptionsToString(hash_option, range_option));
-      NO_FATALS(TestPartitionOptions(hash_option, range_option));
-    }
-  }
+const vector<vector<HashPartitionOptions>> kHashOptions {
+  // No hash partitioning
+  {},
+  // HASH (c1) INTO 4 BUCKETS
+  { HashPartitionOptions { { "c1" }, 4 } },
+  // HASH (c0, c1) INTO 3 BUCKETS
+  { HashPartitionOptions { { "c0", "c1" }, 3 } },
+  // HASH (c1, c0) INTO 3 BUCKETS, HASH (c2) INTO 3 BUCKETS
+  { HashPartitionOptions { { "c1", "c0" }, 3 },
+    HashPartitionOptions { { "c2" }, 3 } },
+  // HASH (c2) INTO 2 BUCKETS, HASH (c1) INTO 2 BUCKETS, HASH (c0) INTO 2 BUCKETS
+  { HashPartitionOptions { { "c2" }, 2 },
+    HashPartitionOptions { { "c1" }, 2 },
+    HashPartitionOptions { { "c0" }, 2 } },
+};
+
+const vector<RangePartitionOptions> kRangeOptions {
+  // No range partitioning
+  RangePartitionOptions { {}, {}, {} },
+  // RANGE (c0)
+  RangePartitionOptions { { "c0" }, { }, { } },
+  // RANGE (c0) SPLIT ROWS (500)
+  RangePartitionOptions { { "c0" }, { { 500 } }, { } },
+  // RANGE (c2, c1) SPLIT ROWS (500, 0), (500, 500), (1000, 0)
+  RangePartitionOptions { { "c2", "c1" }, { { 500, 0 }, { 500, 500 }, { 1000, 0 } }, { } },
+  // RANGE (c0) BOUNDS ((0), (500)), ((500), (1000))
+  RangePartitionOptions { { "c0" }, { }, { { { 0 }, { 500 } }, { { 500 }, { 1000 } } } },
+  // RANGE (c0) SPLIT ROWS (500) BOUNDS ((0), (1000))
+  RangePartitionOptions { { "c0" }, { }, { { { 0 }, { 500 } }, { { 500 }, { 1000 } } } },
+  // RANGE (c0, c1) SPLIT ROWS (500), (2001), (2500), (2999)
+  //                BOUNDS ((0), (1000)), ((2000), (3000))
+   RangePartitionOptions{ { "c0", "c1" }, { { 500 }, { 2001 }, { 2500 }, { 2999 } },
+                          { { { 0 }, { 1000 } }, { { 2000 }, { 3000 } } } },
+};
+
+// Instantiate all combinations of hash options and range options.
+INSTANTIATE_TEST_CASE_P(Shards, FlexPartitioningITest,
+                        testing::Combine(
+                            testing::ValuesIn(kHashOptions),
+                            testing::ValuesIn(kRangeOptions)));
+
+TEST_P(FlexPartitioningITest, TestFlexPartitioning) {
+  const auto& hash_option = std::get<0>(GetParam());
+  const auto& range_option = std::get<1>(GetParam());
+  NO_FATALS(TestPartitionOptions(hash_option, range_option));
 }
 } // namespace itest
 } // namespace kudu
+
+// Define a gtest printer overload so that the test output clearly identifies the test case that
+// failed.
+namespace testing {
+template <>
+std::string PrintToString<kudu::itest::TestParamType>(const kudu::itest::TestParamType& param) {
+  const auto& hash_option = std::get<0>(param);
+  const auto& range_option = std::get<1>(param);
+  return kudu::itest::PartitionOptionsToString(hash_option, range_option);
+}
+} // namespace testing


[5/5] kudu git commit: dist_test: enable sharding on more long tests

Posted by to...@apache.org.
dist_test: enable sharding on more long tests

This enables dist-test sharding for some more of the longest-running
tests.

Change-Id: I2b860d584203c1b9b1ad472f620fe06c03eaaa21
Reviewed-on: http://gerrit.cloudera.org:8080/4377
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/c1d16f22
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/c1d16f22
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/c1d16f22

Branch: refs/heads/master
Commit: c1d16f221c021b117c7b7a2d17b0e934137a17b9
Parents: 43bc07b
Author: Todd Lipcon <to...@apache.org>
Authored: Sun Sep 11 17:43:35 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Mon Sep 12 19:41:54 2016 +0000

----------------------------------------------------------------------
 build-support/dist_test.py | 24 +++++++++++++-----------
 1 file changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/c1d16f22/build-support/dist_test.py
----------------------------------------------------------------------
diff --git a/build-support/dist_test.py b/build-support/dist_test.py
index 44be8bb..d19a78f 100755
--- a/build-support/dist_test.py
+++ b/build-support/dist_test.py
@@ -81,6 +81,18 @@ DEPS_FOR_ALL = \
      "build/latest/bin/kudu",
      ]
 
+# The number of shards to split tests into. This is set on a per-test basis
+# since it's only worth doing when a test has lots of separate cases and
+# more than one of them runs relatively long.
+NUM_SHARDS_BY_TEST = {
+  'cfile-test': 4,
+  'client-test': 8,
+  'delete_table-test': 8,
+  'flex_partitioning-itest': 8,
+  'mt-tablet-test': 4,
+  'raft_consensus-itest': 8
+}
+
 
 class StagingDir(object):
   @staticmethod
@@ -210,16 +222,6 @@ def ldd_deps(exe):
   return ret
 
 
-def num_shards_for_test(test_name):
-  if 'raft_consensus-itest' in test_name:
-    return 8
-  if 'cfile-test' in test_name:
-    return 4
-  if 'mt-tablet-test' in test_name:
-    return 4
-  return 1
-
-
 def create_archive_input(staging, argv,
                          disable_sharding=False):
   """
@@ -262,7 +264,7 @@ def create_archive_input(staging, argv,
   if disable_sharding:
     num_shards = 1
   else:
-    num_shards = num_shards_for_test(test_name)
+    num_shards = NUM_SHARDS_BY_TEST.get(test_name, 1)
   for shard in xrange(0, num_shards):
     out_archive = os.path.join(staging.dir, '%s.%d.gen.json' % (test_name, shard))
     out_isolate = os.path.join(staging.dir, '%s.%d.isolate' % (test_name, shard))


[3/5] kudu git commit: KUDU-854 - [python] Expose additional scanner projection methods

Posted by to...@apache.org.
KUDU-854 - [python] Expose additional scanner projection methods

Kudu's python client does not currently expose all Scanner projection
methods.  This patch adds the bindings for the SetProjectedColumnIndexes
and GetProjectionSchema methods of the KuduScanner class. The patch
includes a unit test to validate the column index projection method
and to ensure that the GetProjectionSchema method returns a result.

Change-Id: I8284af0841d681d1d7c704e1f57edc90c9f5ed6a
Reviewed-on: http://gerrit.cloudera.org:8080/4344
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>
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/6ab1b561
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/6ab1b561
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/6ab1b561

Branch: refs/heads/master
Commit: 6ab1b5611cced0bce4e3fd660c10595171a89a22
Parents: 64f3795
Author: Jordan Birdsell <jo...@gmail.com>
Authored: Fri Sep 9 09:49:59 2016 -0400
Committer: David Ribeiro Alves <dr...@apache.org>
Committed: Mon Sep 12 17:58:20 2016 +0000

----------------------------------------------------------------------
 python/kudu/client.pyx            | 32 ++++++++++++++++++++++++++++++++
 python/kudu/libkudu_client.pxd    |  3 +++
 python/kudu/schema.pyx            | 25 +++++++++++++++++++++++++
 python/kudu/tests/test_scanner.py | 26 ++++++++++++++++++++++++++
 4 files changed, 86 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/6ab1b561/python/kudu/client.pyx
----------------------------------------------------------------------
diff --git a/python/kudu/client.pyx b/python/kudu/client.pyx
index cc87ec7..f111da6 100644
--- a/python/kudu/client.pyx
+++ b/python/kudu/client.pyx
@@ -1145,6 +1145,22 @@ cdef class Scanner:
         check_status(self.scanner.SetProjectedColumnNames(v_names))
         return self
 
+    def set_projected_column_indexes(self, indexes):
+        """
+        Sets the columns to be scanned.
+
+        Parameters
+        ----------
+        indexes : list of integers representing column indexes
+
+        Returns
+        -------
+        self : Scanner
+        """
+        cdef vector[int] v_indexes = indexes
+        check_status(self.scanner.SetProjectedColumnIndexes(v_indexes))
+        return self
+
     def set_fault_tolerant(self):
         """
         Makes the underlying KuduScanner fault tolerant.
@@ -1192,6 +1208,22 @@ cdef class Scanner:
         check_status(self.scanner.AddExclusiveUpperBound(deref(bound.row)))
         return self
 
+    def get_projection_schema(self):
+        """
+        Returns the schema of the projection being scanned
+
+        Returns
+        -------
+        schema : kudu.Schema
+        """
+        result = Schema()
+        # Had to instantiate a new schema to return a pointer since the
+        # GetProjectionSchema method does not
+        cdef KuduSchema* schema = new KuduSchema(self.scanner.
+                                                 GetProjectionSchema())
+        result.schema = schema
+        return result
+
     def open(self):
         """
         Returns a reference to itself to facilitate chaining

http://git-wip-us.apache.org/repos/asf/kudu/blob/6ab1b561/python/kudu/libkudu_client.pxd
----------------------------------------------------------------------
diff --git a/python/kudu/libkudu_client.pxd b/python/kudu/libkudu_client.pxd
index 74ae80e..e6592bb 100644
--- a/python/kudu/libkudu_client.pxd
+++ b/python/kudu/libkudu_client.pxd
@@ -156,6 +156,7 @@ cdef extern from "kudu/client/schema.h" namespace "kudu::client" nogil:
 
     cdef cppclass KuduSchema:
         KuduSchema()
+        KuduSchema(const KuduSchema& schema)
         KuduSchema(vector[KuduColumnSchema]& columns, int key_columns)
 
         c_bool Equals(const KuduSchema& other)
@@ -613,10 +614,12 @@ cdef extern from "kudu/client/client.h" namespace "kudu::client" nogil:
         Status SetSnapshot(uint64_t snapshot_timestamp_micros)
         Status SetTimeoutMillis(int millis)
         Status SetProjectedColumnNames(const vector[string]& col_names)
+        Status SetProjectedColumnIndexes(const vector[int]& col_indexes)
         Status SetFaultTolerant()
         Status AddLowerBound(const KuduPartialRow& key)
         Status AddExclusiveUpperBound(const KuduPartialRow& key)
 
+        KuduSchema GetProjectionSchema()
         string ToString()
 
     cdef cppclass C_KuduError " kudu::client::KuduError":

http://git-wip-us.apache.org/repos/asf/kudu/blob/6ab1b561/python/kudu/schema.pyx
----------------------------------------------------------------------
diff --git a/python/kudu/schema.pyx b/python/kudu/schema.pyx
index 375d8f2..a5ef971 100644
--- a/python/kudu/schema.pyx
+++ b/python/kudu/schema.pyx
@@ -320,6 +320,31 @@ cdef class ColumnSpec:
 
 cdef class SchemaBuilder:
 
+    def copy_column(self, colschema):
+        """
+        Add a new column to the schema by copying it from an existing one.
+        Returns a ColumnSpec object for further configuration and use in a
+        fluid programming style. This method allows the SchemaBuilder to be
+        more easily used to build a new Schema from an existing one.
+
+        Parameters
+        ----------
+        colschema : ColumnSchema
+
+        Examples
+        --------
+        for col in scanner.get_projection_schema():
+            builder.copy_column(col).compression('lz4')
+        builder.set_primary_keys(['key'])
+
+        Returns
+        -------
+        spec : ColumnSpec
+        """
+        return self.add_column(colschema.name,
+                               colschema.type,
+                               colschema.nullable)
+
     def add_column(self, name, type_=None, nullable=None, compression=None,
                    encoding=None, primary_key=False):
         """

http://git-wip-us.apache.org/repos/asf/kudu/blob/6ab1b561/python/kudu/tests/test_scanner.py
----------------------------------------------------------------------
diff --git a/python/kudu/tests/test_scanner.py b/python/kudu/tests/test_scanner.py
index 0ae2036..1a6f11c 100644
--- a/python/kudu/tests/test_scanner.py
+++ b/python/kudu/tests/test_scanner.py
@@ -97,6 +97,32 @@ class TestScanner(KuduTestBase, unittest.TestCase):
 
         self.assertEqual(sorted(tuples), [(20, 'hello_20'), (22, 'hello_22')])
 
+    def test_index_projection_with_schema(self):
+        scanner = self.table.scanner()
+        scanner.set_projected_column_indexes([0, 1])
+
+        scanner.set_fault_tolerant()
+        scanner.open()
+
+        tuples = scanner.read_all_tuples()
+
+        # Build schema to check against
+        builder = kudu.schema_builder()
+        builder.add_column('key', kudu.int32, nullable=False)
+        builder.add_column('int_val', kudu.int32)
+        builder.set_primary_keys(['key'])
+        expected_schema = builder.build()
+
+        # Build new schema from projection schema
+        builder = kudu.schema_builder()
+        for col in scanner.get_projection_schema():
+            builder.copy_column(col)
+        builder.set_primary_keys(['key'])
+        new_schema = builder.build()
+
+        self.assertEqual(tuples, [t[0:2] for t in self.tuples])
+        self.assertTrue(expected_schema.equals(new_schema))
+
     def test_scan_with_bounds(self):
         scanner = self.table.scanner()
         scanner.set_fault_tolerant()


[2/5] kudu git commit: KUDU-1604 - [python] Fix bug getting table column by index

Posted by to...@apache.org.
KUDU-1604 - [python] Fix bug getting table column by index

There is currently a bug when attempting to access a table column
by its index. When attempted, the column definition returned has
the index value as the column name. This patch fixes this bug so
that the appropriate column name is set and returned. An existing
test has been updated to account for this usage.

Change-Id: If754ddb463223430c7fe3c8920e433031162562e
Reviewed-on: http://gerrit.cloudera.org:8080/4378
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/64f37959
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/64f37959
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/64f37959

Branch: refs/heads/master
Commit: 64f3795960db0ea1ffb6b8fba83156f0a2977539
Parents: 17f49b0
Author: Jordan Birdsell <jo...@gmail.com>
Authored: Mon Sep 12 00:17:06 2016 -0400
Committer: David Ribeiro Alves <dr...@apache.org>
Committed: Mon Sep 12 17:57:53 2016 +0000

----------------------------------------------------------------------
 python/kudu/client.pyx           |  6 +++---
 python/kudu/tests/test_client.py | 21 ++++++++++++---------
 2 files changed, 15 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/64f37959/python/kudu/client.pyx
----------------------------------------------------------------------
diff --git a/python/kudu/client.pyx b/python/kudu/client.pyx
index 79a0c52..cc87ec7 100644
--- a/python/kudu/client.pyx
+++ b/python/kudu/client.pyx
@@ -562,7 +562,7 @@ cdef class Table:
 
     def __getitem__(self, key):
         spec = self.schema[key]
-        return Column(self, key, spec)
+        return Column(self, spec)
 
     property name:
 
@@ -664,8 +664,8 @@ cdef class Column:
         Table parent
         ColumnSchema spec
 
-    def __cinit__(self, Table parent, object name, ColumnSchema spec):
-        self.name = tobytes(name)
+    def __cinit__(self, Table parent, ColumnSchema spec):
+        self.name = tobytes(spec.name)
         self.parent = parent
         self.spec = spec
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/64f37959/python/kudu/tests/test_client.py
----------------------------------------------------------------------
diff --git a/python/kudu/tests/test_client.py b/python/kudu/tests/test_client.py
index 8396aea..9dcb0d7 100644
--- a/python/kudu/tests/test_client.py
+++ b/python/kudu/tests/test_client.py
@@ -35,15 +35,18 @@ class TestClient(KuduTestBase, unittest.TestCase):
 
     def test_table_column(self):
         table = self.client.table(self.ex_table)
-        col = table['key']
-
-        assert col.name == b'key'
-        assert col.parent is table
-
-        result_repr = repr(col)
-        expected_repr = ('Column(key, parent={0}, type=int32)'
-                         .format(self.ex_table))
-        assert result_repr == expected_repr
+        cols = [(table['key'], 'key', 'int32'),
+                (table[1], 'int_val', 'int32'),
+                (table[-1], 'string_val', 'string')]
+
+        for col, name, type in cols:
+            assert col.name == bytes(name)
+            assert col.parent is table
+
+            result_repr = repr(col)
+            expected_repr = ('Column({0}, parent={1}, type={2})'
+                             .format(name, self.ex_table, type))
+            assert result_repr == expected_repr
 
     def test_table_schema_retains_reference(self):
         import gc