You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ab...@apache.org on 2019/12/13 23:03:35 UTC

[kudu] branch master updated (4327b6e -> 8c069b3)

This is an automated email from the ASF dual-hosted git repository.

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


    from 4327b6e  KUDU-1938 Fix length check in SchemaBuilder
     new 5588a72  [util] a small clean-up on MAYBE_RETURN_FAILURE
     new 8c069b3  [python] KUDU-1938 Add VARCHAR support

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:
 python/kudu/__init__.py             |  2 +-
 python/kudu/client.pyx              | 18 +++++++++++++-
 python/kudu/libkudu_client.pxd      | 13 ++++++++++
 python/kudu/schema.pyx              | 44 +++++++++++++++++++++++++++++-----
 python/kudu/tests/test_scanner.py   |  9 +++++--
 python/kudu/tests/test_scantoken.py |  4 ++++
 python/kudu/tests/test_schema.py    | 48 +++++++++++++++++++++++++++++++++++++
 python/kudu/tests/util.py           | 17 +++++++++----
 src/kudu/util/env_posix.cc          |  2 +-
 src/kudu/util/fault_injection.h     | 15 ++++++------
 10 files changed, 149 insertions(+), 23 deletions(-)


[kudu] 02/02: [python] KUDU-1938 Add VARCHAR support

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 8c069b3f9d9ac6cb664e0f30b879461d6b40efee
Author: Attila Bukor <ab...@apache.org>
AuthorDate: Wed Dec 11 17:06:57 2019 -0800

    [python] KUDU-1938 Add VARCHAR support
    
    Change-Id: I27b2a85c9caa3f026f4fb0ab8974899babb01dac
    Reviewed-on: http://gerrit.cloudera.org:8080/14879
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 python/kudu/__init__.py             |  2 +-
 python/kudu/client.pyx              | 18 +++++++++++++-
 python/kudu/libkudu_client.pxd      | 13 ++++++++++
 python/kudu/schema.pyx              | 44 +++++++++++++++++++++++++++++-----
 python/kudu/tests/test_scanner.py   |  9 +++++--
 python/kudu/tests/test_scantoken.py |  4 ++++
 python/kudu/tests/test_schema.py    | 48 +++++++++++++++++++++++++++++++++++++
 python/kudu/tests/util.py           | 17 +++++++++----
 8 files changed, 141 insertions(+), 14 deletions(-)

