http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7a464434/query_optimizer/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/CMakeLists.txt 
b/query_optimizer/tests/CMakeLists.txt
index 597dbe0..ac4548a 100644
--- a/query_optimizer/tests/CMakeLists.txt
+++ b/query_optimizer/tests/CMakeLists.txt
@@ -94,22 +94,6 @@ 
add_executable(quickstep_queryoptimizer_tests_ExecutionGeneratorTest
                ExecutionGeneratorTestRunner.hpp
                "${PROJECT_SOURCE_DIR}/utility/textbased_test/TextBasedTest.cpp"
                
"${PROJECT_SOURCE_DIR}/utility/textbased_test/TextBasedTest.hpp")
-add_executable(ExecutionHeuristics_unittest ExecutionHeuristics_unittest.cpp)
-target_link_libraries(ExecutionHeuristics_unittest
-                      gtest
-                      gtest_main
-                      quickstep_catalog_Catalog
-                      quickstep_catalog_CatalogDatabase
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_queryexecution_QueryContext
-                      quickstep_queryexecution_QueryContext_proto
-                      quickstep_queryoptimizer_ExecutionHeuristics
-                      quickstep_queryoptimizer_QueryPlan
-                      quickstep_relationaloperators_BuildHashOperator
-                      quickstep_relationaloperators_HashJoinOperator
-                      quickstep_utility_Macros)
-add_test(ExecutionHeuristics_unittest ExecutionHeuristics_unittest)
-
 add_executable(quickstep_queryoptimizer_tests_OptimizerTextTest
                OptimizerTextTest.cpp
                OptimizerTextTestRunner.cpp

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7a464434/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp 
b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
deleted file mode 100644
index 73b3e84..0000000
--- a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
+++ /dev/null
@@ -1,311 +0,0 @@
-/**
- * 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.
- **/
-
-#include <cstddef>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "catalog/Catalog.hpp"
-#include "catalog/CatalogDatabase.hpp"
-#include "catalog/CatalogTypedefs.hpp"
-#include "query_execution/QueryContext.hpp"
-#include "query_execution/QueryContext.pb.h"
-#include "query_optimizer/ExecutionHeuristics.hpp"
-#include "query_optimizer/QueryPlan.hpp"
-#include "relational_operators/BuildHashOperator.hpp"
-#include "relational_operators/HashJoinOperator.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-#include "gtest/gtest.h"
-
-namespace quickstep {
-namespace optimizer {
-
-namespace {
-constexpr std::size_t kQueryId = 0;
-}
-
-class ExecutionHeuristicsTest : public ::testing::Test {
- protected:
-  virtual void SetUp() {
-    db_ = cat_.getDatabaseByIdMutable(cat_.addDatabase(new 
CatalogDatabase(nullptr, "db")));
-    execution_heuristics_.reset(new ExecutionHeuristics());
-    query_plan_.reset(new QueryPlan());
-    query_context_proto_.reset(new serialization::QueryContext());
-  }
-
-  CatalogRelation* createCatalogRelation(const std::string &name, bool 
temporary = false) {
-    return db_->getRelationByIdMutable(db_->addRelation(new 
CatalogRelation(nullptr, name, -1, temporary)));
-  }
-
-  void addDummyHashJoinInfo(ExecutionHeuristics *execution_heuristics,
-                            const QueryPlan::DAGNodeIndex build_operator_index,
-                            const QueryPlan::DAGNodeIndex join_operator_index,
-                            const CatalogRelation *build_relation,
-                            const CatalogRelation *probe_relation,
-                            const attribute_id build_attribute_id,
-                            const attribute_id probe_attribute_id,
-                            const QueryContext::join_hash_table_id 
join_hash_table_id) {
-    std::vector<attribute_id> build_attribute_ids(1, build_attribute_id);
-    std::vector<attribute_id> probe_attribute_ids(1, probe_attribute_id);
-    execution_heuristics->addHashJoinInfo(build_operator_index,
-                                          join_operator_index,
-                                          build_relation,
-                                          probe_relation,
-                                          std::move(build_attribute_ids),
-                                          std::move(probe_attribute_ids),
-                                          join_hash_table_id);
-  }
-
-  QueryPlan::DAGNodeIndex createDummyBuildHashOperator(QueryPlan *query_plan,
-                                                       const CatalogRelation 
*build_relation,
-                                                       const attribute_id 
build_attribute_id,
-                                                       const 
QueryContext::join_hash_table_id join_hash_table_index) {
-    std::vector<attribute_id> build_attribute_ids;
-    build_attribute_ids.push_back(build_attribute_id);
-    QueryPlan::DAGNodeIndex build_operator_index =
-        query_plan->addRelationalOperator(new BuildHashOperator(kQueryId,
-                                                                
*build_relation,
-                                                                true,
-                                                                
build_attribute_ids,
-                                                                false,
-                                                                
join_hash_table_index));
-    return build_operator_index;
-  }
-
-  QueryPlan::DAGNodeIndex createDummyHashJoinOperator(QueryPlan *query_plan,
-                                                      const CatalogRelation 
*build_relation,
-                                                      const CatalogRelation 
*probe_relation,
-                                                      const attribute_id 
probe_attribute_id,
-                                                      const 
QueryContext::join_hash_table_id join_hash_table_index) {
-    std::vector<attribute_id> probe_attribute_ids;
-    probe_attribute_ids.push_back(probe_attribute_id);
-    QueryPlan::DAGNodeIndex join_operator_index =
-        query_plan->addRelationalOperator(
-            new HashJoinOperator(kQueryId,
-                                 *build_relation,
-                                 *probe_relation,
-                                 true,
-                                 probe_attribute_ids,
-                                 false,
-                                 *probe_relation,
-                                 0,
-                                 join_hash_table_index,
-                                 0,
-                                 0));
-    return join_operator_index;
-  }
-
-  Catalog cat_;
-  CatalogDatabase *db_;  // db_ is owned by cat_.
-  std::unique_ptr<QueryPlan> query_plan_;
-  std::unique_ptr<serialization::QueryContext> query_context_proto_;
-  std::unique_ptr<ExecutionHeuristics> execution_heuristics_;
-};
-
-TEST_F(ExecutionHeuristicsTest, HashJoinOptimizedTest) {
-  // This test case creates three hash joins, all of which are being probed on 
the same relation.
-  // Since the probe are being made on the same relation, ExecutionHeuristics 
should optimize
-  // these hash joins using bloom filters.
-
-  const CatalogRelation *build_relation_1 = 
createCatalogRelation("build_relation_1");
-  const CatalogRelation *build_relation_2 = 
createCatalogRelation("build_relation_2");
-  const CatalogRelation *build_relation_3 = 
createCatalogRelation("build_relation_3");
-  const CatalogRelation *probe_relation_1 = 
createCatalogRelation("probe_relation_1");
-
-  const attribute_id build_attribute_id_1 = 0;
-  const attribute_id build_attribute_id_2 = 0;
-  const attribute_id build_attribute_id_3 = 0;
-  const attribute_id probe_attribute_id_1 = 1;
-  const attribute_id probe_attribute_id_2 = 2;
-  const attribute_id probe_attribute_id_3 = 3;
-
-  const QueryContext::join_hash_table_id join_hash_table_index_1 = 0;
-  const QueryContext::join_hash_table_id join_hash_table_index_2 = 1;
-  const QueryContext::join_hash_table_id join_hash_table_index_3 = 2;
-  query_context_proto_->add_join_hash_tables();
-  query_context_proto_->add_join_hash_tables();
-  query_context_proto_->add_join_hash_tables();
-
-  const QueryPlan::DAGNodeIndex build_operator_index_1 = 
createDummyBuildHashOperator(query_plan_.get(),
-                                                                               
       build_relation_1,
-                                                                               
       build_attribute_id_1,
-                                                                               
       join_hash_table_index_1);
-  const QueryPlan::DAGNodeIndex probe_operator_index_1 = 
createDummyHashJoinOperator(query_plan_.get(),
-                                                                               
      build_relation_1,
-                                                                               
      probe_relation_1,
-                                                                               
      probe_attribute_id_1,
-                                                                               
      join_hash_table_index_1);
-  const QueryPlan::DAGNodeIndex build_operator_index_2 = 
createDummyBuildHashOperator(query_plan_.get(),
-                                                                               
       build_relation_2,
-                                                                               
       build_attribute_id_2,
-                                                                               
       join_hash_table_index_2);
-  const QueryPlan::DAGNodeIndex probe_operator_index_2 = 
createDummyHashJoinOperator(query_plan_.get(),
-                                                                               
      build_relation_2,
-                                                                               
      probe_relation_1,
-                                                                               
      probe_attribute_id_2,
-                                                                               
      join_hash_table_index_2);
-  const QueryPlan::DAGNodeIndex build_operator_index_3 = 
createDummyBuildHashOperator(query_plan_.get(),
-                                                                               
       build_relation_3,
-                                                                               
       build_attribute_id_3,
-                                                                               
       join_hash_table_index_3);
-  const QueryPlan::DAGNodeIndex probe_operator_index_3 = 
createDummyHashJoinOperator(query_plan_.get(),
-                                                                               
      build_relation_3,
-                                                                               
      probe_relation_1,
-                                                                               
      probe_attribute_id_3,
-                                                                               
      join_hash_table_index_3);
-
-  addDummyHashJoinInfo(execution_heuristics_.get(),
-                       build_operator_index_1,
-                       probe_operator_index_1,
-                       build_relation_1,
-                       probe_relation_1,
-                       build_attribute_id_1,
-                       probe_attribute_id_1,
-                       join_hash_table_index_1);
-  addDummyHashJoinInfo(execution_heuristics_.get(),
-                       build_operator_index_2,
-                       probe_operator_index_2,
-                       build_relation_2,
-                       probe_relation_1,
-                       build_attribute_id_2,
-                       probe_attribute_id_2,
-                       join_hash_table_index_2);
-  addDummyHashJoinInfo(execution_heuristics_.get(),
-                       build_operator_index_3,
-                       probe_operator_index_3,
-                       build_relation_3,
-                       probe_relation_1,
-                       build_attribute_id_3,
-                       probe_attribute_id_3,
-                       join_hash_table_index_3);
-
-  execution_heuristics_->optimizeExecutionPlan(query_plan_.get(), 
query_context_proto_.get());
-
-  // Test whether correct number of bloom filters were added.
-  EXPECT_EQ(1, 
query_context_proto_->join_hash_tables(0).build_side_bloom_filter_id_size());
-  EXPECT_EQ(1, 
query_context_proto_->join_hash_tables(1).build_side_bloom_filter_id_size());
-  EXPECT_EQ(1, 
query_context_proto_->join_hash_tables(2).build_side_bloom_filter_id_size());
-  EXPECT_EQ(3, 
query_context_proto_->join_hash_tables(0).probe_side_bloom_filters_size());
-
-  // Test that the DAG was modified correctly or not.
-  // Probe operator 1 should have now build operator 1 and build operator 2 
added as dependencies.
-  auto const probe_node_dependencies = 
query_plan_->getQueryPlanDAG().getDependencies(probe_operator_index_1);
-  EXPECT_EQ(1u, probe_node_dependencies.count(build_operator_index_2));
-  EXPECT_EQ(1u, probe_node_dependencies.count(build_operator_index_3));
-}
-
-TEST_F(ExecutionHeuristicsTest, HashJoinNotOptimizedTest) {
-  // This test case creates three hash joins, all of which are being probed on 
different relations.
-  // Since the probe are being made on the different relations, 
ExecutionHeuristics should optimize
-  // these hash joins using bloom filters.
-
-  const CatalogRelation *build_relation_1 = 
createCatalogRelation("build_relation_1");
-  const CatalogRelation *build_relation_2 = 
createCatalogRelation("build_relation_2");
-  const CatalogRelation *build_relation_3 = 
createCatalogRelation("build_relation_3");
-  const CatalogRelation *probe_relation_1 = 
createCatalogRelation("probe_relation_1");
-  const CatalogRelation *probe_relation_2 = 
createCatalogRelation("probe_relation_2");
-  const CatalogRelation *probe_relation_3 = 
createCatalogRelation("probe_relation_3");
-
-  const attribute_id build_attribute_id_1 = 0;
-  const attribute_id build_attribute_id_2 = 0;
-  const attribute_id build_attribute_id_3 = 0;
-  const attribute_id probe_attribute_id_1 = 1;
-  const attribute_id probe_attribute_id_2 = 2;
-  const attribute_id probe_attribute_id_3 = 3;
-
-  const QueryContext::join_hash_table_id join_hash_table_index_1 = 0;
-  const QueryContext::join_hash_table_id join_hash_table_index_2 = 1;
-  const QueryContext::join_hash_table_id join_hash_table_index_3 = 2;
-  query_context_proto_->add_join_hash_tables();
-  query_context_proto_->add_join_hash_tables();
-  query_context_proto_->add_join_hash_tables();
-
-  const QueryPlan::DAGNodeIndex build_operator_index_1 = 
createDummyBuildHashOperator(query_plan_.get(),
-                                                                               
       build_relation_1,
-                                                                               
       build_attribute_id_1,
-                                                                               
       join_hash_table_index_1);
-  const QueryPlan::DAGNodeIndex probe_operator_index_1 = 
createDummyHashJoinOperator(query_plan_.get(),
-                                                                               
      build_relation_1,
-                                                                               
      probe_relation_1,
-                                                                               
      probe_attribute_id_1,
-                                                                               
      join_hash_table_index_1);
-  const QueryPlan::DAGNodeIndex build_operator_index_2 = 
createDummyBuildHashOperator(query_plan_.get(),
-                                                                               
       build_relation_2,
-                                                                               
       build_attribute_id_2,
-                                                                               
       join_hash_table_index_2);
-  const QueryPlan::DAGNodeIndex probe_operator_index_2 = 
createDummyHashJoinOperator(query_plan_.get(),
-                                                                               
      build_relation_2,
-                                                                               
      probe_relation_2,
-                                                                               
      probe_attribute_id_2,
-                                                                               
      join_hash_table_index_2);
-  const QueryPlan::DAGNodeIndex build_operator_index_3 = 
createDummyBuildHashOperator(query_plan_.get(),
-                                                                               
       build_relation_3,
-                                                                               
       build_attribute_id_3,
-                                                                               
       join_hash_table_index_3);
-  const QueryPlan::DAGNodeIndex probe_operator_index_3 = 
createDummyHashJoinOperator(query_plan_.get(),
-                                                                               
      build_relation_3,
-                                                                               
      probe_relation_3,
-                                                                               
      probe_attribute_id_3,
-                                                                               
      join_hash_table_index_3);
-
-  addDummyHashJoinInfo(execution_heuristics_.get(),
-                       build_operator_index_1,
-                       probe_operator_index_1,
-                       build_relation_1,
-                       probe_relation_1,
-                       build_attribute_id_1,
-                       probe_attribute_id_1,
-                       join_hash_table_index_1);
-  addDummyHashJoinInfo(execution_heuristics_.get(),
-                       build_operator_index_2,
-                       probe_operator_index_2,
-                       build_relation_2,
-                       probe_relation_2,
-                       build_attribute_id_2,
-                       probe_attribute_id_2,
-                       join_hash_table_index_2);
-  addDummyHashJoinInfo(execution_heuristics_.get(),
-                       build_operator_index_3,
-                       probe_operator_index_3,
-                       build_relation_3,
-                       probe_relation_3,
-                       build_attribute_id_3,
-                       probe_attribute_id_3,
-                       join_hash_table_index_3);
-
-  execution_heuristics_->optimizeExecutionPlan(query_plan_.get(), 
query_context_proto_.get());
-
-  // Test that no bloom filters were added.
-  EXPECT_EQ(0, 
query_context_proto_->join_hash_tables(0).build_side_bloom_filter_id_size());
-  EXPECT_EQ(0, 
query_context_proto_->join_hash_tables(1).build_side_bloom_filter_id_size());
-  EXPECT_EQ(0, 
query_context_proto_->join_hash_tables(2).build_side_bloom_filter_id_size());
-  EXPECT_EQ(0, 
query_context_proto_->join_hash_tables(0).probe_side_bloom_filters_size());
-
-  // Test that the DAG was not modified at all.
-  // Probe operator 1 should not have build operator 1 and build operator 2 
added as dependencies.
-  auto probe_node_dependencies = 
query_plan_->getQueryPlanDAG().getDependencies(probe_operator_index_1);
-  EXPECT_EQ(0u, probe_node_dependencies.count(build_operator_index_2));
-  EXPECT_EQ(0u, probe_node_dependencies.count(build_operator_index_3));
-}
-
-}  // namespace optimizer
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7a464434/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index ddaae45..395e264 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -156,6 +156,8 @@ 
QS_PROTOBUF_GENERATE_CPP(quickstep_utility_SortConfiguration_proto_srcs
                          quickstep_utility_SortConfiguration_proto_hdrs
                          SortConfiguration.proto)
 
+add_subdirectory(lip_filter)
+
 # Declare micro-libs:
 add_library(quickstep_utility_Alignment ../empty_src.cpp Alignment.hpp)
 add_library(quickstep_utility_BitManipulation ../empty_src.cpp 
BitManipulation.hpp)
@@ -168,6 +170,7 @@ add_library(quickstep_utility_CalculateInstalledMemory 
CalculateInstalledMemory.
 add_library(quickstep_utility_Cast ../empty_src.cpp Cast.hpp)
 add_library(quickstep_utility_CheckSnprintf ../empty_src.cpp CheckSnprintf.hpp)
 add_library(quickstep_utility_DAG ../empty_src.cpp DAG.hpp)
+add_library(quickstep_utility_DisjointTreeForest ../empty_src.cpp 
DisjointTreeForest.hpp)
 add_library(quickstep_utility_EqualsAnyConstant ../empty_src.cpp 
EqualsAnyConstant.hpp)
 add_library(quickstep_utility_ExecutionDAGVisualizer
             ExecutionDAGVisualizer.cpp
@@ -230,6 +233,8 @@ target_link_libraries(quickstep_utility_CheckSnprintf
 target_link_libraries(quickstep_utility_DAG
                       glog
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_DisjointTreeForest
+                      glog)
 target_link_libraries(quickstep_utility_ExecutionDAGVisualizer
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_queryexecution_QueryExecutionTypedefs
@@ -253,7 +258,9 @@ target_link_libraries(quickstep_utility_PlanVisualizer
                       quickstep_catalog_CatalogRelation
                       
quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
                       quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExprId
                       quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_LIPFilterConfiguration
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_physical_PhysicalType
                       quickstep_queryoptimizer_physical_TableReference
@@ -319,6 +326,7 @@ target_link_libraries(quickstep_utility
                       quickstep_utility_Cast
                       quickstep_utility_CheckSnprintf
                       quickstep_utility_DAG
+                      quickstep_utility_DisjointTreeForest
                       quickstep_utility_EqualsAnyConstant
                       quickstep_utility_ExecutionDAGVisualizer
                       quickstep_utility_Glob
@@ -375,6 +383,13 @@ target_link_libraries(DAG_unittest
                       ${LIBS})
 add_test(DAG_unittest DAG_unittest)
 
+add_executable(DisjointTreeForest_unittest 
"${CMAKE_CURRENT_SOURCE_DIR}/tests/DisjointTreeForest_unittest.cpp")
+target_link_libraries(DisjointTreeForest_unittest
+                      gtest
+                      gtest_main
+                      quickstep_utility_DisjointTreeForest)
+add_test(DisjointTreeForest_unittest DisjointTreeForest_unittest)
+
 add_executable(EqualsAnyConstant_unittest 
"${CMAKE_CURRENT_SOURCE_DIR}/tests/EqualsAnyConstant_unittest.cpp")
 target_link_libraries(EqualsAnyConstant_unittest
                       gtest

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7a464434/utility/DisjointTreeForest.hpp
----------------------------------------------------------------------
diff --git a/utility/DisjointTreeForest.hpp b/utility/DisjointTreeForest.hpp
new file mode 100644
index 0000000..971ba10
--- /dev/null
+++ b/utility/DisjointTreeForest.hpp
@@ -0,0 +1,152 @@
+/**
+ * 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_UTILITY_DISJOINT_TREE_FOREST_HPP_
+#define QUICKSTEP_UTILITY_DISJOINT_TREE_FOREST_HPP_
+
+#include <algorithm>
+#include <cstddef>
+#include <unordered_map>
+#include <vector>
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+/**
+ * @brief Disjoint sets implemented with tree data structures so that the
+ *        union/find operations have nearly O(1) time complexity.
+ */
+template <typename ElementT,
+          class MapperT = std::unordered_map<ElementT, std::size_t>>
+class DisjointTreeForest {
+ public:
+  /**
+   * @brief Whether the given element is in a subset.
+   *
+   * @param element The element.
+   * @return True if the element is in a subset.
+   */
+  bool hasElement(const ElementT &element) const {
+    return elements_map_.find(element) != elements_map_.end();
+  }
+
+  /**
+   * @brief If the given element is not in any subset yet, make a singleton
+   *        subset for it. Otherwise do nothing.
+   *
+   * @param element The element.
+   */
+  void makeSet(const ElementT &element) {
+    if (!hasElement(element)) {
+      std::size_t loc = nodes_.size();
+      nodes_.emplace_back(0, loc);
+      elements_map_.emplace(element, loc);
+    }
+  }
+
+  /**
+   * @brief Find the subset id for the given element.
+   *
+   * @param element The element.
+   */
+  std::size_t find(const ElementT &element) {
+    DCHECK(hasElement(element));
+
+    const std::size_t node_id = elements_map_.at(element);
+    std::size_t root_id = node_id;
+    std::size_t parent_id;
+    while ((parent_id = nodes_[root_id].parent) != root_id) {
+      root_id = parent_id;
+    }
+    compress_path(node_id, root_id);
+    return root_id;
+  }
+
+  /**
+   * @brief Union the two subsets that the two given elements belong to.
+   *
+   * @param element1 The first element.
+   * @param element2 The second element.
+   */
+  void merge(const ElementT &element1, const ElementT &element2) {
+    std::size_t root_id1 = find(element1);
+    std::size_t root_id2 = find(element2);
+    if (root_id1 != root_id2) {
+      Node &n1 = nodes_[root_id1];
+      Node &n2 = nodes_[root_id2];
+      if (n1.rank > n2.rank) {
+        n2.parent = root_id1;
+      } else if (n1.rank < n2.rank) {
+        n1.parent = root_id2;
+      } else {
+        n1.parent = root_id2;
+        n2.rank += 1;
+      }
+    }
+  }
+
+  /**
+   * @brief Whether the two given elements are in the same subset.
+   *
+   * @param element1 The first element.
+   * @param element2 The second element.
+   * @return True if the twos elements are in the same subset, false otherwise.
+   */
+  bool isConnected(const ElementT &element1, const ElementT &element2) {
+    return find(element1) == find(element2);
+  }
+
+ private:
+  struct Node {
+    Node(const std::size_t rank_in, const std::size_t parent_in)
+        : rank(rank_in), parent(parent_in) {
+    }
+    std::size_t rank;
+    std::size_t parent;
+  };
+
+  inline void compress_path(const std::size_t leaf_node_id,
+                            const std::size_t root_node_id) {
+    std::size_t node_id = leaf_node_id;
+    std::size_t max_rank = 0;
+    while (node_id != root_node_id) {
+      const Node &node = nodes_[node_id];
+      max_rank = std::max(max_rank, node.rank);
+
+      const std::size_t parent_id = node.parent;
+      nodes_[node_id].parent = root_node_id;
+      node_id = parent_id;
+    }
+    nodes_[root_node_id].rank = max_rank + 1;
+  }
+
+  std::vector<Node> nodes_;
+  MapperT elements_map_;
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_DISJOINT_TREE_FOREST_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7a464434/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index 50cf7f0..2adf674 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -21,15 +21,16 @@
 
 #include <cstddef>
 #include <memory>
+#include <set>
 #include <sstream>
 #include <string>
 #include <unordered_map>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
-
 #include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/PhysicalType.hpp"
@@ -47,9 +48,12 @@ namespace C = ::quickstep::optimizer::cost;
 
 std::string PlanVisualizer::visualize(const P::PhysicalPtr &input) {
   DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+  const P::TopLevelPlanPtr top_level_plan =
+      std::static_pointer_cast<const P::TopLevelPlan>(input);
   cost_model_.reset(
       new C::StarSchemaSimpleCostModel(
-          std::static_pointer_cast<const 
P::TopLevelPlan>(input)->shared_subplans()));
+          top_level_plan->shared_subplans()));
+  lip_filter_conf_ = top_level_plan->lip_filter_configuration();
 
   color_map_["TableReference"] = "skyblue";
   color_map_["Selection"] = "#90EE90";
@@ -86,6 +90,9 @@ std::string PlanVisualizer::visualize(const P::PhysicalPtr 
&input) {
   for (const EdgeInfo &edge_info : edges_) {
     graph_oss << "  " << edge_info.src_node_id << " -> "
               << edge_info.dst_node_id << " [";
+    if (edge_info.dashed) {
+      graph_oss << "style=dashed ";
+    }
     if (!edge_info.labels.empty()) {
       graph_oss << "label=\""
                 << EscapeSpecialChars(JoinToString(edge_info.labels, "&#10;"))
@@ -103,6 +110,10 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
   int node_id = ++id_counter_;
   node_id_map_.emplace(input, node_id);
 
+  std::set<E::ExprId> referenced_ids;
+  for (const auto &attr : input->getReferencedAttributes()) {
+    referenced_ids.emplace(attr->id());
+  }
   for (const auto &child : input->children()) {
     visit(child);
 
@@ -112,12 +123,18 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
     EdgeInfo &edge_info = edges_.back();
     edge_info.src_node_id = child_id;
     edge_info.dst_node_id = node_id;
+    edge_info.dashed = false;
 
-    // Print output attributes except for TableReference -- there are just too 
many
-    // attributes out of TableReference.
-    if (child->getPhysicalType() != P::PhysicalType::kTableReference) {
-      for (const auto &attr : child->getOutputAttributes()) {
-        edge_info.labels.emplace_back(attr->attribute_alias());
+    if (input->getPhysicalType() == P::PhysicalType::kHashJoin &&
+        child == input->children()[1]) {
+      edge_info.dashed = true;
+    }
+
+    for (const auto &attr : child->getOutputAttributes()) {
+      if (referenced_ids.find(attr->id()) != referenced_ids.end()) {
+        edge_info.labels.emplace_back(
+            attr->attribute_alias() + ", est # distinct = " +
+            std::to_string(cost_model_->estimateNumDistinctValues(attr->id(), 
child)));
       }
     }
   }
@@ -154,6 +171,26 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
       break;
     }
   }
+
+  if (lip_filter_conf_ != nullptr) {
+    const auto &build_filters = lip_filter_conf_->getBuildInfoMap();
+    const auto build_it = build_filters.find(input);
+    if (build_it != build_filters.end()) {
+      for (const auto &build_info : build_it->second) {
+        node_info.labels.emplace_back(
+            std::string("[LIP build] ") + 
build_info.build_attribute->attribute_alias());
+      }
+    }
+    const auto &probe_filters = lip_filter_conf_->getProbeInfoMap();
+    const auto probe_it = probe_filters.find(input);
+    if (probe_it != probe_filters.end()) {
+      for (const auto &probe_info : probe_it->second) {
+        node_info.labels.emplace_back(
+            std::string("[LIP probe] ") + 
probe_info.probe_attribute->attribute_alias());
+      }
+    }
+  }
+
   node_info.labels.emplace_back(
       "est. # = " + std::to_string(cost_model_->estimateCardinality(input)));
   node_info.labels.emplace_back(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7a464434/utility/PlanVisualizer.hpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.hpp b/utility/PlanVisualizer.hpp
index 1c0df77..9b8b0db 100644
--- a/utility/PlanVisualizer.hpp
+++ b/utility/PlanVisualizer.hpp
@@ -26,6 +26,7 @@
 #include <vector>
 
 #include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/physical/LIPFilterConfiguration.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "utility/Macros.hpp"
 
@@ -73,6 +74,7 @@ class PlanVisualizer {
     int src_node_id;
     int dst_node_id;
     std::vector<std::string> labels;
+    bool dashed;
   };
 
   void visit(const optimizer::physical::PhysicalPtr &input);
@@ -85,6 +87,7 @@ class PlanVisualizer {
   std::vector<EdgeInfo> edges_;
 
   std::unique_ptr<optimizer::cost::StarSchemaSimpleCostModel> cost_model_;
+  optimizer::physical::LIPFilterConfigurationPtr lip_filter_conf_;
 
   DISALLOW_COPY_AND_ASSIGN(PlanVisualizer);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7a464434/utility/lip_filter/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/lip_filter/CMakeLists.txt 
b/utility/lip_filter/CMakeLists.txt
new file mode 100644
index 0000000..2232abe
--- /dev/null
+++ b/utility/lip_filter/CMakeLists.txt
@@ -0,0 +1,19 @@
+# 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.
+
+# Declare micro-libs:
+add_library(quickstep_utility_lipfilter_LIPFilter ../../empty_src.cpp 
LIPFilter.hpp)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7a464434/utility/lip_filter/LIPFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilter.hpp b/utility/lip_filter/LIPFilter.hpp
new file mode 100644
index 0000000..33165ed
--- /dev/null
+++ b/utility/lip_filter/LIPFilter.hpp
@@ -0,0 +1,39 @@
+/**
+ * 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_UTILITY_LIP_FILTER_LIP_FILTER_HPP_
+#define QUICKSTEP_UTILITY_LIP_FILTER_LIP_FILTER_HPP_
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+enum class LIPFilterType {
+  kBloomFilter,
+  kExactFilter,
+  kSingleIdentityHashFilter
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_LIP_FILTER_LIP_FILTER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7a464434/utility/tests/DisjointTreeForest_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/DisjointTreeForest_unittest.cpp 
b/utility/tests/DisjointTreeForest_unittest.cpp
new file mode 100644
index 0000000..2e12fad
--- /dev/null
+++ b/utility/tests/DisjointTreeForest_unittest.cpp
@@ -0,0 +1,98 @@
+/**
+ * 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.
+ **/
+
+#include "utility/DisjointTreeForest.hpp"
+
+#include <cstddef>
+#include <string>
+#include <vector>
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+
+TEST(DisjointTreeForestTest, IntTest) {
+  DisjointTreeForest<int> forest;
+  for (int i = 10; i < 20; ++i) {
+    forest.makeSet(i);
+  }
+
+  for (int i = 10; i < 20; i += 2) {
+    EXPECT_NE(forest.find(i), forest.find(i+1));
+    EXPECT_FALSE(forest.isConnected(i, i+1));
+
+    forest.merge(i, i+1);
+    EXPECT_EQ(forest.find(i), forest.find(i+1));
+    EXPECT_TRUE(forest.isConnected(i, i+1));
+
+    forest.merge(i+1, i);
+    EXPECT_EQ(forest.find(i), forest.find(i+1));
+    EXPECT_TRUE(forest.isConnected(i, i+1));
+  }
+
+  for (int i = 12; i < 20; i += 2) {
+    EXPECT_NE(forest.find(i), forest.find(i-1));
+    EXPECT_FALSE(forest.isConnected(i, i-1));
+  }
+
+  forest.merge(10, 17);
+  forest.merge(11, 18);
+  EXPECT_EQ(forest.find(11), forest.find(16));
+  EXPECT_EQ(forest.find(10), forest.find(19));
+  EXPECT_NE(forest.find(10), forest.find(12));
+  EXPECT_NE(forest.find(15), forest.find(17));
+
+  forest.merge(12, 14);
+  forest.merge(15, 16);
+  const std::size_t id = forest.find(10);
+  for (int i = 10; i < 20; ++i) {
+    EXPECT_EQ(forest.find(i), id);
+  }
+}
+
+TEST(DisjointTreeForestTest, StringTest) {
+  DisjointTreeForest<std::string> forest;
+  const std::vector<std::string> elements = { "aaa", "bbb", "ccc", "ddd" };
+  for (const std::string &element : elements) {
+    forest.makeSet(element);
+  }
+
+  EXPECT_NE(forest.find("aaa"), forest.find("bbb"));
+  forest.merge("aaa", "bbb");
+  EXPECT_EQ(forest.find("aaa"), forest.find("bbb"));
+
+  EXPECT_NE(forest.find("ccc"), forest.find("ddd"));
+  forest.merge("ccc", "ddd");
+  EXPECT_EQ(forest.find("ccc"), forest.find("ddd"));
+
+  EXPECT_NE(forest.find("aaa"), forest.find("ccc"));
+  EXPECT_NE(forest.find("aaa"), forest.find("ddd"));
+  EXPECT_NE(forest.find("bbb"), forest.find("ccc"));
+  EXPECT_NE(forest.find("bbb"), forest.find("ddd"));
+
+  forest.merge("aaa", "ddd");
+  for (const std::string &e1 : elements) {
+    for (const std::string &e2 : elements) {
+      EXPECT_EQ(forest.find(e1), forest.find(e2));
+      EXPECT_TRUE(forest.isConnected(e1, e2));
+    }
+  }
+}
+
+}  // namespace quickstep


Reply via email to