You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by ji...@apache.org on 2016/10/28 19:34:19 UTC

[06/11] incubator-quickstep git commit: Add backend support for LIPFilters.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/393eba55/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index ec5990f..dd3e19d 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -340,7 +340,7 @@ void StorageBlock::sample(const bool is_block_sample,
 }
 
 void StorageBlock::select(const vector<unique_ptr<const Scalar>> &selection,
-                          const Predicate *predicate,
+                          const TupleIdSequence *filter,
                           InsertDestinationInterface *destination) const {
   ColumnVectorsValueAccessor temp_result;
   {
@@ -348,14 +348,8 @@ void StorageBlock::select(const vector<unique_ptr<const Scalar>> &selection,
                                       indices_,
                                       indices_consistent_);
 
-    std::unique_ptr<TupleIdSequence> matches;
-    std::unique_ptr<ValueAccessor> accessor;
-    if (predicate == nullptr) {
-      accessor.reset(tuple_store_->createValueAccessor());
-    } else {
-      matches.reset(getMatchesForPredicate(predicate));
-      accessor.reset(tuple_store_->createValueAccessor(matches.get()));
-    }
+    std::unique_ptr<ValueAccessor> accessor(
+        tuple_store_->createValueAccessor(filter));
 
     for (vector<unique_ptr<const Scalar>>::const_iterator selection_cit = selection.begin();
          selection_cit != selection.end();
@@ -370,16 +364,10 @@ void StorageBlock::select(const vector<unique_ptr<const Scalar>> &selection,
 }
 
 void StorageBlock::selectSimple(const std::vector<attribute_id> &selection,
-                                const Predicate *predicate,
+                                const TupleIdSequence *filter,
                                 InsertDestinationInterface *destination) const {
-  std::unique_ptr<ValueAccessor> accessor;
-  std::unique_ptr<TupleIdSequence> matches;
-  if (predicate == nullptr) {
-    accessor.reset(tuple_store_->createValueAccessor());
-  } else {
-    matches.reset(getMatchesForPredicate(predicate));
-    accessor.reset(tuple_store_->createValueAccessor(matches.get()));
-  }
+  std::unique_ptr<ValueAccessor> accessor(
+      tuple_store_->createValueAccessor(filter));
 
   destination->bulkInsertTuplesWithRemappedAttributes(selection,
                                                       accessor.get());
@@ -389,37 +377,28 @@ AggregationState* StorageBlock::aggregate(
     const AggregationHandle &handle,
     const std::vector<std::unique_ptr<const Scalar>> &arguments,
     const std::vector<attribute_id> *arguments_as_attributes,
-    const Predicate *predicate,
-    std::unique_ptr<TupleIdSequence> *reuse_matches) const {
-  // If there is a filter predicate that hasn't already been evaluated,
-  // evaluate it now and save the results for other aggregates on this same
-  // block.
-  if (predicate && !*reuse_matches) {
-    reuse_matches->reset(getMatchesForPredicate(predicate));
-  }
-
+    const TupleIdSequence *filter) const {
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   // If all the arguments to this aggregate are plain relation attributes,
   // aggregate directly on a ValueAccessor from this block to avoid a copy.
   if ((arguments_as_attributes != nullptr) && (!arguments_as_attributes->empty())) {
     DCHECK_EQ(arguments.size(), arguments_as_attributes->size())
         << "Mismatch between number of arguments and number of attribute_ids";
-    return aggregateHelperValueAccessor(handle, *arguments_as_attributes, reuse_matches->get());
+    return aggregateHelperValueAccessor(handle, *arguments_as_attributes, filter);
   }
   // TODO(shoban): We may want to optimize for ScalarLiteral here.
 #endif
 
   // Call aggregateHelperColumnVector() to materialize each argument as a
   // ColumnVector, then aggregate over those.
-  return aggregateHelperColumnVector(handle, arguments, reuse_matches->get());
+  return aggregateHelperColumnVector(handle, arguments, filter);
 }
 
 void StorageBlock::aggregateGroupBy(
     const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
     const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const Predicate *predicate,
+    const TupleIdSequence *filter,
     AggregationStateHashTableBase *hash_table,
-    std::unique_ptr<TupleIdSequence> *reuse_matches,
     std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
   DCHECK_GT(group_by.size(), 0u)
       << "Called aggregateGroupBy() with zero GROUP BY expressions";
@@ -438,23 +417,7 @@ void StorageBlock::aggregateGroupBy(
   // this aggregate, as well as the GROUP BY expression values.
   ColumnVectorsValueAccessor temp_result;
   {
-    std::unique_ptr<ValueAccessor> accessor;
-    if (predicate) {
-      if (!*reuse_matches) {
-        // If there is a filter predicate that hasn't already been evaluated,
-        // evaluate it now and save the results for other aggregates on this
-        // same block.
-        reuse_matches->reset(getMatchesForPredicate(predicate));
-      }
-
-      // Create a filtered ValueAccessor that only iterates over predicate
-      // matches.
-      accessor.reset(tuple_store_->createValueAccessor(reuse_matches->get()));
-    } else {
-      // Create a ValueAccessor that iterates over all tuples in this block
-      accessor.reset(tuple_store_->createValueAccessor());
-    }
-
+    std::unique_ptr<ValueAccessor> accessor(tuple_store_->createValueAccessor(filter));
     attribute_id attr_id = 0;
 
     // First, put GROUP BY keys into 'temp_result'.
@@ -503,9 +466,8 @@ void StorageBlock::aggregateDistinct(
     const std::vector<std::unique_ptr<const Scalar>> &arguments,
     const std::vector<attribute_id> *arguments_as_attributes,
     const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const Predicate *predicate,
+    const TupleIdSequence *filter,
     AggregationStateHashTableBase *distinctify_hash_table,
-    std::unique_ptr<TupleIdSequence> *reuse_matches,
     std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
   DCHECK_GT(arguments.size(), 0u)
       << "Called aggregateDistinct() with zero argument expressions";
@@ -517,22 +479,7 @@ void StorageBlock::aggregateDistinct(
   // this aggregate, as well as the GROUP BY expression values.
   ColumnVectorsValueAccessor temp_result;
   {
-    std::unique_ptr<ValueAccessor> accessor;
-    if (predicate) {
-      if (!*reuse_matches) {
-        // If there is a filter predicate that hasn't already been evaluated,
-        // evaluate it now and save the results for other aggregates on this
-        // same block.
-        reuse_matches->reset(getMatchesForPredicate(predicate));
-      }
-
-      // Create a filtered ValueAccessor that only iterates over predicate
-      // matches.
-      accessor.reset(tuple_store_->createValueAccessor(reuse_matches->get()));
-    } else {
-      // Create a ValueAccessor that iterates over all tuples in this block
-      accessor.reset(tuple_store_->createValueAccessor());
-    }
+    std::unique_ptr<ValueAccessor> accessor(tuple_store_->createValueAccessor(filter));
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
     // If all the arguments to this aggregate are plain relation attributes,
@@ -1246,23 +1193,36 @@ bool StorageBlock::rebuildIndexes(bool short_circuit) {
   return all_indices_consistent_;
 }
 
-TupleIdSequence* StorageBlock::getMatchesForPredicate(const Predicate *predicate) const {
+TupleIdSequence* StorageBlock::getMatchesForPredicate(const Predicate *predicate,
+                                                      const TupleIdSequence *filter) const {
   if (predicate == nullptr) {
-    return tuple_store_->getExistenceMap();
+    TupleIdSequence *matches = tuple_store_->getExistenceMap();
+    if (filter != nullptr) {
+      matches->intersectWith(*filter);
+    }
+    return matches;
   }
 
   std::unique_ptr<ValueAccessor> value_accessor(tuple_store_->createValueAccessor());
-  std::unique_ptr<TupleIdSequence> existence_map;
-  if (!tuple_store_->isPacked()) {
-    existence_map.reset(tuple_store_->getExistenceMap());
-  }
   SubBlocksReference sub_blocks_ref(*tuple_store_,
                                     indices_,
                                     indices_consistent_);
-  return predicate->getAllMatches(value_accessor.get(),
-                                  &sub_blocks_ref,
-                                  nullptr,
-                                  existence_map.get());
+
+  if (!tuple_store_->isPacked()) {
+    std::unique_ptr<TupleIdSequence> existence_map(tuple_store_->getExistenceMap());
+    if (filter != nullptr) {
+      existence_map->intersectWith(*filter);
+    }
+    return predicate->getAllMatches(value_accessor.get(),
+                                    &sub_blocks_ref,
+                                    nullptr,
+                                    existence_map.get());
+  } else {
+    return predicate->getAllMatches(value_accessor.get(),
+                                    &sub_blocks_ref,
+                                    nullptr,
+                                    filter);
+  }
 }
 
 std::unordered_map<attribute_id, TypedValue>* StorageBlock::generateUpdatedValues(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/393eba55/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index bab5bab..488efcc 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -313,6 +313,17 @@ class StorageBlock : public StorageBlockBase {
       ValueAccessor *accessor);
 
   /**
+   * @brief Get the IDs of tuples in this StorageBlock which match a given Predicate.
+   *
+   * @param predicate The predicate to match.
+   * @param filter If non-NULL, then only tuple IDs which are set in the
+   *        filter will be checked (all others will be assumed to be false).
+   * @return A TupleIdSequence which contains matching tuple IDs for predicate.
+   **/
+  TupleIdSequence* getMatchesForPredicate(const Predicate *predicate,
+                                          const TupleIdSequence *filter = nullptr) const;
+
+  /**
    * @brief Perform a random sampling of data on  the StorageBlock. The number
    *       of records sampled is determined by the sample percentage in case of
    *       tuple sample. For block sample all the records in a block are taken.
@@ -336,8 +347,8 @@ class StorageBlock : public StorageBlockBase {
    *
    * @param selection A list of scalars, which will be evaluated to obtain
    *        attribute values for each result tuple.
-   * @param predicate A predicate for selection. NULL indicates that all tuples
-   *        should be matched.
+   * @param filter If non-NULL, then only tuple IDs which are set in the
+   *        filter will be checked (all others will be assumed to be false).
    * @param destination Where to insert the tuples resulting from the SELECT
    *        query.
    * @exception TupleTooLargeForBlock A tuple produced by this selection was
@@ -348,18 +359,18 @@ class StorageBlock : public StorageBlockBase {
    *
    **/
   void select(const std::vector<std::unique_ptr<const Scalar>> &selection,
-              const Predicate *predicate,
+              const TupleIdSequence *filter,
               InsertDestinationInterface *destination) const;
 
   /**
    * @brief Perform a simple SELECT query on this StorageBlock which only
    *        projects attributes and does not evaluate expressions.
    *
+   * @param selection The attributes to project.
+   * @param filter If non-NULL, then only tuple IDs which are set in the
+   *        filter will be checked (all others will be assumed to be false).
    * @param destination Where to insert the tuples resulting from the SELECT
    *        query.
-   * @param selection The attributes to project.
-   * @param predicate A predicate for selection. NULL indicates that all tuples
-   *        should be matched.
    * @exception TupleTooLargeForBlock A tuple produced by this selection was
    *            too large to insert into an empty block provided by
    *            destination. Selection may be partially complete (with some
@@ -371,7 +382,7 @@ class StorageBlock : public StorageBlockBase {
    *         an inconsistent IndexSubBlock (see indicesAreConsistent()).
    **/
   void selectSimple(const std::vector<attribute_id> &selection,
-                    const Predicate *predicate,
+                    const TupleIdSequence *filter,
                     InsertDestinationInterface *destination) const;
 
   /**
@@ -384,23 +395,8 @@ class StorageBlock : public StorageBlockBase {
    * @param arguments_as_attributes If non-NULL, indicates a valid attribute_id
    *        for each of the elements in arguments, and is used to elide a copy.
    *        Has no effect if NULL, or if VECTOR_COPY_ELISION_LEVEL is NONE.
-   * @param predicate A predicate for selection. nullptr indicates that all
-   *        tuples should be aggregated on.
-   * @param reuse_matches This parameter is used to store and reuse tuple-id
-   *        sequence of matches pre-computed in an earlier invocations to
-   *        aggregate(). \c reuse_matches is never \c nullptr for ease of use.
-   *        Current invocation of aggregate() will reuse TupleIdSequence if
-   *        passed, otherwise compute a TupleIdSequence based on \c predicate
-   *        and store in \c reuse_matches. We use std::unique_ptr for each of
-   *        use, since the caller will not have to selective free.
-   *
-   * For example, see this relevant pseudo-C++ code:
-   * \code
-   * std::unique_ptr<TupleIdSequence> matches;
-   * for each aggregate {
-   *   block.aggregate(..., &matches);
-   * }
-   * \endcode
+   * @param filter If non-NULL, then only tuple IDs which are set in the
+   *        filter will be checked (all others will be assumed to be false).
    *
    * @return Aggregated state for this block in the form of an
    *         AggregationState. AggregationHandle::mergeStates() can be called
@@ -412,8 +408,7 @@ class StorageBlock : public StorageBlockBase {
       const AggregationHandle &handle,
       const std::vector<std::unique_ptr<const Scalar>> &arguments,
       const std::vector<attribute_id> *arguments_as_attributes,
-      const Predicate *predicate,
-      std::unique_ptr<TupleIdSequence> *reuse_matches) const;
+      const TupleIdSequence *filter) const;
 
   /**
    * @brief Perform GROUP BY aggregation on the tuples in the this storage
@@ -423,18 +418,10 @@ class StorageBlock : public StorageBlockBase {
    * @param group_by The list of GROUP BY attributes/expressions. The tuples in
    *        this storage block are grouped by these attributes before
    *        aggregation.
-   * @param predicate A predicate for selection. nullptr indicates that all
-   *        tuples should be aggregated on.
+   * @param filter If non-NULL, then only tuple IDs which are set in the
+   *        filter will be checked (all others will be assumed to be false).
    * @param hash_table Hash table to store aggregation state mapped based on
    *        GROUP BY value list (defined by \c group_by).
-   * @param reuse_matches This parameter is used to store and reuse tuple-id
-   *        sequence of matches pre-computed in an earlier invocations of
-   *        aggregateGroupBy(). \c reuse_matches is never \c nullptr for ease of
-   *        use.  Current invocation of aggregateGroupBy() will reuse
-   *        TupleIdSequence if passed, otherwise computes a TupleIdSequence based
-   *        on \c predicate and stores in \c reuse_matches. We use
-   *        std::unique_ptr for each of use, since the caller will not have to
-   *        selective free.
    * @param reuse_group_by_vectors This parameter is used to store and reuse
    *        GROUP BY attribute vectors pre-computed in an earlier invocation of
    *        aggregateGroupBy(). \c reuse_group_by_vectors is never \c nullptr
@@ -444,10 +431,9 @@ class StorageBlock : public StorageBlockBase {
    *
    * For sample usage of aggregateGroupBy, see this relevant pseudo-C++ code:
    * \code
-   * std::unique_ptr<TupleIdSequence> matches;
    * std::vector<std::unique_ptr<ColumnVector>> group_by_vectors;
    * for each aggregate {
-   *   block.aggregateGroupBy(..., &matches, &group_by_vectors);
+   *   block.aggregateGroupBy(..., &group_by_vectors);
    * }
    * \endcode
    **/
@@ -461,9 +447,8 @@ class StorageBlock : public StorageBlockBase {
   void aggregateGroupBy(
       const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
       const std::vector<std::unique_ptr<const Scalar>> &group_by,
-      const Predicate *predicate,
+      const TupleIdSequence *filter,
       AggregationStateHashTableBase *hash_table,
-      std::unique_ptr<TupleIdSequence> *reuse_matches,
       std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
 
   /**
@@ -481,19 +466,11 @@ class StorageBlock : public StorageBlockBase {
    *        for each of the elements in arguments, and is used to elide a copy.
    *        Has no effect if NULL, or if VECTOR_COPY_ELISION_LEVEL is NONE.
    * @param group_by The list of GROUP BY attributes/expressions.
-   * @param predicate A predicate for selection. \c nullptr indicates that all
-   *        tuples should be aggregated on.
+   * @param filter If non-NULL, then only tuple IDs which are set in the
+   *        filter will be checked (all others will be assumed to be false).
    * @param distinctify_hash_table Hash table to store the arguments and GROUP
    *        BY expressions together as hash table key and a bool constant \c true
    *        as hash table value. (So the hash table actually serves as a hash set.)
-   * @param reuse_matches This parameter is used to store and reuse tuple-id
-   *        sequence of matches pre-computed in an earlier invocations of
-   *        aggregateGroupBy(). \c reuse_matches is never \c nullptr for ease of
-   *        use.  Current invocation of aggregateGroupBy() will reuse
-   *        TupleIdSequence if passed, otherwise computes a TupleIdSequence based
-   *        on \c predicate and stores in \c reuse_matches. We use
-   *        std::unique_ptr for each of use, since the caller will not have to
-   *        selective free.
    * @param reuse_group_by_vectors This parameter is used to store and reuse
    *        GROUP BY attribute vectors pre-computed in an earlier invocation of
    *        aggregateGroupBy(). \c reuse_group_by_vectors is never \c nullptr
@@ -505,9 +482,8 @@ class StorageBlock : public StorageBlockBase {
                          const std::vector<std::unique_ptr<const Scalar>> &arguments,
                          const std::vector<attribute_id> *arguments_as_attributes,
                          const std::vector<std::unique_ptr<const Scalar>> &group_by,
-                         const Predicate *predicate,
+                         const TupleIdSequence *filter,
                          AggregationStateHashTableBase *distinctify_hash_table,
-                         std::unique_ptr<TupleIdSequence> *reuse_matches,
                          std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
 
   /**
@@ -627,8 +603,6 @@ class StorageBlock : public StorageBlockBase {
   // StorageBlock's header.
   bool rebuildIndexes(bool short_circuit);
 
-  TupleIdSequence* getMatchesForPredicate(const Predicate *predicate) const;
-
   std::unordered_map<attribute_id, TypedValue>* generateUpdatedValues(
       const ValueAccessor &accessor,
       const tuple_id tuple,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/393eba55/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index 2adf674..5d70c86 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -132,9 +132,7 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
 
     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)));
+        edge_info.labels.emplace_back(attr->attribute_alias());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/393eba55/utility/lip_filter/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/lip_filter/CMakeLists.txt b/utility/lip_filter/CMakeLists.txt
index b7224d2..23b3763 100644
--- a/utility/lip_filter/CMakeLists.txt
+++ b/utility/lip_filter/CMakeLists.txt
@@ -25,6 +25,7 @@ add_library(quickstep_utility_lipfilter_LIPFilterAdaptiveProber ../../empty_src.
 add_library(quickstep_utility_lipfilter_LIPFilterBuilder ../../empty_src.cpp LIPFilterBuilder.hpp)
 add_library(quickstep_utility_lipfilter_LIPFilterDeployment LIPFilterDeployment.cpp LIPFilterDeployment.hpp)
 add_library(quickstep_utility_lipfilter_LIPFilterFactory LIPFilterFactory.cpp LIPFilterFactory.hpp)
+add_library(quickstep_utility_lipfilter_LIPFilterUtil ../../empty_src.cpp LIPFilterUtil.hpp)
 add_library(quickstep_utility_lipfilter_LIPFilter_proto
             ${utility_lipfilter_LIPFilter_proto_srcs})
 add_library(quickstep_utility_lipfilter_SingleIdentityHashFilter ../../empty_src.cpp SingleIdentityHashFilter.hpp)
@@ -58,6 +59,9 @@ target_link_libraries(quickstep_utility_lipfilter_LIPFilterFactory
                       quickstep_utility_lipfilter_LIPFilter_proto
                       quickstep_utility_lipfilter_SingleIdentityHashFilter
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_lipfilter_LIPFilterUtil
+                      quickstep_queryexecution_QueryContext
+                      quickstep_utility_lipfilter_LIPFilterDeployment)
 target_link_libraries(quickstep_utility_lipfilter_LIPFilter_proto
                       ${PROTOBUF_LIBRARY}
                       quickstep_types_Type_proto)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/393eba55/utility/lip_filter/LIPFilterBuilder.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilterBuilder.hpp b/utility/lip_filter/LIPFilterBuilder.hpp
index deb8f66..aa84a06 100644
--- a/utility/lip_filter/LIPFilterBuilder.hpp
+++ b/utility/lip_filter/LIPFilterBuilder.hpp
@@ -39,9 +39,6 @@ class ValueAccessor;
  *  @{
  */
 
-class LIPFilterBuilder;
-typedef std::shared_ptr<LIPFilterBuilder> LIPFilterBuilderPtr;
-
 /**
  * @brief Helper class for building LIPFilters from a relation (i.e. ValueAccessor).
  */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/393eba55/utility/lip_filter/LIPFilterUtil.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilterUtil.hpp b/utility/lip_filter/LIPFilterUtil.hpp
new file mode 100644
index 0000000..5d43c49
--- /dev/null
+++ b/utility/lip_filter/LIPFilterUtil.hpp
@@ -0,0 +1,79 @@
+/**
+ * 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_UTIL_HPP_
+#define QUICKSTEP_UTILITY_LIP_FILTER_LIP_FILTER_UTIL_HPP_
+
+#include "query_execution/QueryContext.hpp"
+#include "utility/lip_filter/LIPFilterDeployment.hpp"
+
+namespace quickstep {
+
+class LIPFilterBuilder;
+class LIPFilterAdaptiveProber;
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+/**
+ * @brief Create a LIPFilterBuilder for the given LIPFilterDeployment in QueryContext.
+ *
+ * @param lip_deployment_index The id of the LIPFilterDeployment in QueryContext.
+ * @param query_context The QueryContext.
+ * @return A LIPFilterBuilder object, or nullptr if \p lip_deployment_index is invalid.
+ */
+inline LIPFilterBuilder* CreateLIPFilterBuilderHelper(
+    const QueryContext::lip_deployment_id lip_deployment_index,
+    const QueryContext *query_context) {
+  if (lip_deployment_index == QueryContext::kInvalidLIPDeploymentId) {
+    return nullptr;
+  } else {
+    const LIPFilterDeployment *lip_filter_deployment =
+        query_context->getLIPDeployment(lip_deployment_index);
+    return lip_filter_deployment->createLIPFilterBuilder();
+  }
+}
+
+/**
+ * @brief Create a LIPFilterAdaptiveProber for the given LIPFilterDeployment
+ *        in QueryContext.
+ *
+ * @param lip_deployment_index The id of the LIPFilterDeployment in QueryContext.
+ * @param query_context The QueryContext.
+ * @return A LIPFilterAdaptiveProber object, or nullptr if \p lip_deployment_index
+ *         is invalid.
+ */
+inline LIPFilterAdaptiveProber* CreateLIPFilterAdaptiveProberHelper(
+    const QueryContext::lip_deployment_id lip_deployment_index,
+    const QueryContext *query_context) {
+  if (lip_deployment_index == QueryContext::kInvalidLIPDeploymentId) {
+    return nullptr;
+  } else {
+    const LIPFilterDeployment *lip_filter_deployment =
+        query_context->getLIPDeployment(lip_deployment_index);
+    return lip_filter_deployment->createLIPFilterAdaptiveProber();
+  }
+}
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_LIP_FILTER_LIP_FILTER_UTIL_HPP_