You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2016/08/06 04:29:11 UTC

incubator-impala git commit: IMPALA-2878: Fix Base64Decode error and remove duplicate codes.

Repository: incubator-impala
Updated Branches:
  refs/heads/master 17bf14417 -> a46b73191


IMPALA-2878: Fix Base64Decode error and remove duplicate codes.

Original impala::Base64Decode() method wouldn't return original
string if there were trailing '\0'. For example, string "a\0" would
be encoded into "YQA=", while calling original Base64Decode() method,
the return value is "a", which losts the trailing '\0' of original
string "a\0". Besides, this commit remove duplicate codes of function
impala::Base64En/Decode() and impala::StringFunctions::Base64En/Decode()

Change-Id: I0170a7d180ab048d0ff2196a24ddc53626aa7aab
Reviewed-on: http://gerrit.cloudera.org:8080/3824
Reviewed-by: Yuanhao Luo <lu...@software.ict.ac.cn>
Reviewed-by: Jim Apple <jb...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: a46b731915bbe8c71f4f0ca7aacc839edebcf6c5
Parents: 17bf144
Author: luoyuanhao <lu...@software.ict.ac.cn>
Authored: Mon Aug 1 10:04:11 2016 +0800
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Sat Aug 6 02:40:46 2016 +0000

----------------------------------------------------------------------
 be/src/exec/hdfs-table-sink.cc        |   2 +-
 be/src/exprs/string-functions-ir.cc   |  44 ++----
 be/src/service/impala-http-handler.cc |   2 +-
 be/src/util/CMakeLists.txt            |   4 +-
 be/src/util/coding-util-test.cc       | 123 ++++++++++++++++
 be/src/util/coding-util.cc            | 217 +++++++++++++++++++++++++++++
 be/src/util/coding-util.h             |  78 +++++++++++
 be/src/util/runtime-profile.cc        |   2 +-
 be/src/util/thread.cc                 |   2 +-
 be/src/util/url-coding-test.cc        | 110 ---------------
 be/src/util/url-coding.cc             | 192 -------------------------
 be/src/util/url-coding.h              |  62 ---------
 be/src/util/webserver.cc              |   2 +-
 13 files changed, 440 insertions(+), 400 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/exec/hdfs-table-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-table-sink.cc b/be/src/exec/hdfs-table-sink.cc
index 9132638..caf7b6d 100644
--- a/be/src/exec/hdfs-table-sink.cc
+++ b/be/src/exec/hdfs-table-sink.cc
@@ -29,7 +29,7 @@
 #include "runtime/string-value.inline.h"
 #include "util/impalad-metrics.h"
 #include "runtime/mem-tracker.h"
-#include "util/url-coding.h"
+#include "util/coding-util.h"
 
 #include <vector>
 #include <sstream>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/exprs/string-functions-ir.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/string-functions-ir.cc b/be/src/exprs/string-functions-ir.cc
index f524903..1c0de9d 100644
--- a/be/src/exprs/string-functions-ir.cc
+++ b/be/src/exprs/string-functions-ir.cc
@@ -24,7 +24,7 @@
 #include "exprs/expr.h"
 #include "runtime/string-value.inline.h"
 #include "runtime/tuple-row.h"
-#include "sasl/saslutil.h"
+#include "util/coding-util.h"
 #include "util/url-parser.h"
 
 #include "common/names.h"
