You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by sp...@apache.org on 2017/02/28 16:14:49 UTC

[1/3] incubator-quickstep git commit: Minor refactored CommandExecutor. [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/fix-gcc-build 6b0396ae5 -> 132fed63d (forced update)


Minor refactored CommandExecutor.


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

Branch: refs/heads/fix-gcc-build
Commit: 4437b9d44e888229a3b6b2884ccdd0033c8e5fcb
Parents: b24349c
Author: Zuyu Zhang <zu...@apache.org>
Authored: Sun Feb 26 23:17:26 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Feb 27 00:24:31 2017 -0800

----------------------------------------------------------------------
 catalog/CatalogDatabase.hpp | 10 +++++
 cli/CMakeLists.txt          |  6 ++-
 cli/CommandExecutor.cpp     | 96 +++++++++++++++++-----------------------
 cli/CommandExecutor.hpp     |  9 ----
 cli/Constants.hpp           | 44 ++++++++++++++++++
 utility/StringUtil.cpp      | 11 +++++
 utility/StringUtil.hpp      |  6 +++
 7 files changed, 117 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4437b9d4/catalog/CatalogDatabase.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogDatabase.hpp b/catalog/CatalogDatabase.hpp
index a0804a2..b071d09 100644
--- a/catalog/CatalogDatabase.hpp
+++ b/catalog/CatalogDatabase.hpp
@@ -343,6 +343,16 @@ class CatalogDatabase : public CatalogDatabaseLite {
   serialization::CatalogDatabase getProto() const;
 
   /**
+   * @brief Check whether this CatalogDatabase is empty.
+   *
+   * @return true if empty, false otherwise.
+   **/
+  bool empty() const {
+    SpinSharedMutexSharedLock<false> lock(relations_mutex_);
+    return rel_map_.empty();
+  }
+
+  /**
    * @brief Get the number of child relations.
    *
    * @return The number of child relations.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4437b9d4/cli/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/CMakeLists.txt b/cli/CMakeLists.txt
index c5f3915..7b4319a 100644
--- a/cli/CMakeLists.txt
+++ b/cli/CMakeLists.txt
@@ -44,6 +44,7 @@ configure_file (
   "${CMAKE_CURRENT_BINARY_DIR}/CliConfig.h"
 )
 add_library(quickstep_cli_CommandExecutor CommandExecutor.cpp CommandExecutor.hpp)
+add_library(quickstep_cli_Constants ../empty_src.cpp Constants.hpp)
 
 # Declare micro-libs and link dependencies:
 add_library(quickstep_cli_DropRelation DropRelation.cpp DropRelation.hpp)
@@ -86,6 +87,7 @@ target_link_libraries(quickstep_cli_CommandExecutor
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogRelationStatistics
+                      quickstep_cli_Constants
                       quickstep_cli_DropRelation
                       quickstep_cli_PrintToScreen
                       quickstep_expressions_aggregation_AggregateFunctionMax
@@ -105,7 +107,8 @@ target_link_libraries(quickstep_cli_CommandExecutor
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_utility_PtrVector
-                      quickstep_utility_SqlError)
+                      quickstep_utility_SqlError
+                      quickstep_utility_StringUtil)
 
 target_link_libraries(quickstep_cli_DefaultsConfigurator
                       glog
@@ -148,6 +151,7 @@ add_library(quickstep_cli ../empty_src.cpp CliModule.hpp)
 
 target_link_libraries(quickstep_cli
                       quickstep_cli_CommandExecutor
+                      quickstep_cli_Constants
                       quickstep_cli_DefaultsConfigurator
                       quickstep_cli_DropRelation
                       quickstep_cli_Flags

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4437b9d4/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index 78bff98..7f63469 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -32,6 +32,7 @@
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogRelationSchema.hpp"
 #include "catalog/CatalogRelationStatistics.hpp"
+#include "cli/Constants.hpp"
 #include "cli/DropRelation.hpp"
 #include "cli/PrintToScreen.hpp"
 #include "expressions/aggregation/AggregateFunctionMax.hpp"
@@ -52,6 +53,7 @@
 #include "types/TypedValue.hpp"
 #include "utility/PtrVector.hpp"
 #include "utility/SqlError.hpp"
+#include "utility/StringUtil.hpp"
 
 #include "glog/logging.h"
 
@@ -70,63 +72,60 @@ namespace quickstep {
 namespace cli {
 namespace {
 
-namespace C = ::quickstep::cli;
-
 void ExecuteDescribeDatabase(
-    const PtrVector<ParseString> *arguments,
+    const PtrVector<ParseString> &arguments,
     const CatalogDatabase &catalog_database,
-    StorageManager *storage_manager,
     FILE *out) {
   // Column width initialized to 6 to take into account the header name
   // and the column value table
-  int max_column_width = C::kInitMaxColumnWidth;
+  int max_column_width = kInitMaxColumnWidth;
   vector<std::size_t> num_blocks;
   const CatalogRelation *relation = nullptr;
-  if (arguments->size() == 0) {
+  if (arguments.empty()) {
     for (const CatalogRelation &rel : catalog_database) {
       max_column_width =
           std::max(static_cast<int>(rel.getName().length()), max_column_width);
       num_blocks.push_back(rel.size_blocks());
     }
   } else {
-    const ParseString &table_name = arguments->front();
+    const ParseString &table_name = arguments.front();
     const std::string &table_name_val = table_name.value();
     relation = catalog_database.getRelationByName(table_name_val);
 
     if (relation == nullptr) {
-      THROW_SQL_ERROR_AT(&(arguments->front())) << " Unrecognized relation " << table_name_val;
+      THROW_SQL_ERROR_AT(&(arguments.front())) << " Unrecognized relation " << table_name_val;
     }
     max_column_width = std::max(static_cast<int>(relation->getName().length()),
                                     max_column_width);
     num_blocks.push_back(relation->size_blocks());
   }
   // Only if we have relations work on the printing logic.
-  if (catalog_database.size() > 0) {
+  if (!catalog_database.empty()) {
     const std::size_t max_num_blocks = *std::max_element(num_blocks.begin(), num_blocks.end());
     const int max_num_blocks_digits = std::max(PrintToScreen::GetNumberOfDigits(max_num_blocks),
-                                      C::kInitMaxColumnWidth+2);
+                                      kInitMaxColumnWidth + 2);
     vector<int> column_widths;
-    column_widths.push_back(max_column_width +1);
-    column_widths.push_back(C::kInitMaxColumnWidth + 1);
+    column_widths.push_back(max_column_width + 1);
+    column_widths.push_back(kInitMaxColumnWidth + 1);
     column_widths.push_back(max_num_blocks_digits + 1);
     fputs("       List of relations\n\n", out);
-    fprintf(out, "%-*s |", max_column_width+1, " Name");
-    fprintf(out, "%-*s |", C::kInitMaxColumnWidth, " Type");
+    fprintf(out, "%-*s |", max_column_width + 1, " Name");
+    fprintf(out, "%-*s |", kInitMaxColumnWidth, " Type");
     fprintf(out, "%-*s\n", max_num_blocks_digits, " Blocks");
     PrintToScreen::printHBar(column_widths, out);
     //  If there are no argument print the entire list of tables
     //  else print the particular table only.
     vector<std::size_t>::const_iterator num_blocks_it = num_blocks.begin();
-    if (arguments->size() == 0) {
+    if (arguments.empty()) {
       for (const CatalogRelation &rel : catalog_database) {
         fprintf(out, " %-*s |", max_column_width, rel.getName().c_str());
-        fprintf(out, " %-*s |", C::kInitMaxColumnWidth - 1, "table");
+        fprintf(out, " %-*s |", kInitMaxColumnWidth - 1, "table");
         fprintf(out, " %-*lu\n", max_num_blocks_digits - 1, *num_blocks_it);
         ++num_blocks_it;
       }
     } else {
       fprintf(out, " %-*s |", max_column_width, relation->getName().c_str());
-      fprintf(out, " %-*s |", C::kInitMaxColumnWidth -1, "table");
+      fprintf(out, " %-*s |", kInitMaxColumnWidth - 1, "table");
       fprintf(out, " %-*lu\n", max_num_blocks_digits - 1, *num_blocks_it);
       ++num_blocks_it;
     }
@@ -135,18 +134,18 @@ void ExecuteDescribeDatabase(
 }
 
 void ExecuteDescribeTable(
-    const PtrVector<ParseString> *arguments,
+    const PtrVector<ParseString> &arguments,
     const CatalogDatabase &catalog_database, FILE *out) {
-  const ParseString &table_name = arguments->front();
+  const ParseString &table_name = arguments.front();
   const std::string &table_name_val = table_name.value();
   const CatalogRelation *relation =
       catalog_database.getRelationByName(table_name_val);
   if (relation == nullptr) {
-    THROW_SQL_ERROR_AT(&(arguments->front())) << " Unrecognized relation "  << table_name_val;
+    THROW_SQL_ERROR_AT(&(arguments.front())) << " Unrecognized relation "  << table_name_val;
   }
   vector<int> column_widths;
-  int max_attr_column_width = C::kInitMaxColumnWidth;
-  int max_type_column_width = C::kInitMaxColumnWidth;
+  int max_attr_column_width = kInitMaxColumnWidth;
+  int max_type_column_width = kInitMaxColumnWidth;
 
   for (const CatalogAttribute &attr : *relation) {
     // Printed column needs to be wide enough to print:
@@ -160,12 +159,12 @@ void ExecuteDescribeTable(
             static_cast<int>(attr.getType().getName().length()));
   }
   // Add room for one extra character to allow spacing between the column ending and the vertical bar
-  column_widths.push_back(max_attr_column_width+1);
-  column_widths.push_back(max_type_column_width+1);
+  column_widths.push_back(max_attr_column_width + 1);
+  column_widths.push_back(max_type_column_width + 1);
 
-  fprintf(out, "%*s \"%s\"\n", C::kInitMaxColumnWidth, "Table", table_name_val.c_str());
-  fprintf(out, "%-*s |", max_attr_column_width+1, " Column");
-  fprintf(out, "%-*s\n", max_type_column_width+1, " Type");
+  fprintf(out, "%*s \"%s\"\n", kInitMaxColumnWidth, "Table", table_name_val.c_str());
+  fprintf(out, "%-*s |", max_attr_column_width + 1, " Column");
+  fprintf(out, "%-*s\n", max_type_column_width + 1, " Type");
   PrintToScreen::printHBar(column_widths, out);
   for (const CatalogAttribute &attr : *relation) {
     fprintf(out, " %-*s |", max_attr_column_width,
@@ -175,7 +174,7 @@ void ExecuteDescribeTable(
   }
   // TODO(rogers): Add handlers for partitioning information.
   if (relation->hasIndexScheme()) {
-    fprintf(out, "%*s\n", C::kInitMaxColumnWidth+2, " Indexes");
+    fprintf(out, "%*s\n", kInitMaxColumnWidth + 2, " Indexes");
     const quickstep::IndexScheme &index_scheme = relation->getIndexScheme();
     for (auto index_it = index_scheme.begin(); index_it != index_scheme.end();
          ++index_it) {
@@ -213,7 +212,7 @@ inline std::vector<TypedValue> ExecuteQueryForSingleRow(
   parser_wrapper->feedNextBuffer(new std::string(query_string));
 
   ParseResult result = parser_wrapper->getNextStatement();
-  DCHECK(result.condition == ParseResult::kSuccess);
+  DCHECK_EQ(ParseResult::kSuccess, result.condition);
 
   const ParseStatement &statement = *result.parsed_statement;
   const CatalogRelation *query_result_relation = nullptr;
@@ -293,21 +292,7 @@ inline TypedValue ExecuteQueryForSingleResult(
   return results[0];
 }
 
-/**
- * @brief A helper function for escaping quotes (i.e. ' or ").
- */
-std::string EscapeQuotes(const std::string &str, const char quote) {
-  std::string ret;
-  for (const char c : str) {
-    ret.push_back(c);
-    if (c == quote) {
-      ret.push_back(c);
-    }
-  }
-  return ret;
-}
-
-void ExecuteAnalyze(const PtrVector<ParseString> *arguments,
+void ExecuteAnalyze(const PtrVector<ParseString> &arguments,
                     const tmb::client_id main_thread_client_id,
                     const tmb::client_id foreman_client_id,
                     MessageBus *bus,
@@ -318,10 +303,10 @@ void ExecuteAnalyze(const PtrVector<ParseString> *arguments,
 
   std::unique_ptr<SqlParserWrapper> parser_wrapper(new SqlParserWrapper());
   std::vector<std::reference_wrapper<const CatalogRelation>> relations;
-  if (arguments->size() == 0) {
+  if (arguments.empty()) {
     relations.insert(relations.begin(), database.begin(), database.end());
   } else {
-    for (const auto &rel_name : *arguments) {
+    for (const auto &rel_name : arguments) {
       const CatalogRelation *rel = database.getRelationByName(rel_name.value());
       if (rel == nullptr) {
         THROW_SQL_ERROR_AT(&rel_name) << "Table does not exist";
@@ -382,7 +367,7 @@ void ExecuteAnalyze(const PtrVector<ParseString> *arguments,
                                    parser_wrapper.get());
 
       auto results_it = results.begin();
-      DCHECK(results_it->getTypeID() == TypeID::kLong);
+      DCHECK_EQ(TypeID::kLong, results_it->getTypeID());
 
       const attribute_id attr_id = attribute.getID();
       mutable_stat->setNumDistinctValues(attr_id,
@@ -411,7 +396,7 @@ void ExecuteAnalyze(const PtrVector<ParseString> *arguments,
                                     query_processor,
                                     parser_wrapper.get());
 
-    DCHECK(num_tuples.getTypeID() == TypeID::kLong);
+    DCHECK_EQ(TypeID::kLong, num_tuples.getTypeID());
     mutable_stat->setNumTuples(num_tuples.getLiteral<std::int64_t>());
 
     mutable_stat->setExactness(true);
@@ -434,17 +419,17 @@ void executeCommand(const ParseStatement &statement,
                     QueryProcessor *query_processor,
                     FILE *out) {
   const ParseCommand &command = static_cast<const ParseCommand &>(statement);
-  const PtrVector<ParseString> *arguments = command.arguments();
+  const PtrVector<ParseString> &arguments = *(command.arguments());
   const std::string &command_str = command.command()->value();
-  if (command_str == C::kDescribeDatabaseCommand) {
-    ExecuteDescribeDatabase(arguments, catalog_database, storage_manager, out);
-  } else if (command_str == C::kDescribeTableCommand) {
-    if (arguments->size() == 0) {
-      ExecuteDescribeDatabase(arguments, catalog_database, storage_manager, out);
+  if (command_str == kDescribeDatabaseCommand) {
+    ExecuteDescribeDatabase(arguments, catalog_database, out);
+  } else if (command_str == kDescribeTableCommand) {
+    if (arguments.empty()) {
+      ExecuteDescribeDatabase(arguments, catalog_database, out);
     } else {
       ExecuteDescribeTable(arguments, catalog_database, out);
     }
-  } else if (command_str == C::kAnalyzeCommand) {
+  } else if (command_str == kAnalyzeCommand) {
     ExecuteAnalyze(arguments,
                    main_thread_client_id,
                    foreman_client_id,
@@ -455,5 +440,6 @@ void executeCommand(const ParseStatement &statement,
     THROW_SQL_ERROR_AT(command.command()) << "Invalid Command";
   }
 }
+
 }  // namespace cli
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4437b9d4/cli/CommandExecutor.hpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.hpp b/cli/CommandExecutor.hpp
index a1d9af9..b214884 100644
--- a/cli/CommandExecutor.hpp
+++ b/cli/CommandExecutor.hpp
@@ -39,15 +39,6 @@ namespace cli {
  *  @{
  */
 
-// Adding the max column width as 6  as the default initializer
-// as the length of the word Column is 6 characters.
-// This is used while describing the table.
-constexpr int kInitMaxColumnWidth = 6;
-
-constexpr char kDescribeDatabaseCommand[] = "\\dt";
-constexpr char kDescribeTableCommand[] = "\\d";
-constexpr char kAnalyzeCommand[] = "\\analyze";
-
 /**
   * @brief Executes the command by calling the command handler.
   *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4437b9d4/cli/Constants.hpp
----------------------------------------------------------------------
diff --git a/cli/Constants.hpp b/cli/Constants.hpp
new file mode 100644
index 0000000..1aaa5be
--- /dev/null
+++ b/cli/Constants.hpp
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 QUICKSTEP_CLI_CONSTANTS_HPP_
+#define QUICKSTEP_CLI_CONSTANTS_HPP_
+
+namespace quickstep {
+namespace cli {
+
+/** \addtogroup CLI
+ *  @{
+ */
+
+// Adding the max column width as 6 as the default initializer
+// as the length of the word Column is 6 characters.
+// This is used while describing the table.
+constexpr int kInitMaxColumnWidth = 6;
+
+constexpr char kDescribeDatabaseCommand[] = "\\dt";
+constexpr char kDescribeTableCommand[] = "\\d";
+constexpr char kAnalyzeCommand[] = "\\analyze";
+
+/** @} */
+
+}  // namespace cli
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_CLI_CONSTANTS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4437b9d4/utility/StringUtil.cpp
----------------------------------------------------------------------
diff --git a/utility/StringUtil.cpp b/utility/StringUtil.cpp
index 49ef337..2745457 100644
--- a/utility/StringUtil.cpp
+++ b/utility/StringUtil.cpp
@@ -93,6 +93,17 @@ std::string EscapeSpecialChars(const std::string& text) {
   return new_text;
 }
 
+std::string EscapeQuotes(const std::string &str, const char quote) {
+  std::string ret;
+  for (const char c : str) {
+    ret.push_back(c);
+    if (c == quote) {
+      ret.push_back(c);
+    }
+  }
+  return ret;
+}
+
 bool ParseIntString(const std::string &int_string,
                     const char delimiter,
                     std::vector<int> *parsed_output) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4437b9d4/utility/StringUtil.hpp
----------------------------------------------------------------------
diff --git a/utility/StringUtil.hpp b/utility/StringUtil.hpp
index 9aa57ab..abda8f3 100644
--- a/utility/StringUtil.hpp
+++ b/utility/StringUtil.hpp
@@ -48,6 +48,12 @@ extern std::string ToLower(const std::string &str);
 extern std::string EscapeSpecialChars(const std::string &text);
 
 /**
+ * @brief Escape quotes (i.e. ' or ") in 'str'.
+ * @return Escaped string.
+ */
+extern std::string EscapeQuotes(const std::string &str, const char quote);
+
+/**
  * @brief Join all objects in a iterable container into a single string. The object
  *        must have implemented the operator<< overloading with std::stringstream.
  *


[2/3] incubator-quickstep git commit: patch to fix gcc compile error gflags

Posted by sp...@apache.org.
patch to fix gcc compile error gflags


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

Branch: refs/heads/fix-gcc-build
Commit: c008b7ad3b2333f7ea2bf4eae1967ce4ef31726d
Parents: 4437b9d
Author: cramja <ma...@gmail.com>
Authored: Tue Feb 28 09:49:57 2017 -0600
Committer: cramja <ma...@gmail.com>
Committed: Tue Feb 28 09:49:57 2017 -0600

----------------------------------------------------------------------
 third_party/download_and_patch_prerequisites.sh      | 1 +
 third_party/patches/gflags/gflags_reporting.cc.patch | 4 ++++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c008b7ad/third_party/download_and_patch_prerequisites.sh
----------------------------------------------------------------------
diff --git a/third_party/download_and_patch_prerequisites.sh b/third_party/download_and_patch_prerequisites.sh
index fd6106c..d16d4de 100755
--- a/third_party/download_and_patch_prerequisites.sh
+++ b/third_party/download_and_patch_prerequisites.sh
@@ -99,6 +99,7 @@ patch ${THIRD_PARTY_SRC_DIR}/linenoise/linenoise.c ${PATCH_DIR}/linenoise/lineno
 echo "Patching for gflags:"
 cd ${THIRD_PARTY_SRC_DIR}/gflags
 patch -p0 < ${PATCH_DIR}/gflags/CMakeLists.patch
+patch src/gflags_reporting.cc ${PATCH_DIR}/gflags/gflags_reporting.cc.patch
 cd ${THIRD_PARTY_SRC_DIR}
 
 # Apply re2 patch.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c008b7ad/third_party/patches/gflags/gflags_reporting.cc.patch
----------------------------------------------------------------------
diff --git a/third_party/patches/gflags/gflags_reporting.cc.patch b/third_party/patches/gflags/gflags_reporting.cc.patch
new file mode 100644
index 0000000..3e33e37
--- /dev/null
+++ b/third_party/patches/gflags/gflags_reporting.cc.patch
@@ -0,0 +1,4 @@
+129c129
+<     assert(chars_left == strlen(c_string));  // Unless there's a \0 in there?
+---
+>     assert(static_cast<size_t>(chars_left) == strlen(c_string));  // Unless there's a \0 in there?


[3/3] incubator-quickstep git commit: Fix PackedPayloadHashTable for gcc build

Posted by sp...@apache.org.
Fix PackedPayloadHashTable for gcc build


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

Branch: refs/heads/fix-gcc-build
Commit: 132fed63d14a26917a4f3aacc2337c17b260f708
Parents: c008b7a
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Feb 27 14:45:50 2017 -0600
Committer: cramja <ma...@gmail.com>
Committed: Tue Feb 28 10:13:53 2017 -0600

----------------------------------------------------------------------
 storage/PackedPayloadHashTable.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/132fed63/storage/PackedPayloadHashTable.cpp
----------------------------------------------------------------------
diff --git a/storage/PackedPayloadHashTable.cpp b/storage/PackedPayloadHashTable.cpp
index 3d672f2..8c4a9fc 100644
--- a/storage/PackedPayloadHashTable.cpp
+++ b/storage/PackedPayloadHashTable.cpp
@@ -251,7 +251,7 @@ bool PackedPayloadHashTable::upsertValueAccessorCompositeKey(
       [&](auto use_two_accessors,  // NOLINT(build/c++11)
           auto key_only,
           auto has_variable_size) -> bool {
-    return upsertValueAccessorCompositeKeyInternal<
+    return this->upsertValueAccessorCompositeKeyInternal<
         decltype(use_two_accessors)::value,
         decltype(key_only)::value,
         decltype(has_variable_size)::value>(