diff --git a/python/kudu/__init__.py b/python/kudu/__init__.py
index 1146f05..989eea1 100644
--- a/python/kudu/__init__.py
+++ b/python/kudu/__init__.py
@@ -39,7 +39,7 @@ from kudu.errors import (KuduException, KuduBadStatus, KuduNotFound,  # noqa
 
 from kudu.schema import (int8, int16, int32, int64, string_ as string,  # noqa
                          double_ as double, float_, float_ as float, binary,
-                         unixtime_micros, bool_ as bool, decimal,
+                         unixtime_micros, bool_ as bool, decimal, varchar,
                          KuduType,
                          SchemaBuilder, ColumnSpec, Schema, ColumnSchema,
                          COMPRESSION_DEFAULT,
diff --git a/python/kudu/client.pyx b/python/kudu/client.pyx
index a37bd93..368d170 100644
--- a/python/kudu/client.pyx
+++ b/python/kudu/client.pyx
@@ -85,7 +85,8 @@ cdef dict _type_names = {
     KUDU_DOUBLE : "KUDU_DOUBLE",
     KUDU_BINARY : "KUDU_BINARY",
     KUDU_UNIXTIME_MICROS : "KUDU_UNIXTIME_MICROS",
-    KUDU_DECIMAL : "KUDU_DECIMAL"
+    KUDU_DECIMAL : "KUDU_DECIMAL",
+    KUDU_VARCHAR : "KUDU_VARCHAR"
 }
 
 # Range Partition Bound Type enums
@@ -1545,6 +1546,12 @@ cdef class Row:
         scale = self.parent.batch.projection_schema().Column(i).type_attributes().scale()
         return from_unscaled_decimal(self.__get_unscaled_decimal(i), scale)
 
+    cdef inline get_varchar(self, int i):
+        cdef Slice val
+        check_status(self.row.GetVarchar(i, &val))
+        return cpython.PyBytes_FromStringAndSize(<char*> val.mutable_data(),
+                                                 val.size())
+
     cdef inline get_slot(self, int i):
         cdef:
             Status s
@@ -1572,6 +1579,8 @@ cdef class Row:
             return from_unixtime_micros(self.get_unixtime_micros(i))
         elif t == KUDU_DECIMAL:
             return self.get_decimal(i)
+        elif t == KUDU_VARCHAR:
+            return frombytes(self.get_varchar(i))
         else:
             raise TypeError("Cannot get kudu type <{0}>"
                                 .format(_type_names[t]))
@@ -2770,6 +2779,11 @@ cdef class PartialRow:
 
             slc = Slice(<char*> value, len(value))
             check_status(self.row.SetBinaryCopy(i, slc))
+        elif t == KUDU_VARCHAR:
+            if isinstance(value, unicode):
+                value = value.encode('utf8')
+            slc = Slice(<char*> value, len(value))
+            check_status(self.row.SetVarchar(i, slc))
         elif t == KUDU_UNIXTIME_MICROS:
             check_status(self.row.SetUnixTimeMicros(i, <int64_t>
                 to_unixtime_micros(value)))
@@ -2884,6 +2898,8 @@ cdef inline cast_pyvalue(DataType t, object o):
         return UnixtimeMicrosVal(o)
     elif t == KUDU_BINARY:
         return StringVal(o)
+    elif t == KUDU_VARCHAR:
+        return StringVal(o)
     else:
         raise TypeError("Cannot cast kudu type <{0}>".format(_type_names[t]))
 
diff --git a/python/kudu/libkudu_client.pxd b/python/kudu/libkudu_client.pxd
index a34d6c2..2c65a26 100644
--- a/python/kudu/libkudu_client.pxd
+++ b/python/kudu/libkudu_client.pxd
@@ -125,6 +125,7 @@ cdef extern from "kudu/client/schema.h" namespace "kudu::client" nogil:
         KUDU_BINARY " kudu::client::KuduColumnSchema::BINARY"
         KUDU_UNIXTIME_MICROS " kudu::client::KuduColumnSchema::UNIXTIME_MICROS"
         KUDU_DECIMAL " kudu::client::KuduColumnSchema::DECIMAL"
+        KUDU_VARCHAR " kudu::client::KuduColumnSchema::VARCHAR"
 
     enum EncodingType" kudu::client::KuduColumnStorageAttributes::EncodingType":
         EncodingType_AUTO " kudu::client::KuduColumnStorageAttributes::AUTO_ENCODING"
@@ -152,9 +153,11 @@ cdef extern from "kudu/client/schema.h" namespace "kudu::client" nogil:
         KuduColumnTypeAttributes()
         KuduColumnTypeAttributes(const KuduColumnTypeAttributes& other)
         KuduColumnTypeAttributes(int8_t precision, int8_t scale)
+        KuduColumnTypeAttributes(uint16_t length)
 
         int8_t precision()
         int8_t scale()
+        uint16_t length()
 
         c_bool Equals(KuduColumnTypeAttributes& other)
         void CopyFrom(KuduColumnTypeAttributes& other)
@@ -203,6 +206,7 @@ cdef extern from "kudu/client/schema.h" namespace "kudu::client" nogil:
 
          KuduColumnSpec* Precision(int8_t precision);
          KuduColumnSpec* Scale(int8_t scale);
+         KuduColumnSpec* Length(uint16_t length);
 
          KuduColumnSpec* RenameTo(const string& new_name)
 
@@ -264,6 +268,9 @@ cdef extern from "kudu/client/scan_batch.h" namespace "kudu::client" nogil:
         Status GetBinary(const Slice& col_name, Slice* val)
         Status GetBinary(int col_idx, Slice* val)
 
+        Status GetVarchar(const Slice& col_name, Slice* val)
+        Status GetVarchar(int col_idx, Slice* val)
+
         const void* cell(int col_idx)
         string ToString()
 
@@ -359,6 +366,9 @@ cdef extern from "kudu/common/partial_row.h" namespace "kudu" nogil:
         Status SetBinaryCopy(const Slice& col_name, const Slice& val)
         Status SetBinaryCopy(int col_idx, const Slice& val)
 
+        Status SetVarchar(Slice& col_name, Slice& val)
+        Status SetVarchar(int col_idx, Slice& val)
+
         Status SetNull(Slice& col_name)
         Status SetNull(int col_idx)
 
@@ -410,6 +420,9 @@ cdef extern from "kudu/common/partial_row.h" namespace "kudu" nogil:
         Status GetBinary(const Slice& col_name, Slice* val)
         Status GetBinary(int col_idx, Slice* val)
 
+        Status GetVarchar(Slice& col_name, Slice* val)
+        Status GetVarchar(int col_idx, Slice* val)
+
         Status EncodeRowKey(string* encoded_key)
         string ToEncodedRowKeyOrDie()
 
diff --git a/python/kudu/schema.pyx b/python/kudu/schema.pyx
index 8a3f9ee..35f7192 100644
--- a/python/kudu/schema.pyx
+++ b/python/kudu/schema.pyx
@@ -35,6 +35,7 @@ from . import util
 
 BOOL = KUDU_BOOL
 STRING = KUDU_STRING
+VARCHAR = KUDU_VARCHAR
 
 INT8 = KUDU_INT8
 INT16 = KUDU_INT16
@@ -123,6 +124,7 @@ double_ = KuduType(KUDU_DOUBLE)
 binary = KuduType(KUDU_BINARY)
 unixtime_micros = KuduType(KUDU_UNIXTIME_MICROS)
 decimal = KuduType(KUDU_DECIMAL)
+varchar = KuduType(KUDU_VARCHAR)
 
 
 cdef dict _type_names = {
@@ -136,7 +138,8 @@ cdef dict _type_names = {
     DOUBLE: 'double',
     BINARY: 'binary',
     UNIXTIME_MICROS: 'unixtime_micros',
-    DECIMAL: 'decimal'
+    DECIMAL: 'decimal',
+    VARCHAR: 'varchar'
 }
 
 
@@ -153,7 +156,8 @@ cdef dict _type_to_obj = {
     DOUBLE: double_,
     BINARY: binary,
     UNIXTIME_MICROS: unixtime_micros,
-    DECIMAL: decimal
+    DECIMAL: decimal,
+    VARCHAR: varchar
 }
 
 
@@ -171,9 +175,11 @@ cdef cppclass KuduColumnTypeAttributes:
         KuduColumnTypeAttributes()
         KuduColumnTypeAttributes(const KuduColumnTypeAttributes& other)
         KuduColumnTypeAttributes(int8_t precision, int8_t scale)
+        KuduColumnTypeAttributes(uint16_t length)
 
         int8_t precision()
         int8_t scale()
+        uint16_t length()
 
         c_bool Equals(KuduColumnTypeAttributes& other)
         void CopyFrom(KuduColumnTypeAttributes& other)
@@ -197,10 +203,15 @@ cdef class ColumnTypeAttributes:
         def __get__(self):
             return self.type_attributes.scale()
 
+    property length:
+        def __get__(self):
+            return self.type_attributes.length()
+
     def __repr__(self):
-        return ('ColumnTypeAttributes(precision=%s, scale=%s)'
+        return ('ColumnTypeAttributes(precision=%s, scale=%s, length=%s)'
                 % (self.type_attributes.precision(),
-                   self.type_attributes.scale()))
+                   self.type_attributes.scale(),
+                   self.type_attributes.length()))
 
 cdef class ColumnSchema:
     """
@@ -382,6 +393,22 @@ cdef class ColumnSpec:
         self.spec.Scale(scale)
         return self
 
+    def length(self, length):
+        """
+        Set the length for the column.
+
+        Clients can specify a length for varchar columns. Length is the maximum
+        length in characters (UTF-8) of the string that the varchar can hold.
+
+        The length must be between 1 and 65,535 (inclusive).
+
+        Returns
+        -------
+        self
+        """
+        self.spec.Length(length)
+        return self
+
     def primary_key(self):
         """
         Make this column a primary key. If you use this method, it will be the
@@ -473,7 +500,7 @@ cdef class SchemaBuilder:
 
     def add_column(self, name, type_=None, nullable=None, compression=None,
                    encoding=None, primary_key=False, block_size=None,
-                   default=None, precision=None, scale=None):
+                   default=None, precision=None, scale=None, length=None):
         """
         Add a new column to the schema. Returns a ColumnSpec object for further
         configuration and use in a fluid programming style.
@@ -502,6 +529,8 @@ cdef class SchemaBuilder:
           Use this precision for the decimal column
         scale : int
           Use this scale for the decimal column
+        length : int
+          Use this length for the varchar column
 
         Examples
         --------
@@ -537,6 +566,9 @@ cdef class SchemaBuilder:
         if scale is not None:
             result.scale(scale)
 
+        if length is not None:
+            result.length(length)
+
         if primary_key:
             result.primary_key()
 
@@ -749,7 +781,7 @@ cdef class KuduValue:
 
         if (type_.name[:3] == 'int'):
             self._value = C_KuduValue.FromInt(value)
-        elif (type_.name in ['string', 'binary']):
+        elif (type_.name in ['string', 'binary', 'varchar']):
             if isinstance(value, unicode):
                 value = value.encode('utf8')
 
diff --git a/python/kudu/tests/test_scanner.py b/python/kudu/tests/test_scanner.py
index 4bedf96..b7e21b6 100644
--- a/python/kudu/tests/test_scanner.py
+++ b/python/kudu/tests/test_scanner.py
@@ -312,6 +312,9 @@ class TestScanner(TestScanBase):
         # Test a binary predicate
         self._test_binary_pred()
 
+    def test_varchar_pred(self):
+        self._test_varchar_pred()
+
     def test_scan_selection(self):
         """
         This test confirms that setting the scan selection policy on the
@@ -348,7 +351,8 @@ class TestScanner(TestScanBase):
             self.assertEqual(types[5], np.float64)
             self.assertEqual(types[6], np.int8)
             self.assertEqual(types[7], np.object)
-            self.assertEqual(types[8], np.float32)
+            self.assertEqual(types[8], np.object)
+            self.assertEqual(types[9], np.float32)
         else:
             self.assertEqual(types[0], np.int64)
             self.assertEqual(types[1], 'datetime64[ns, UTC]')
@@ -357,7 +361,8 @@ class TestScanner(TestScanBase):
             self.assertEqual(types[4], np.float64)
             self.assertEqual(types[5], np.int8)
             self.assertEqual(types[6], np.object)
-            self.assertEqual(types[7], np.float32)
+            self.assertEqual(types[7], np.object)
+            self.assertEqual(types[8], np.float32)
 
     @pytest.mark.skipif(not (kudu.CLIENT_SUPPORTS_PANDAS),
                         reason="Pandas required to run this test.")
diff --git a/python/kudu/tests/test_scantoken.py b/python/kudu/tests/test_scantoken.py
index 37b273d..cfea352 100644
--- a/python/kudu/tests/test_scantoken.py
+++ b/python/kudu/tests/test_scantoken.py
@@ -268,6 +268,10 @@ class TestScanToken(TestScanBase):
         # Test a binary predicate
         self._test_binary_pred()
 
+    def test_varchar_pred(self):
+        # Test a varchar predicate
+        self._test_varchar_pred()
+
     def test_scan_selection(self):
         """
         This test confirms that setting the scan selection policy on the
diff --git a/python/kudu/tests/test_schema.py b/python/kudu/tests/test_schema.py
index 4870ab7..1b81bbb 100644
--- a/python/kudu/tests/test_schema.py
+++ b/python/kudu/tests/test_schema.py
@@ -182,6 +182,54 @@ class TestSchema(unittest.TestCase):
         with self.assertRaises(kudu.KuduInvalidArgument):
             builder.build()
 
+    def test_varchar(self):
+        builder = kudu.schema_builder()
+        (builder.add_column('key')
+         .type('varchar')
+         .primary_key()
+         .nullable(False)
+         .length(10))
+        schema = builder.build()
+
+        column = schema[0]
+        tp = column.type
+        assert tp.name == 'varchar'
+        assert tp.type == kudu.schema.VARCHAR
+        ta = column.type_attributes
+        assert ta.length == 10
+
+    def test_varchar_without_length(self):
+        builder = kudu.schema_builder()
+        (builder.add_column('key')
+         .type('varchar')
+         .primary_key()
+         .nullable(False))
+
+        with self.assertRaises(kudu.KuduInvalidArgument):
+            builder.build()
+
+    def test_varchar_invalid_length(self):
+        builder = kudu.schema_builder()
+        (builder.add_column('key')
+         .type('varchar')
+         .primary_key()
+         .length(0)
+         .nullable(False))
+
+        with self.assertRaises(kudu.KuduInvalidArgument):
+            builder.build()
+
+    def test_length_on_non_varchar_column(self):
+        builder = kudu.schema_builder()
+        (builder.add_column('key')
+         .type('decimal')
+         .primary_key()
+         .nullable(False)
+         .length(10))
+
+        with self.assertRaises(kudu.KuduInvalidArgument):
+            builder.build()
+
     def test_unsupported_col_spec_methods_for_create_table(self):
         builder = kudu.schema_builder()
         builder.add_column('test', 'int64').rename('test')
diff --git a/python/kudu/tests/util.py b/python/kudu/tests/util.py
index 6823b3a..62f94d2 100644
--- a/python/kudu/tests/util.py
+++ b/python/kudu/tests/util.py
@@ -72,6 +72,7 @@ class TestScanBase(KuduTestBase, unittest.TestCase):
         builder.add_column('double_val', type_=kudu.double)
         builder.add_column('int8_val', type_=kudu.int8)
         builder.add_column('binary_val', type_='binary', compression=kudu.COMPRESSION_SNAPPY, encoding='prefix')
+        builder.add_column('varchar_val', type_=kudu.varchar, length=10)
         builder.add_column('float_val', type_=kudu.float)
         builder.set_primary_keys(['key', 'unixtime_micros_val'])
         schema = builder.build()
@@ -103,22 +104,22 @@ class TestScanBase(KuduTestBase, unittest.TestCase):
                 (1, datetime.datetime(2016, 1, 1).replace(tzinfo=pytz.utc), Decimal('111.11'),
                  "Test One", True, 1.7976931348623157 * (10^308), 127,
                  b'\xce\x99\xce\xbf\xcf\x81\xce\xb4\xce\xb1\xce\xbd\xce\xaf\xce\xb1',
-                 3.402823 * (10^38)),
+                 "Test One", 3.402823 * (10^38)),
                 (2, datetime.datetime.utcnow().replace(tzinfo=pytz.utc), Decimal('0.99'),
                  "测试二", False, 200.1, -1,
                  b'\xd0\x98\xd0\xbe\xd1\x80\xd0\xb4\xd0\xb0\xd0\xbd\xd0\xb8\xd1\x8f',
-                 -150.2)
+                 "测试二", -150.2)
             ]
         else:
             self.type_test_rows = [
                 (1, datetime.datetime(2016, 1, 1).replace(tzinfo=pytz.utc),
                  "Test One", True, 1.7976931348623157 * (10 ^ 308), 127,
                  b'\xce\x99\xce\xbf\xcf\x81\xce\xb4\xce\xb1\xce\xbd\xce\xaf\xce\xb1',
-                 3.402823 * (10 ^ 38)),
+                 "Test One", 3.402823 * (10 ^ 38)),
                 (2, datetime.datetime.utcnow().replace(tzinfo=pytz.utc),
                  "测试二", False, 200.1, -1,
                  b'\xd0\x98\xd0\xbe\xd1\x80\xd0\xb4\xd0\xb0\xd0\xbd\xd0\xb8\xd1\x8f',
-                 -150.2)
+                 "测试二", -150.2)
             ]
         session = self.client.new_session()
         for row in self.type_test_rows:
@@ -255,3 +256,11 @@ class TestScanBase(KuduTestBase, unittest.TestCase):
             ],
             row_indexes=slice(1, 2)
         )
+
+    def _test_varchar_pred(self):
+        self.verify_pred_type_scans(
+            preds=[
+                self.type_table['varchar_val'] == 'Test One'
+            ],
+            row_indexes=slice(0, 1)
+        )


[kudu] 01/02: [util] a small clean-up on MAYBE_RETURN_FAILURE

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 5588a729c9769fad3a0ff6b1dfa4b8bb5c5505e0
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Thu Dec 12 21:46:31 2019 -0800

    [util] a small clean-up on MAYBE_RETURN_FAILURE
    
    Updated MAYBE_RETURN_FAILURE macro to follow the 'do {...} while (0)'
    pattern so static code analyzers doesn't output warnings on an
    empty statements when encountering constructs like
    MAYBE_RETURN_FAILURE(...);
    
    This patch doesn't contain any functional modifications.
    
    Change-Id: Ieb9e172b4f2e1cfa712be08b9527f50bd0fe09e5
    Reviewed-on: http://gerrit.cloudera.org:8080/14899
    Tested-by: Alexey Serbin <as...@cloudera.com>
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/util/env_posix.cc      |  2 +-
 src/kudu/util/fault_injection.h | 15 +++++++--------
 2 files changed, 8 insertions(+), 9 deletions(-)

diff --git a/src/kudu/util/env_posix.cc b/src/kudu/util/env_posix.cc
index cfe4b03..df29a32 100644
--- a/src/kudu/util/env_posix.cc
+++ b/src/kudu/util/env_posix.cc
@@ -125,7 +125,7 @@ typedef struct xfs_flock64 {
 #define MAYBE_RETURN_EIO(filename_expr, error_expr) do { \
   const string& f_ = (filename_expr); \
   MAYBE_RETURN_FAILURE(FLAGS_env_inject_eio, \
-      ShouldInject(f_, FLAGS_env_inject_eio_globs) ? (error_expr) : Status::OK()) \
+      ShouldInject(f_, FLAGS_env_inject_eio_globs) ? (error_expr) : Status::OK()); \
 } while (0)
 
 bool ShouldInject(const string& candidate, const string& glob_patterns) {
diff --git a/src/kudu/util/fault_injection.h b/src/kudu/util/fault_injection.h
index 7a71698..7e756c4 100644
--- a/src/kudu/util/fault_injection.h
+++ b/src/kudu/util/fault_injection.h
@@ -14,10 +14,9 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_UTIL_FAULT_INJECTION_H
-#define KUDU_UTIL_FAULT_INJECTION_H
+#pragma once
 
-#include <stdint.h>
+#include <cstdint>
 
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
@@ -52,9 +51,11 @@
 // With some probability, return the status described by 'status_expr'.
 // This will not evaluate 'status_expr' if 'fraction_flag' is zero.
 #define MAYBE_RETURN_FAILURE(fraction_flag, status_expr) \
-  if (kudu::fault_injection::MaybeTrue(fraction_flag)) { \
-    RETURN_NOT_OK((status_expr)); \
-  }
+  do { \
+    if (kudu::fault_injection::MaybeTrue(fraction_flag)) { \
+      RETURN_NOT_OK((status_expr)); \
+    } \
+  } while (false)
 
 // Implementation details below.
 // Use the MAYBE_FAULT macro instead.
@@ -92,7 +93,5 @@ inline void MaybeInjectFixedLatency(int32_t latency) {
   DoInjectFixedLatency(latency);
 }
 
-
 } // namespace fault_injection
 } // namespace kudu
-#endif /* KUDU_UTIL_FAULT_INJECTION_H */