@@ -802,12 +802,8 @@ StringVal StringFunctions::SplitPart(FunctionContext* context,
 StringVal StringFunctions::Base64Encode(FunctionContext* ctx, const StringVal& str) {
   if (str.is_null) return StringVal::null();
   if (str.len == 0) return StringVal(ctx, 0);
-  // Base64 encoding turns every 3 bytes into 4 characters. If the length is not divisible
-  // by 3, it pads the input with extra 0 bytes until it is divisible by 3. One more
-  // character must be allocated to account for sasl_encode64's null-padding of its
-  // output.
-  const unsigned out_max = 1 + 4 * ((static_cast<unsigned>(str.len) + 2) / 3);
-  if (UNLIKELY(out_max > static_cast<unsigned>(std::numeric_limits<int>::max()))) {
+  int64_t out_max = 0;
+  if (UNLIKELY(!Base64EncodeBufLen(str.len, &out_max))) {
     stringstream ss;
     ss << "Could not base64 encode a string of length " << str.len;
     ctx->AddWarning(ss.str().c_str());
@@ -815,10 +811,10 @@ StringVal StringFunctions::Base64Encode(FunctionContext* ctx, const StringVal& s
   }
   StringVal result(ctx, out_max);
   if (UNLIKELY(result.is_null)) return result;
-  unsigned out_len = 0;
-  const int encode_result = sasl_encode64(reinterpret_cast<const char*>(str.ptr), str.len,
-      reinterpret_cast<char*>(result.ptr), out_max, &out_len);
-  if (UNLIKELY(encode_result != SASL_OK || out_len != out_max - 1)) {
+  int64_t out_len = 0;
+  if (UNLIKELY(!impala::Base64Encode(
+          reinterpret_cast<const char*>(str.ptr), str.len,
+          out_max, reinterpret_cast<char*>(result.ptr), &out_len))) {
     stringstream ss;
     ss << "Could not base64 encode input in space " << out_max
        << "; actual output length " << out_len;
@@ -832,32 +828,22 @@ StringVal StringFunctions::Base64Encode(FunctionContext* ctx, const StringVal& s
 StringVal StringFunctions::Base64Decode(FunctionContext* ctx, const StringVal& str) {
   if (str.is_null) return StringVal::null();
   if (0 == str.len) return StringVal(ctx, 0);
-  // Base64 decoding turns every 4 characters into 3 bytes. If the last character of the
-  // encoded string is '=', that character (which represents 6 bits) and the last two bits
-  // of the previous character is ignored, for a total of 8 ignored bits, therefore
-  // producing one fewer byte of output. This is repeated if the second-to-last character
-  // is '='. One more byte must be allocated to account for sasl_decode64's null-padding
-  // of its output.
-  if (UNLIKELY((str.len & 3) != 0)) {
+  int64_t out_max = 0;
+  if (UNLIKELY(!Base64DecodeBufLen(
+          reinterpret_cast<const char*>(str.ptr), static_cast<int64_t>(str.len),
+          &out_max))) {
     stringstream ss;
     ss << "Invalid base64 string; input length is " << str.len
        << ", which is not a multiple of 4.";
     ctx->AddWarning(ss.str().c_str());
     return StringVal::null();
   }
-  unsigned out_max = 1 + 3 * (str.len / 4);
-  if (static_cast<char>(str.ptr[str.len - 1]) == '=') {
-    --out_max;
-    if (static_cast<char>(str.ptr[str.len - 2]) == '=') {
-      --out_max;
-    }
-  }
   StringVal result(ctx, out_max);
   if (UNLIKELY(result.is_null)) return result;
-  unsigned out_len = 0;
-  const int decode_result = sasl_decode64(reinterpret_cast<const char*>(str.ptr), str.len,
-      reinterpret_cast<char*>(result.ptr), out_max, &out_len);
-  if (UNLIKELY(decode_result != SASL_OK || out_len != out_max - 1)) {
+  int64_t out_len = 0;
+  if (UNLIKELY(!impala::Base64Decode(
+          reinterpret_cast<const char*>(str.ptr), static_cast<int64_t>(str.len),
+          out_max, reinterpret_cast<char*>(result.ptr), &out_len))) {
     stringstream ss;
     ss << "Could not base64 decode input in space " << out_max
        << "; actual output length " << out_len;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/service/impala-http-handler.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-http-handler.cc b/be/src/service/impala-http-handler.cc
index fba756d..332a429 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -24,10 +24,10 @@
 #include "service/impala-server.h"
 #include "service/query-exec-state.h"
 #include "thrift/protocol/TDebugProtocol.h"
+#include "util/coding-util.h"
 #include "util/redactor.h"
 #include "util/summary-util.h"
 #include "util/time.h"
-#include "util/url-coding.h"
 #include "util/webserver.h"
 
 #include "common/names.h"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt
index 2bf5e49..8153eaf 100644
--- a/be/src/util/CMakeLists.txt
+++ b/be/src/util/CMakeLists.txt
@@ -34,6 +34,7 @@ add_library(Util
   bit-util.cc
   bloom-filter.cc
   cgroups-mgr.cc
+  coding-util.cc
   codec.cc
   compress.cc
   cpu-info.cc
@@ -79,7 +80,6 @@ add_library(Util
   time.cc
   tuple-row-compare.cc
   url-parser.cc
-  url-coding.cc
 )
 add_dependencies(Util thrift-deps gen_ir_descriptions)
 
@@ -111,7 +111,7 @@ ADD_BE_TEST(benchmark-test)
 ADD_BE_TEST(decompress-test)
 ADD_BE_TEST(metrics-test)
 ADD_BE_TEST(debug-util-test)
-ADD_BE_TEST(url-coding-test)
+ADD_BE_TEST(coding-util-test)
 ADD_BE_TEST(bit-util-test)
 ADD_BE_TEST(rle-test)
 ADD_BE_TEST(blocking-queue-test)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/util/coding-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/coding-util-test.cc b/be/src/util/coding-util-test.cc
new file mode 100644
index 0000000..c7b0244
--- /dev/null
+++ b/be/src/util/coding-util-test.cc
@@ -0,0 +1,123 @@
+// Copyright 2012 Cloudera Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include <stdlib.h>
+#include <stdio.h>
+#include <iostream>
+#include <gtest/gtest.h>
+
+#include "common/logging.h"
+#include "util/coding-util.h"
+
+#include "common/names.h"
+
+namespace impala {
+
+// Tests encoding/decoding of input.  If expected_encoded is non-empty, the
+// encoded string is validated against it.
+void TestUrl(const string& input, const string& expected_encoded, bool hive_compat) {
+  string intermediate;
+  UrlEncode(input, &intermediate, hive_compat);
+  string output;
+  if (!expected_encoded.empty()) {
+    EXPECT_EQ(intermediate, expected_encoded);
+  }
+  EXPECT_TRUE(UrlDecode(intermediate, &output, hive_compat));
+  EXPECT_EQ(input, output);
+
+  // Convert string to vector and try that also
+  vector<uint8_t> input_vector;
+  input_vector.resize(input.size());
+  memcpy(&input_vector[0], input.c_str(), input.size());
+  string intermediate2;
+  UrlEncode(input_vector, &intermediate2, hive_compat);
+  EXPECT_EQ(intermediate, intermediate2);
+}
+
+void TestBase64(const string& input, const string& expected_encoded) {
+  string intermediate;
+  Base64Encode(input, &intermediate);
+  if (!expected_encoded.empty()) {
+    EXPECT_EQ(intermediate, expected_encoded);
+  }
+  int64_t out_max = 0;
+  EXPECT_TRUE(Base64DecodeBufLen(intermediate.c_str(), intermediate.size(), &out_max));
+  string output(out_max, '\0');
+  int64_t out_len = 0;
+  EXPECT_TRUE(Base64Decode(intermediate.c_str(), intermediate.size(),
+        out_max, const_cast<char*>(output.c_str()), &out_len));
+  output.resize(out_len);
+  EXPECT_EQ(input, output);
+
+  // Convert string to vector and try that also
+  vector<uint8_t> input_vector;
+  input_vector.resize(input.size());
+  memcpy(&input_vector[0], input.c_str(), input.size());
+  string intermediate2;
+  Base64Encode(input_vector, &intermediate2);
+  EXPECT_EQ(intermediate, intermediate2);
+}
+
+// Test URL encoding. Check that the values that are put in are the
+// same that come out.
+TEST(UrlCodingTest, Basic) {
+  string input = "ABCDEFGHIJKLMNOPQRSTUWXYZ1234567890~!@#$%^&*()<>?,./:\";'{}|[]\\_+-=";
+  TestUrl(input, "", false);
+  TestUrl(input, "", true);
+}
+
+TEST(UrlCodingTest, HiveExceptions) {
+  TestUrl(" +", " +", true);
+}
+
+TEST(UrlCodingTest, BlankString) {
+  TestUrl("", "", false);
+  TestUrl("", "", true);
+}
+
+TEST(UrlCodingTest, PathSeparators) {
+  TestUrl("/home/impala/directory/", "%2Fhome%2Fimpala%2Fdirectory%2F", false);
+  TestUrl("/home/impala/directory/", "%2Fhome%2Fimpala%2Fdirectory%2F", true);
+}
+
+TEST(Base64Test, Basic) {
+  TestBase64("a", "YQ==");
+  TestBase64("ab", "YWI=");
+  TestBase64("abc", "YWJj");
+  TestBase64("abcd", "YWJjZA==");
+  TestBase64("abcde", "YWJjZGU=");
+  TestBase64("abcdef", "YWJjZGVm");
+  TestBase64(string("a\0", 2), "YQA=");
+  TestBase64(string("ab\0", 3), "YWIA");
+  TestBase64(string("abc\0", 4), "YWJjAA==");
+  TestBase64(string("abcd\0", 5), "YWJjZAA=");
+  TestBase64(string("abcde\0", 6), "YWJjZGUA");
+  TestBase64(string("abcdef\0", 7), "YWJjZGVmAA==");
+  TestBase64(string("a\0b", 3), "YQBi");
+  TestBase64(string("a\0b\0", 4), "YQBiAA==");
+}
+
+TEST(HtmlEscapingTest, Basic) {
+  string before = "<html><body>&amp";
+  stringstream after;
+  EscapeForHtml(before, &after);
+  EXPECT_EQ(after.str(), "&lt;html&gt;&lt;body&gt;&amp;amp");
+}
+
+}
+
+int main(int argc, char **argv) {
+  ::testing::InitGoogleTest(&argc, argv);
+  return RUN_ALL_TESTS();
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/util/coding-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/coding-util.cc b/be/src/util/coding-util.cc
new file mode 100644
index 0000000..bba6fc1
--- /dev/null
+++ b/be/src/util/coding-util.cc
@@ -0,0 +1,217 @@
+// Copyright 2012 Cloudera Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "util/coding-util.h"
+
+#include <exception>
+#include <sstream>
+#include <boost/algorithm/string.hpp>
+
+#include "common/compiler-util.h"
+#include "common/logging.h"
+
+#include "common/names.h"
+#include "sasl/saslutil.h"
+
+using boost::algorithm::is_any_of;
+using namespace impala;
+using std::uppercase;
+
+namespace impala {
+
+// Hive selectively encodes characters. This is the whitelist of
+// characters it will encode.
+// See common/src/java/org/apache/hadoop/hive/common/FileUtils.java
+// in the Hive source code for the source of this list.
+static function<bool (char)> HiveShouldEscape = is_any_of("\"#%\\*/:=?\u00FF");
+
+// It is more convenient to maintain the complement of the set of
+// characters to escape when not in Hive-compat mode.
+static function<bool (char)> ShouldNotEscape = is_any_of("-_.~");
+
+static inline void UrlEncode(const char* in, int in_len, string* out, bool hive_compat) {
+  (*out).reserve(in_len);
+  stringstream ss;
+  for (int i = 0; i < in_len; ++i) {
+    const char ch = in[i];
+    // Escape the character iff a) we are in Hive-compat mode and the
+    // character is in the Hive whitelist or b) we are not in
+    // Hive-compat mode, and the character is not alphanumeric or one
+    // of the four commonly excluded characters.
+    if ((hive_compat && HiveShouldEscape(ch)) ||
+        (!hive_compat && !(isalnum(ch) || ShouldNotEscape(ch)))) {
+      ss << '%' << uppercase << hex << static_cast<uint32_t>(ch);
+    } else {
+      ss << ch;
+    }
+  }
+
+  (*out) = ss.str();
+}
+
+void UrlEncode(const vector<uint8_t>& in, string* out, bool hive_compat) {
+  if (in.empty()) {
+    *out = "";
+  } else {
+    UrlEncode(reinterpret_cast<const char*>(&in[0]), in.size(), out, hive_compat);
+  }
+}
+
+void UrlEncode(const string& in, string* out, bool hive_compat) {
+  UrlEncode(in.c_str(), in.size(), out, hive_compat);
+}
+
+// Adapted from
+// http://www.boost.org/doc/libs/1_40_0/doc/html/boost_asio/
+//   example/http/server3/request_handler.cpp
+// See http://www.boost.org/LICENSE_1_0.txt for license for this method.
+bool UrlDecode(const string& in, string* out, bool hive_compat) {
+  out->clear();
+  out->reserve(in.size());
+  for (size_t i = 0; i < in.size(); ++i) {
+    if (in[i] == '%') {
+      if (i + 3 <= in.size()) {
+        int value = 0;
+        istringstream is(in.substr(i + 1, 2));
+        if (is >> hex >> value) {
+          (*out) += static_cast<char>(value);
+          i += 2;
+        } else {
+          return false;
+        }
+      } else {
+        return false;
+      }
+    } else if (!hive_compat && in[i] == '+') { // Hive does not encode ' ' as '+'
+      (*out) += ' ';
+    } else {
+      (*out) += in[i];
+    }
+  }
+  return true;
+}
+
+bool Base64EncodeBufLen(int64_t in_len, int64_t* out_max) {
+  // Base64 encoding turns every 3 bytes into 4 characters. If the length is not
+  // divisible by 3, it pads the input with extra 0 bytes until it is divisible by 3.
+  // One more character must be allocated to account for Base64Encode's null-padding
+  // of its output.
+  *out_max = 1 + 4 * ((in_len + 2) / 3);
+  if (UNLIKELY(in_len < 0 ||
+        *out_max > static_cast<unsigned>(std::numeric_limits<int>::max()))) {
+    return false;
+  }
+  return true;
+}
+
+bool Base64Encode(const char* in, int64_t in_len, int64_t out_max, char* out,
+    int64_t* out_len) {
+  if (UNLIKELY(in_len < 0 || in_len > std::numeric_limits<unsigned>::max() ||
+        out_max < 0 || out_max > std::numeric_limits<unsigned>::max())) {
+    return false;
+  }
+  const int encode_result = sasl_encode64(in, static_cast<unsigned>(in_len), out,
+      static_cast<unsigned>(out_max), reinterpret_cast<unsigned*>(out_len));
+  if (UNLIKELY(encode_result != SASL_OK || *out_len != out_max - 1)) return false;
+  return true;
+}
+
+static inline void Base64Encode(const char* in, int64_t in_len, stringstream* out) {
+  if (in_len == 0) {
+    (*out) << "";
+    return;
+  }
+  int64_t out_max = 0;
+  if (UNLIKELY(!Base64EncodeBufLen(in_len, &out_max))) return;
+  string result(out_max, '\0');
+  int64_t out_len = 0;
+  if (UNLIKELY(!Base64Encode(in, in_len, out_max, const_cast<char*>(result.c_str()),
+          &out_len))) {
+    return;
+  }
+  result.resize(out_len);
+  (*out) << result;
+}
+
+void Base64Encode(const vector<uint8_t>& in, string* out) {
+  if (in.empty()) {
+    *out = "";
+  } else {
+    stringstream ss;
+    Base64Encode(in, &ss);
+    *out = ss.str();
+  }
+}
+
+void Base64Encode(const vector<uint8_t>& in, stringstream* out) {
+  if (!in.empty()) {
+    // Boost does not like non-null terminated strings
+    string tmp(reinterpret_cast<const char*>(&in[0]), in.size());
+    Base64Encode(tmp.c_str(), tmp.size(), out);
+  }
+}
+
+void Base64Encode(const string& in, string* out) {
+  stringstream ss;
+  Base64Encode(in.c_str(), in.size(), &ss);
+  *out = ss.str();
+}
+
+void Base64Encode(const string& in, stringstream* out) {
+  Base64Encode(in.c_str(), in.size(), out);
+}
+
+bool Base64DecodeBufLen(const char* in, int64_t in_len, int64_t* out_max) {
+  // Base64 decoding turns every 4 characters into 3 bytes. If the last character of the
+  // encoded string is '=', that character (which represents 6 bits) and the last two bits
+  // of the previous character is ignored, for a total of 8 ignored bits, therefore
+  // producing one fewer byte of output. This is repeated if the second-to-last character
+  // is '='. One more byte must be allocated to account for Base64Decode's null-padding
+  // of its output.
+  if (UNLIKELY((in_len & 3) != 0)) return false;
+  *out_max = 1 + 3 * (in_len / 4);
+  if (in[in_len - 1] == '=') {
+    --(*out_max);
+    if (in[in_len - 2] == '=') {
+      --(*out_max);
+    }
+  }
+  return true;
+}
+
+bool Base64Decode(const char* in, int64_t in_len, int64_t out_max, char* out,
+    int64_t* out_len) {
+  if (UNLIKELY((in_len & 3) != 0)) return false;
+  const int decode_result = sasl_decode64(in, static_cast<unsigned>(in_len), out,
+      static_cast<unsigned>(out_max), reinterpret_cast<unsigned*>(out_len));
+  if (UNLIKELY(decode_result != SASL_OK || *out_len != out_max - 1)) return false;
+  return true;
+}
+
+void EscapeForHtml(const string& in, stringstream* out) {
+  DCHECK(out != NULL);
+  for (const char c: in) {
+    switch (c) {
+      case '<': (*out) << "&lt;";
+                break;
+      case '>': (*out) << "&gt;";
+                break;
+      case '&': (*out) << "&amp;";
+                break;
+      default: (*out) << c;
+    }
+  }
+}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/util/coding-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/coding-util.h b/be/src/util/coding-util.h
new file mode 100644
index 0000000..5286953
--- /dev/null
+++ b/be/src/util/coding-util.h
@@ -0,0 +1,78 @@
+// Copyright 2012 Cloudera Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef UTIL_CODING_UTIL_H
+#define UTIL_CODING_UTIL_H
+
+#include <string>
+#include <vector>
+#include <boost/cstdint.hpp>
+
+namespace impala {
+
+/// Utility method to URL-encode a string (that is, replace special
+/// characters with %<hex value in ascii>).
+/// The optional parameter hive_compat controls whether we mimic Hive's
+/// behaviour when encoding a string, which is only to encode certain
+/// characters (excluding, e.g., ' ')
+void UrlEncode(const std::string& in, std::string* out, bool hive_compat = false);
+void UrlEncode(const std::vector<uint8_t>& in, std::string* out,
+    bool hive_compat = false);
+
+/// Utility method to decode a string that was URL-encoded. Returns
+/// true unless the string could not be correctly decoded.
+/// The optional parameter hive_compat controls whether or not we treat
+/// the strings as encoded by Hive, which means selectively ignoring
+/// certain characters like ' '.
+bool UrlDecode(const std::string& in, std::string* out, bool hive_compat = false);
+
+/// Calculate the maximum output buffer size needed for Base64Encode. Returns false if
+/// in_len is negative or too large.
+bool Base64EncodeBufLen(int64_t in_len, int64_t* out_max);
+
+/// Returns true if encoded successfully, otherwise false. out points to the output
+/// data, the space of size out_max should be allocated before calling this function.
+/// out_len saves the actual length of encoded string.
+bool Base64Encode(const char* in, int64_t in_len, int64_t out_max, char* out,
+    int64_t* out_len);
+
+/// Utility method to encode input as base-64 encoded.  This is not
+/// very performant (multiple string copies) and should not be used
+/// in a hot path.
+void Base64Encode(const std::vector<uint8_t>& in, std::string* out);
+void Base64Encode(const std::vector<uint8_t>& in, std::stringstream* out);
+void Base64Encode(const std::string& in, std::string* out);
+void Base64Encode(const std::string& in, std::stringstream* out);
+
+/// Calculate the maximum output buffer size needed for Base64Decode. Returns false if
+/// in_len is invalid.
+bool Base64DecodeBufLen(const char* in, int64_t in_len, int64_t* out_max);
+
+/// Utility method to decode a base-64 encoded string. Returns true if decoded
+/// successfully, otherwise false. out points to the output data, the space of size
+/// out_max should be allocated before calling this function. out_len saves the actual
+/// length of decoded string.
+bool Base64Decode(const char* in, int64_t in_len, int64_t out_max, char* out,
+    int64_t* out_len);
+
+/// Replaces &, < and > with &amp;, &lt; and &gt; respectively. This is
+/// not the full set of required encodings, but one that should be
+/// added to on a case-by-case basis. Slow, since it necessarily
+/// inspects each character in turn, and copies them all to *out; use
+/// judiciously.
+void EscapeForHtml(const std::string& in, std::stringstream* out);
+
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/util/runtime-profile.cc
----------------------------------------------------------------------
diff --git a/be/src/util/runtime-profile.cc b/be/src/util/runtime-profile.cc
index ce5e934..a16967f 100644
--- a/be/src/util/runtime-profile.cc
+++ b/be/src/util/runtime-profile.cc
@@ -23,13 +23,13 @@
 
 #include "common/object-pool.h"
 #include "rpc/thrift-util.h"
+#include "util/coding-util.h"
 #include "util/compress.h"
 #include "util/container-util.h"
 #include "util/cpu-info.h"
 #include "util/debug-util.h"
 #include "util/periodic-counter-updater.h"
 #include "util/redactor.h"
-#include "util/url-coding.h"
 
 #include "common/names.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/util/thread.cc
----------------------------------------------------------------------
diff --git a/be/src/util/thread.cc b/be/src/util/thread.cc
index 6325a0d..af09db5 100644
--- a/be/src/util/thread.cc
+++ b/be/src/util/thread.cc
@@ -20,12 +20,12 @@
 #include <sys/syscall.h>
 #include <sys/types.h>
 
+#include "util/coding-util.h"
 #include "util/debug-util.h"
 #include "util/error-util.h"
 #include "util/cgroups-mgr.h"
 #include "util/metrics.h"
 #include "util/webserver.h"
-#include "util/url-coding.h"
 #include "util/os-util.h"
 
 #include "common/names.h"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/util/url-coding-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/url-coding-test.cc b/be/src/util/url-coding-test.cc
deleted file mode 100644
index 438934b..0000000
--- a/be/src/util/url-coding-test.cc
+++ /dev/null
@@ -1,110 +0,0 @@
-// Copyright 2012 Cloudera Inc.
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-// http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-#include <stdlib.h>
-#include <stdio.h>
-#include <iostream>
-#include <gtest/gtest.h>
-
-#include "common/logging.h"
-#include "util/url-coding.h"
-
-#include "common/names.h"
-
-namespace impala {
-
-// Tests encoding/decoding of input.  If expected_encoded is non-empty, the
-// encoded string is validated against it.
-void TestUrl(const string& input, const string& expected_encoded, bool hive_compat) {
-  string intermediate;
-  UrlEncode(input, &intermediate, hive_compat);
-  string output;
-  if (!expected_encoded.empty()) {
-    EXPECT_EQ(intermediate, expected_encoded);
-  }
-  EXPECT_TRUE(UrlDecode(intermediate, &output, hive_compat));
-  EXPECT_EQ(input, output);
-
-  // Convert string to vector and try that also
-  vector<uint8_t> input_vector;
-  input_vector.resize(input.size());
-  memcpy(&input_vector[0], input.c_str(), input.size());
-  string intermediate2;
-  UrlEncode(input_vector, &intermediate2, hive_compat);
-  EXPECT_EQ(intermediate, intermediate2);
-}
-
-void TestBase64(const string& input, const string& expected_encoded) {
-  string intermediate;
-  Base64Encode(input, &intermediate);
-  string output;
-  if (!expected_encoded.empty()) {
-    EXPECT_EQ(intermediate, expected_encoded);
-  }
-  EXPECT_TRUE(Base64Decode(intermediate, &output));
-  EXPECT_EQ(input, output);
-
-  // Convert string to vector and try that also
-  vector<uint8_t> input_vector;
-  input_vector.resize(input.size());
-  memcpy(&input_vector[0], input.c_str(), input.size());
-  string intermediate2;
-  Base64Encode(input_vector, &intermediate2);
-  EXPECT_EQ(intermediate, intermediate2);
-}
-
-// Test URL encoding. Check that the values that are put in are the
-// same that come out.
-TEST(UrlCodingTest, Basic) {
-  string input = "ABCDEFGHIJKLMNOPQRSTUWXYZ1234567890~!@#$%^&*()<>?,./:\";'{}|[]\\_+-=";
-  TestUrl(input, "", false);
-  TestUrl(input, "", true);
-}
-
-TEST(UrlCodingTest, HiveExceptions) {
-  TestUrl(" +", " +", true);
-}
-
-TEST(UrlCodingTest, BlankString) {
-  TestUrl("", "", false);
-  TestUrl("", "", true);
-}
-
-TEST(UrlCodingTest, PathSeparators) {
-  TestUrl("/home/impala/directory/", "%2Fhome%2Fimpala%2Fdirectory%2F", false);
-  TestUrl("/home/impala/directory/", "%2Fhome%2Fimpala%2Fdirectory%2F", true);
-}
-
-TEST(Base64Test, Basic) {
-  TestBase64("a", "YQ==");
-  TestBase64("ab", "YWI=");
-  TestBase64("abc", "YWJj");
-  TestBase64("abcd", "YWJjZA==");
-  TestBase64("abcde", "YWJjZGU=");
-  TestBase64("abcdef", "YWJjZGVm");
-}
-
-TEST(HtmlEscapingTest, Basic) {
-  string before = "<html><body>&amp";
-  stringstream after;
-  EscapeForHtml(before, &after);
-  EXPECT_EQ(after.str(), "&lt;html&gt;&lt;body&gt;&amp;amp");
-}
-
-}
-
-int main(int argc, char **argv) {
-  ::testing::InitGoogleTest(&argc, argv);
-  return RUN_ALL_TESTS();
-}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/util/url-coding.cc
----------------------------------------------------------------------
diff --git a/be/src/util/url-coding.cc b/be/src/util/url-coding.cc
deleted file mode 100644
index 20e5783..0000000
--- a/be/src/util/url-coding.cc
+++ /dev/null
@@ -1,192 +0,0 @@
-// Copyright 2012 Cloudera Inc.
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-// http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-#include "util/url-coding.h"
-
-#include <exception>
-#include <sstream>
-#include <boost/algorithm/string.hpp>
-#include <boost/archive/iterators/base64_from_binary.hpp>
-#include <boost/archive/iterators/binary_from_base64.hpp>
-#include <boost/archive/iterators/transform_width.hpp>
-
-#include "common/logging.h"
-
-#include "common/names.h"
-
-using boost::algorithm::is_any_of;
-using boost::archive::iterators::base64_from_binary;
-using boost::archive::iterators::binary_from_base64;
-using boost::archive::iterators::transform_width;
-using namespace impala;
-using std::uppercase;
-
-namespace impala {
-
-// Hive selectively encodes characters. This is the whitelist of
-// characters it will encode.
-// See common/src/java/org/apache/hadoop/hive/common/FileUtils.java
-// in the Hive source code for the source of this list.
-static function<bool (char)> HiveShouldEscape = is_any_of("\"#%\\*/:=?\u00FF");
-
-// It is more convenient to maintain the complement of the set of
-// characters to escape when not in Hive-compat mode.
-static function<bool (char)> ShouldNotEscape = is_any_of("-_.~");
-
-static inline void UrlEncode(const char* in, int in_len, string* out, bool hive_compat) {
-  (*out).reserve(in_len);
-  stringstream ss;
-  for (int i = 0; i < in_len; ++i) {
-    const char ch = in[i];
-    // Escape the character iff a) we are in Hive-compat mode and the
-    // character is in the Hive whitelist or b) we are not in
-    // Hive-compat mode, and the character is not alphanumeric or one
-    // of the four commonly excluded characters.
-    if ((hive_compat && HiveShouldEscape(ch)) ||
-        (!hive_compat && !(isalnum(ch) || ShouldNotEscape(ch)))) {
-      ss << '%' << uppercase << hex << static_cast<uint32_t>(ch);
-    } else {
-      ss << ch;
-    }
-  }
-
-  (*out) = ss.str();
-}
-
-void UrlEncode(const vector<uint8_t>& in, string* out, bool hive_compat) {
-  if (in.empty()) {
-    *out = "";
-  } else {
-    UrlEncode(reinterpret_cast<const char*>(&in[0]), in.size(), out, hive_compat);
-  }
-}
-
-void UrlEncode(const string& in, string* out, bool hive_compat) {
-  UrlEncode(in.c_str(), in.size(), out, hive_compat);
-}
-
-// Adapted from
-// http://www.boost.org/doc/libs/1_40_0/doc/html/boost_asio/
-//   example/http/server3/request_handler.cpp
-// See http://www.boost.org/LICENSE_1_0.txt for license for this method.
-bool UrlDecode(const string& in, string* out, bool hive_compat) {
-  out->clear();
-  out->reserve(in.size());
-  for (size_t i = 0; i < in.size(); ++i) {
-    if (in[i] == '%') {
-      if (i + 3 <= in.size()) {
-        int value = 0;
-        istringstream is(in.substr(i + 1, 2));
-        if (is >> hex >> value) {
-          (*out) += static_cast<char>(value);
-          i += 2;
-        } else {
-          return false;
-        }
-      } else {
-        return false;
-      }
-    } else if (!hive_compat && in[i] == '+') { // Hive does not encode ' ' as '+'
-      (*out) += ' ';
-    } else {
-      (*out) += in[i];
-    }
-  }
-  return true;
-}
-
-static inline void Base64Encode(const char* in, int in_len, stringstream* out) {
-  typedef base64_from_binary<transform_width<const char*, 6, 8>> base64_encode;
-  // Base64 encodes 8 byte chars as 6 bit values.
-  stringstream::pos_type len_before = out->tellp();
-  copy(base64_encode(in), base64_encode(in + in_len), std::ostream_iterator<char>(*out));
-  int bytes_written = out->tellp() - len_before;
-  // Pad with = to make it valid base64 encoded string
-  int num_pad = bytes_written % 4;
-  if (num_pad != 0) {
-    num_pad = 4 - num_pad;
-    for (int i = 0; i < num_pad; ++i) {
-      (*out) << "=";
-    }
-  }
-  DCHECK_EQ((out->tellp() - len_before) % 4, 0);
-}
-
-void Base64Encode(const vector<uint8_t>& in, string* out) {
-  if (in.empty()) {
-    *out = "";
-  } else {
-    stringstream ss;
-    Base64Encode(in, &ss);
-    *out = ss.str();
-  }
-}
-
-void Base64Encode(const vector<uint8_t>& in, stringstream* out) {
-  if (!in.empty()) {
-    // Boost does not like non-null terminated strings
-    string tmp(reinterpret_cast<const char*>(&in[0]), in.size());
-    Base64Encode(tmp.c_str(), tmp.size(), out);
-  }
-}
-
-void Base64Encode(const string& in, string* out) {
-  stringstream ss;
-  Base64Encode(in.c_str(), in.size(), &ss);
-  *out = ss.str();
-}
-
-void Base64Encode(const string& in, stringstream* out) {
-  Base64Encode(in.c_str(), in.size(), out);
-}
-
-bool Base64Decode(const string& in, string* out) {
-  typedef transform_width<
-      binary_from_base64<string::const_iterator> , 8, 6> base64_decode;
-  string tmp = in;
-  // Replace padding with base64 encoded NULL
-  replace(tmp.begin(), tmp.end(), '=', 'A');
-  try {
-    *out = string(base64_decode(tmp.begin()), base64_decode(tmp.end()));
-  } catch(std::exception& e) {
-    return false;
-  }
-
-  // Remove trailing '\0' that were added as padding.  Since \0 is special,
-  // the boost functions get confused so do this manually.
-  int num_padded_chars = 0;
-  for (int i = out->size() - 1; i >= 0; --i) {
-    if ((*out)[i] != '\0') break;
-    ++num_padded_chars;
-  }
-  out->resize(out->size() - num_padded_chars);
-  return true;
-}
-
-void EscapeForHtml(const string& in, stringstream* out) {
-  DCHECK(out != NULL);
-  for (const char c: in) {
-    switch (c) {
-      case '<': (*out) << "&lt;";
-                break;
-      case '>': (*out) << "&gt;";
-                break;
-      case '&': (*out) << "&amp;";
-                break;
-      default: (*out) << c;
-    }
-  }
-}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/util/url-coding.h
----------------------------------------------------------------------
diff --git a/be/src/util/url-coding.h b/be/src/util/url-coding.h
deleted file mode 100644
index 1ac5d5e..0000000
--- a/be/src/util/url-coding.h
+++ /dev/null
@@ -1,62 +0,0 @@
-// Copyright 2012 Cloudera Inc.
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-// http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-#ifndef UTIL_URL_CODING_H
-#define UTIL_URL_CODING_H
-
-#include <string>
-#include <vector>
-#include <boost/cstdint.hpp>
-
-namespace impala {
-
-/// Utility method to URL-encode a string (that is, replace special
-/// characters with %<hex value in ascii>).
-/// The optional parameter hive_compat controls whether we mimic Hive's
-/// behaviour when encoding a string, which is only to encode certain
-/// characters (excluding, e.g., ' ')
-void UrlEncode(const std::string& in, std::string* out, bool hive_compat = false);
-void UrlEncode(const std::vector<uint8_t>& in, std::string* out,
-    bool hive_compat = false);
-
-/// Utility method to decode a string that was URL-encoded. Returns
-/// true unless the string could not be correctly decoded.
-/// The optional parameter hive_compat controls whether or not we treat
-/// the strings as encoded by Hive, which means selectively ignoring
-/// certain characters like ' '.
-bool UrlDecode(const std::string& in, std::string* out, bool hive_compat = false);
-
-/// Utility method to encode input as base-64 encoded.  This is not
-/// very performant (multiple string copies) and should not be used
-/// in a hot path.
-void Base64Encode(const std::vector<uint8_t>& in, std::string* out);
-void Base64Encode(const std::vector<uint8_t>& in, std::stringstream* out);
-void Base64Encode(const std::string& in, std::string* out);
-void Base64Encode(const std::string& in, std::stringstream* out);
-
-/// Utility method to decode base64 encoded strings.  Also not extremely
-/// performant.
-/// Returns true unless the string could not be correctly decoded.
-bool Base64Decode(const std::string& in, std::string* out);
-
-/// Replaces &, < and > with &amp;, &lt; and &gt; respectively. This is
-/// not the full set of required encodings, but one that should be
-/// added to on a case-by-case basis. Slow, since it necessarily
-/// inspects each character in turn, and copies them all to *out; use
-/// judiciously.
-void EscapeForHtml(const std::string& in, std::stringstream* out);
-
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a46b7319/be/src/util/webserver.cc
----------------------------------------------------------------------
diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index b514c36..8069e01 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -33,13 +33,13 @@
 #include "rpc/thrift-util.h"
 #include "thirdparty/mustache/mustache.h"
 #include "util/asan.h"
+#include "util/coding-util.h"
 #include "util/cpu-info.h"
 #include "util/disk-info.h"
 #include "util/mem-info.h"
 #include "util/os-info.h"
 #include "util/os-util.h"
 #include "util/process-state-info.h"
-#include "util/url-coding.h"
 #include "util/debug-util.h"
 #include "util/pretty-printer.h"
 #include "util/stopwatch.h"